From 6e296d034272503acef51a979d28cacc1e5a6a2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Oct 2021 13:44:52 +0300 Subject: [PATCH 001/582] refactor ctor arguments --- src/Common/ZooKeeper/TestKeeper.cpp | 18 +-- src/Common/ZooKeeper/TestKeeper.h | 8 +- src/Common/ZooKeeper/ZooKeeper.cpp | 156 +++++----------------- src/Common/ZooKeeper/ZooKeeper.h | 27 +--- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 71 ++++++++++ src/Common/ZooKeeper/ZooKeeperArgs.h | 37 +++++ src/Common/ZooKeeper/ZooKeeperConstants.h | 1 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 77 +++++------ src/Common/ZooKeeper/ZooKeeperImpl.h | 13 +- utils/keeper-bench/Runner.cpp | 14 +- utils/zookeeper-cli/zookeeper-cli.cpp | 2 +- utils/zookeeper-test/main.cpp | 2 +- 12 files changed, 203 insertions(+), 223 deletions(-) create mode 100644 src/Common/ZooKeeper/ZooKeeperArgs.cpp create mode 100644 src/Common/ZooKeeper/ZooKeeperArgs.h diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 065b1cf65ba..ee9dfad82b0 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -474,15 +474,15 @@ ResponsePtr TestKeeperCheckRequest::createResponse() const { return std::make_sh ResponsePtr TestKeeperMultiRequest::createResponse() const { return std::make_shared(); } -TestKeeper::TestKeeper(const String & root_path_, Poco::Timespan operation_timeout_) - : root_path(root_path_), operation_timeout(operation_timeout_) +TestKeeper::TestKeeper(const zkutil::ZooKeeperArgs & args_) + : args(args_) { container.emplace("/", Node()); - if (!root_path.empty()) + if (!args.chroot.empty()) { - if (root_path.back() == '/') - root_path.pop_back(); + if (args.chroot.back() == '/') + args.chroot.pop_back(); } processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); @@ -514,7 +514,7 @@ void TestKeeper::processingThread() { RequestInfo info; - UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + UInt64 max_wait = UInt64(args.operation_timeout_ms); if (requests_queue.tryPop(info, max_wait)) { if (expired) @@ -523,7 +523,7 @@ void TestKeeper::processingThread() ++zxid; - info.request->addRootPath(root_path); + info.request->addRootPath(args.chroot); auto [response, _] = info.request->process(container, zxid); if (info.watch) @@ -547,7 +547,7 @@ void TestKeeper::processingThread() if (response->error == Error::ZOK) info.request->processWatches(watches, list_watches); - response->removeRootPath(root_path); + response->removeRootPath(args.chroot); if (info.callback) info.callback(*response); } @@ -656,7 +656,7 @@ void TestKeeper::pushRequest(RequestInfo && request) if (expired) throw Exception("Session expired", Error::ZSESSIONEXPIRED); - if (!requests_queue.tryPush(std::move(request), operation_timeout.totalMilliseconds())) + if (!requests_queue.tryPush(std::move(request), args.operation_timeout_ms)) throw Exception("Cannot push request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); } catch (...) diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index e57471341e8..1ded9e706c6 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -33,7 +34,7 @@ using TestKeeperRequestPtr = std::shared_ptr; class TestKeeper final : public IKeeper { public: - TestKeeper(const String & root_path_, Poco::Timespan operation_timeout_); + TestKeeper(const zkutil::ZooKeeperArgs & args_); ~TestKeeper() override; bool isExpired() const override { return expired; } @@ -113,10 +114,7 @@ private: Container container; - String root_path; - ACLs default_acls; - - Poco::Timespan operation_timeout; + zkutil::ZooKeeperArgs args; std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 3d505c088db..fe70f13293e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -8,7 +8,6 @@ #include #include -#include #include #include #include @@ -16,8 +15,6 @@ #include -#define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000 - namespace fs = std::filesystem; namespace DB @@ -46,26 +43,20 @@ static void check(Coordination::Error code, const std::string & path) } -void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, - int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_) +void ZooKeeper::init(ZooKeeperArgs args_) { + args = std::move(args_); log = &Poco::Logger::get("ZooKeeper"); - hosts = hosts_; - identity = identity_; - session_timeout_ms = session_timeout_ms_; - operation_timeout_ms = operation_timeout_ms_; - chroot = chroot_; - implementation = implementation_; - if (implementation == "zookeeper") + if (args.implementation == "zookeeper") { - if (hosts.empty()) + if (args.hosts.empty()) throw KeeperException("No hosts passed to ZooKeeper constructor.", Coordination::Error::ZBADARGUMENTS); Coordination::ZooKeeper::Nodes nodes; - nodes.reserve(hosts.size()); + nodes.reserve(args.hosts.size()); - Strings shuffled_hosts = hosts; + Strings shuffled_hosts = args.hosts; /// Shuffle the hosts to distribute the load among ZooKeeper nodes. pcg64 generator(randomSeed()); std::shuffle(shuffled_hosts.begin(), shuffled_hosts.end(), generator); @@ -104,33 +95,23 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ throw KeeperException("Cannot use any of provided ZooKeeper nodes", Coordination::Error::ZBADARGUMENTS); } - impl = std::make_unique( - nodes, - chroot, - identity_.empty() ? "" : "digest", - identity_, - Poco::Timespan(0, session_timeout_ms_ * 1000), - Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000), - Poco::Timespan(0, operation_timeout_ms_ * 1000), - zk_log); + impl = std::make_unique(nodes, args, zk_log); - if (chroot.empty()) - LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(hosts, ",")); + if (args.chroot.empty()) + LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); else - LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(hosts, ","), chroot); + LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), chroot); } - else if (implementation == "testkeeper") + else if (args.implementation == "testkeeper") { - impl = std::make_unique( - chroot, - Poco::Timespan(0, operation_timeout_ms_ * 1000)); + impl = std::make_unique(args); } else { - throw DB::Exception("Unknown implementation of coordination service: " + implementation, DB::ErrorCodes::NOT_IMPLEMENTED); + throw DB::Exception("Unknown implementation of coordination service: " + args.implementation, DB::ErrorCodes::NOT_IMPLEMENTED); } - if (!chroot.empty()) + if (!args.chroot.empty()) { /// Here we check that zk root exists. /// This check is clumsy. The reason is we do this request under common mutex, and never want to hung here. @@ -140,7 +121,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ /// This should not happen now, when memory tracker is disabled. /// But let's keep it just in case (it is also easy to backport). auto future = asyncExists("/"); - auto res = future.wait_for(std::chrono::milliseconds(operation_timeout_ms)); + auto res = future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)); if (res != std::future_status::ready) throw KeeperException("Cannot check if zookeeper root exists.", Coordination::Error::ZOPERATIONTIMEOUT); @@ -149,107 +130,32 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ throw KeeperException(code, "/"); if (code == Coordination::Error::ZNONODE) - throw KeeperException("ZooKeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::Error::ZNONODE); + throw KeeperException("ZooKeeper root doesn't exist. You should create root node " + args.chroot + " before start.", Coordination::Error::ZNONODE); } } -ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_, - int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_, - std::shared_ptr zk_log_) + +ZooKeeper::ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_) { zk_log = std::move(zk_log_); - Strings hosts_strings; - splitInto<','>(hosts_strings, hosts_string); - - init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); + init(args_); } -ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, - int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_, - std::shared_ptr zk_log_) -{ - zk_log = std::move(zk_log_); - init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); -} - -struct ZooKeeperArgs -{ - ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) - { - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_name, keys); - - session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; - operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; - implementation = "zookeeper"; - for (const auto & key : keys) - { - if (startsWith(key, "node")) - { - hosts.push_back( - (config.getBool(config_name + "." + key + ".secure", false) ? "secure://" : "") + - config.getString(config_name + "." + key + ".host") + ":" - + config.getString(config_name + "." + key + ".port", "2181") - ); - } - else if (key == "session_timeout_ms") - { - session_timeout_ms = config.getInt(config_name + "." + key); - } - else if (key == "operation_timeout_ms") - { - operation_timeout_ms = config.getInt(config_name + "." + key); - } - else if (key == "identity") - { - identity = config.getString(config_name + "." + key); - } - else if (key == "root") - { - chroot = config.getString(config_name + "." + key); - } - else if (key == "implementation") - { - implementation = config.getString(config_name + "." + key); - } - else - throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); - } - - if (!chroot.empty()) - { - if (chroot.front() != '/') - throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot, Coordination::Error::ZBADARGUMENTS); - if (chroot.back() == '/') - chroot.pop_back(); - } - } - - Strings hosts; - std::string identity; - int session_timeout_ms; - int operation_timeout_ms; - std::string chroot; - std::string implementation; -}; - ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, std::shared_ptr zk_log_) : zk_log(std::move(zk_log_)) { - ZooKeeperArgs args(config, config_name); - init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot); + init(ZooKeeperArgs(config, config_name)); } bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) const { - ZooKeeperArgs args(config, config_name); + ZooKeeperArgs new_args(config, config_name); // skip reload testkeeper cause it's for test and data in memory - if (args.implementation == implementation && implementation == "testkeeper") + if (new_args.implementation == args.implementation && args.implementation == "testkeeper") return false; - return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot) - != std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot); + return args != new_args; } @@ -267,7 +173,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings { auto future_result = asyncTryGetChildrenNoThrow(path, watch_callback); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::List), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -328,7 +234,7 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s { auto future_result = asyncTryCreateNoThrow(path, data, mode); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -398,7 +304,7 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers auto future_result = asyncTryRemoveNoThrow(path, version); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Remove), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -430,7 +336,7 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination { auto future_result = asyncTryExistsNoThrow(path, watch_callback); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Exists), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -464,7 +370,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r { auto future_result = asyncTryGetNoThrow(path, watch_callback); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Get), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -537,7 +443,7 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri { auto future_result = asyncTrySetNoThrow(path, data, version); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Set), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -589,7 +495,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests auto future_result = asyncTryMultiNoThrow(requests); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Multi), requests[0]->getPath())); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -752,7 +658,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & ZooKeeperPtr ZooKeeper::startNewSession() const { - return std::make_shared(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log); + return std::make_shared(args, zk_log); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 8e015b1f331..9a022c947ac 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -50,22 +51,10 @@ constexpr size_t MULTI_BATCH_SIZE = 100; class ZooKeeper { public: + using Ptr = std::shared_ptr; - /// hosts_string -- comma separated [secure://]host:port list - ZooKeeper(const std::string & hosts_string, const std::string & identity_ = "", - int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, - int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, - const std::string & chroot_ = "", - const std::string & implementation_ = "zookeeper", - std::shared_ptr zk_log_ = nullptr); - - ZooKeeper(const Strings & hosts_, const std::string & identity_ = "", - int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, - int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, - const std::string & chroot_ = "", - const std::string & implementation_ = "zookeeper", - std::shared_ptr zk_log_ = nullptr); + ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); /** Config of the form: @@ -283,8 +272,7 @@ public: private: friend class EphemeralNodeHolder; - void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, - int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_); + void init(ZooKeeperArgs args_); /// The following methods don't any throw exceptions but return error codes. Coordination::Error createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); @@ -299,12 +287,7 @@ private: std::unique_ptr impl; - Strings hosts; - std::string identity; - int32_t session_timeout_ms; - int32_t operation_timeout_ms; - std::string chroot; - std::string implementation; + ZooKeeperArgs args; std::mutex mutex; diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp new file mode 100644 index 00000000000..a8010fdad13 --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -0,0 +1,71 @@ +#include +#include +#include +#include + +namespace zkutil +{ + +ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, const String & config_name) +{ + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_name, keys); + + for (const auto & key : keys) + { + if (key.starts_with("node")) + { + hosts.push_back( + (config.getBool(config_name + "." + key + ".secure", false) ? "secure://" : "") + + config.getString(config_name + "." + key + ".host") + ":" + config.getString(config_name + "." + key + ".port", "2181")); + } + else if (key == "session_timeout_ms") + { + session_timeout_ms = config.getInt(config_name + "." + key); + } + else if (key == "operation_timeout_ms") + { + operation_timeout_ms = config.getInt(config_name + "." + key); + } + else if (key == "connection_timeout_ms") + { + connection_timeout_ms = config.getInt(config_name + "." + key); + } + else if (key == "session_fault_probability") + { + session_fault_probability = config.getDouble(config_name + "." + key); + } + else if (key == "identity") + { + identity = config.getString(config_name + "." + key); + if (!identity.empty()) + auth_scheme = "digest"; + } + else if (key == "root") + { + chroot = config.getString(config_name + "." + key); + } + else if (key == "implementation") + { + implementation = config.getString(config_name + "." + key); + } + else + throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); + } + + if (!chroot.empty()) + { + if (chroot.front() != '/') + throw KeeperException( + std::string("Root path in config file should start with '/', but got ") + chroot, Coordination::Error::ZBADARGUMENTS); + if (chroot.back() == '/') + chroot.pop_back(); + } +} + +ZooKeeperArgs::ZooKeeperArgs(const String & hosts_string) +{ + splitInto<','>(hosts, hosts_string); +} + +} diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h new file mode 100644 index 00000000000..826dd12bbb7 --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -0,0 +1,37 @@ +#pragma once +#include +#include + +namespace Poco +{ +namespace Util +{ + class AbstractConfiguration; +} +} + +namespace zkutil +{ + +struct ZooKeeperArgs +{ + ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, const String & config_name); + + /// hosts_string -- comma separated [secure://]host:port list + ZooKeeperArgs(const String & hosts_string); + ZooKeeperArgs() = default; + bool operator == (const ZooKeeperArgs &) const = default; + bool operator != (const ZooKeeperArgs &) const = default; + + String implementation; + Strings hosts; + String auth_scheme; + String identity; + String chroot; + int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; + int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; + int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; + float session_fault_probability = 0; +}; + +} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index ed7afd83628..3fca25bad43 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -49,5 +49,6 @@ static constexpr int32_t PASSWORD_LENGTH = 16; static constexpr int32_t MAX_STRING_OR_ARRAY_SIZE = 1 << 28; /// 256 MiB static constexpr int32_t DEFAULT_SESSION_TIMEOUT_MS = 30000; static constexpr int32_t DEFAULT_OPERATION_TIMEOUT_MS = 10000; +static constexpr int32_t DEFAULT_CONNECTION_TIMEOUT_MS = 1000; } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 63b4a61b866..e77231c2e9f 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -274,15 +274,15 @@ void ZooKeeper::read(T & x) Coordination::read(x, *in); } -static void removeRootPath(String & path, const String & root_path) +static void removeRootPath(String & path, const String & chroot) { - if (root_path.empty()) + if (chroot.empty()) return; - if (path.size() <= root_path.size()) - throw Exception("Received path is not longer than root_path", Error::ZDATAINCONSISTENCY); + if (path.size() <= chroot.size()) + throw Exception("Received path is not longer than chroot", Error::ZDATAINCONSISTENCY); - path = path.substr(root_path.size()); + path = path.substr(chroot.size()); } ZooKeeper::~ZooKeeper() @@ -306,27 +306,20 @@ ZooKeeper::~ZooKeeper() ZooKeeper::ZooKeeper( const Nodes & nodes, - const String & root_path_, - const String & auth_scheme, - const String & auth_data, - Poco::Timespan session_timeout_, - Poco::Timespan connection_timeout, - Poco::Timespan operation_timeout_, + const zkutil::ZooKeeperArgs & args_, std::shared_ptr zk_log_) - : root_path(root_path_), - session_timeout(session_timeout_), - operation_timeout(std::min(operation_timeout_, session_timeout_)) + : args(args_) { log = &Poco::Logger::get("ZooKeeperClient"); std::atomic_store(&zk_log, std::move(zk_log_)); - if (!root_path.empty()) + if (!args.chroot.empty()) { - if (root_path.back() == '/') - root_path.pop_back(); + if (args.chroot.back() == '/') + args.chroot.pop_back(); } - if (auth_scheme.empty()) + if (args.auth_scheme.empty()) { ACL acl; acl.permissions = ACL::All; @@ -343,10 +336,10 @@ ZooKeeper::ZooKeeper( default_acls.emplace_back(std::move(acl)); } - connect(nodes, connection_timeout); + connect(nodes, args.connection_timeout_ms * 1000); - if (!auth_scheme.empty()) - sendAuth(auth_scheme, auth_data); + if (!args.auth_scheme.empty()) + sendAuth(args.auth_scheme, args.identity); send_thread = ThreadFromGlobalPool([this] { sendThread(); }); receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); @@ -390,8 +383,8 @@ void ZooKeeper::connect( socket.connect(node.address, connection_timeout); socket_address = socket.peerAddress(); - socket.setReceiveTimeout(operation_timeout); - socket.setSendTimeout(operation_timeout); + socket.setReceiveTimeout(args.operation_timeout_ms); + socket.setSendTimeout(args.operation_timeout_ms); socket.setNoDelay(true); in.emplace(socket); @@ -462,7 +455,7 @@ void ZooKeeper::sendHandshake() { int32_t handshake_length = 44; int64_t last_zxid_seen = 0; - int32_t timeout = session_timeout.totalMilliseconds(); + int32_t timeout = args.session_timeout_ms; int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero. constexpr int32_t passwd_len = 16; std::array passwd {}; @@ -494,9 +487,9 @@ void ZooKeeper::receiveHandshake() throw Exception("Unexpected protocol version: " + DB::toString(protocol_version_read), Error::ZMARSHALLINGERROR); read(timeout); - if (timeout != session_timeout.totalMilliseconds()) + if (timeout != args.session_timeout_ms) /// Use timeout from server. - session_timeout = timeout * Poco::Timespan::MILLISECONDS; + args.session_timeout_ms = timeout * 1000; read(session_id); read(passwd); @@ -550,14 +543,14 @@ void ZooKeeper::sendThread() auto prev_bytes_sent = out->count(); auto now = clock::now(); - auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3); + auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(args.session_timeout_ms / 3); if (next_heartbeat_time > now) { /// Wait for the next request in queue. No more than operation timeout. No more than until next heartbeat time. UInt64 max_wait = std::min( UInt64(std::chrono::duration_cast(next_heartbeat_time - now).count()), - UInt64(operation_timeout.totalMilliseconds())); + UInt64(args.operation_timeout_ms)); RequestInfo info; if (requests_queue.tryPop(info, max_wait)) @@ -582,7 +575,7 @@ void ZooKeeper::sendThread() break; } - info.request->addRootPath(root_path); + info.request->addRootPath(args.chroot); info.request->probably_sent = true; info.request->write(*out); @@ -621,13 +614,13 @@ void ZooKeeper::receiveThread() try { - Int64 waited = 0; + Int64 waited_us = 0; while (!requests_queue.isFinished()) { auto prev_bytes_received = in->count(); clock::time_point now = clock::now(); - UInt64 max_wait = operation_timeout.totalMicroseconds(); + UInt64 max_wait_us = args.operation_timeout_ms; std::optional earliest_operation; { @@ -636,20 +629,20 @@ void ZooKeeper::receiveThread() { /// Operations are ordered by xid (and consequently, by time). earliest_operation = operations.begin()->second; - auto earliest_operation_deadline = earliest_operation->time + std::chrono::microseconds(operation_timeout.totalMicroseconds()); + auto earliest_operation_deadline = earliest_operation->time + std::chrono::microseconds(args.operation_timeout_ms * 1000); if (now > earliest_operation_deadline) throw Exception("Operation timeout (deadline already expired) for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); - max_wait = std::chrono::duration_cast(earliest_operation_deadline - now).count(); + max_wait_us = std::chrono::duration_cast(earliest_operation_deadline - now).count(); } } - if (in->poll(max_wait)) + if (in->poll(max_wait_us)) { if (requests_queue.isFinished()) break; receiveEvent(); - waited = 0; + waited_us = 0; } else { @@ -657,8 +650,8 @@ void ZooKeeper::receiveThread() { throw Exception("Operation timeout (no response) for request " + toString(earliest_operation->request->getOpNum()) + " for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); } - waited += max_wait; - if (waited >= session_timeout.totalMicroseconds()) + waited_us += max_wait_us; + if (waited_us >= args.session_timeout_ms * 1000) throw Exception("Nothing is received in session timeout", Error::ZOPERATIONTIMEOUT); } @@ -768,7 +761,7 @@ void ZooKeeper::receiveEvent() else { response->readImpl(*in); - response->removeRootPath(root_path); + response->removeRootPath(args.chroot); } /// Instead of setting the watch in sendEvent, set it in receiveEvent because need to check the response. /// The watch shouldn't be set if the node does not exist and it will never exist like sequential ephemeral nodes. @@ -788,9 +781,9 @@ void ZooKeeper::receiveEvent() { CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); - /// The key of wathces should exclude the root_path + /// The key of wathces should exclude the args.chroot String req_path = request_info.request->getPath(); - removeRootPath(req_path, root_path); + removeRootPath(req_path, args.chroot); std::lock_guard lock(watches_mutex); watches[req_path].emplace_back(std::move(request_info.watch)); } @@ -1026,7 +1019,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) } } - if (!requests_queue.tryPush(std::move(info), operation_timeout.totalMilliseconds())) + if (!requests_queue.tryPush(std::move(info), args.operation_timeout_ms)) { if (requests_queue.isFinished()) throw Exception("Session expired", Error::ZSESSIONEXPIRED); @@ -1211,7 +1204,7 @@ void ZooKeeper::close() RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); - if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) + if (!requests_queue.tryPush(std::move(request_info), args.operation_timeout_ms)) throw Exception("Cannot push close request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); ProfileEvents::increment(ProfileEvents::ZooKeeperClose); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index b87469bd339..f49fa8ee829 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -109,12 +110,7 @@ public: */ ZooKeeper( const Nodes & nodes, - const String & root_path, - const String & auth_scheme, - const String & auth_data, - Poco::Timespan session_timeout_, - Poco::Timespan connection_timeout, - Poco::Timespan operation_timeout_, + const zkutil::ZooKeeperArgs & args_, std::shared_ptr zk_log_); ~ZooKeeper() override; @@ -192,11 +188,10 @@ public: void setZooKeeperLog(std::shared_ptr zk_log_); private: - String root_path; ACLs default_acls; - Poco::Timespan session_timeout; - Poco::Timespan operation_timeout; + zkutil::ZooKeeperArgs args; + Poco::Net::StreamSocket socket; /// To avoid excessive getpeername(2) calls. diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 1c8deeca476..ac57e59428a 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -174,15 +174,11 @@ std::vector> Runner::getConnections() Coordination::ZooKeeper::Node node{Poco::Net::SocketAddress{host_string}, false}; std::vector nodes; nodes.push_back(node); - zookeepers.emplace_back(std::make_shared( - nodes, - "", /*chroot*/ - "", /*identity type*/ - "", /*identity*/ - Poco::Timespan(0, 30000 * 1000), - Poco::Timespan(0, 1000 * 1000), - Poco::Timespan(0, 10000 * 1000), - nullptr)); + zkutil::ZooKeeperArgs args; + args.session_timeout_ms = 30000; + args.connection_timeout_ms = 1000; + args.operation_timeout_ms = 10000; + zookeepers.emplace_back(std::make_shared(nodes, args, nullptr)); } return zookeepers; diff --git a/utils/zookeeper-cli/zookeeper-cli.cpp b/utils/zookeeper-cli/zookeeper-cli.cpp index 690a9d6b097..47899bce34f 100644 --- a/utils/zookeeper-cli/zookeeper-cli.cpp +++ b/utils/zookeeper-cli/zookeeper-cli.cpp @@ -69,7 +69,7 @@ int main(int argc, char ** argv) Poco::Logger::root().setChannel(channel); Poco::Logger::root().setLevel("trace"); - zkutil::ZooKeeper zk(argv[1]); + zkutil::ZooKeeper zk{zkutil::ZooKeeperArgs(argv[1])}; LineReader lr({}, false, {"\\"}, {}); do diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp index b60c45c9df6..e43a7d4e9bf 100644 --- a/utils/zookeeper-test/main.cpp +++ b/utils/zookeeper-test/main.cpp @@ -334,7 +334,7 @@ int main(int argc, char *argv[]) Poco::Logger::root().setChannel(channel); Poco::Logger::root().setLevel("trace"); - zkutil::ZooKeeper zk(argv[1]); + zkutil::ZooKeeper zk{zkutil::ZooKeeperArgs(argv[1])}; try { From 27ca9438632c4ecc972371b5d0135a370a2aafce Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Oct 2021 16:59:13 +0300 Subject: [PATCH 002/582] add fault injection in ZooKeeper client --- docker/test/stress/run.sh | 1 + src/Common/ZooKeeper/ZooKeeperArgs.cpp | 8 ++++++-- src/Common/ZooKeeper/ZooKeeperArgs.h | 5 +++-- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 18 ++++++++++++++++++ src/Common/ZooKeeper/ZooKeeperImpl.h | 3 +++ .../config.d/zookeeper_fault_injection.xml | 19 +++++++++++++++++++ tests/config/install.sh | 8 +++++++- 7 files changed, 57 insertions(+), 5 deletions(-) create mode 100644 tests/config/config.d/zookeeper_fault_injection.xml diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index afdc026732f..770242f8933 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -34,6 +34,7 @@ dpkg -i package_folder/clickhouse-test_*.deb function configure() { + export ZOOKEEPER_FAULT_INJECTION=1 # install test configs /usr/share/clickhouse-test/config/install.sh diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index a8010fdad13..97ee50d8a3e 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -31,9 +31,13 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c { connection_timeout_ms = config.getInt(config_name + "." + key); } - else if (key == "session_fault_probability") + else if (key == "send_fault_probability") { - session_fault_probability = config.getDouble(config_name + "." + key); + send_fault_probability = config.getDouble(config_name + "." + key); + } + else if (key == "recv_fault_probability") + { + recv_fault_probability = config.getDouble(config_name + "." + key); } else if (key == "identity") { diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index 826dd12bbb7..d2d597eae83 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -23,7 +23,7 @@ struct ZooKeeperArgs bool operator == (const ZooKeeperArgs &) const = default; bool operator != (const ZooKeeperArgs &) const = default; - String implementation; + String implementation = "zookeeper"; Strings hosts; String auth_scheme; String identity; @@ -31,7 +31,8 @@ struct ZooKeeperArgs int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; - float session_fault_probability = 0; + float send_fault_probability = 0; + float recv_fault_probability = 0; }; } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index e77231c2e9f..aac9d957468 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -336,6 +336,18 @@ ZooKeeper::ZooKeeper( default_acls.emplace_back(std::move(acl)); } + + /// It makes sense (especially, for async requests) to inject a fault in two places: + /// pushRequest (before request is sent) and receiveEvent (after request was executed). + if (0 < args.send_fault_probability && args.send_fault_probability <= 1) + { + send_inject_fault.emplace(args.send_fault_probability); + } + if (0 < args.recv_fault_probability && args.recv_fault_probability <= 1) + { + recv_inject_fault.emplace(args.recv_fault_probability); + } + connect(nodes, args.connection_timeout_ms * 1000); if (!args.auth_scheme.empty()) @@ -683,6 +695,9 @@ void ZooKeeper::receiveEvent() RequestInfo request_info; ZooKeeperResponsePtr response; + if (unlikely(recv_inject_fault) && recv_inject_fault.value()(thread_local_rng)) + throw Exception("Session expired (fault injected)", Error::ZSESSIONEXPIRED); + if (xid == PING_XID) { if (err != Error::ZOK) @@ -1019,6 +1034,9 @@ void ZooKeeper::pushRequest(RequestInfo && info) } } + if (unlikely(send_inject_fault) && send_inject_fault.value()(thread_local_rng)) + throw Exception("Session expired (fault injected)", Error::ZSESSIONEXPIRED); + if (!requests_queue.tryPush(std::move(info), args.operation_timeout_ms)) { if (requests_queue.isFinished()) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index f49fa8ee829..c7779e1bd55 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -26,6 +26,7 @@ #include #include #include +#include /** ZooKeeper C++ library, a replacement for libzookeeper. @@ -192,6 +193,8 @@ private: zkutil::ZooKeeperArgs args; + std::optional send_inject_fault; + std::optional recv_inject_fault; Poco::Net::StreamSocket socket; /// To avoid excessive getpeername(2) calls. diff --git a/tests/config/config.d/zookeeper_fault_injection.xml b/tests/config/config.d/zookeeper_fault_injection.xml new file mode 100644 index 00000000000..45d3cc8193d --- /dev/null +++ b/tests/config/config.d/zookeeper_fault_injection.xml @@ -0,0 +1,19 @@ + + + + localhost + 9181 + + + + 0.00002 + 0.00002 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index ba6ba0cd07c..a0e6e8a89c2 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -15,7 +15,6 @@ mkdir -p $DEST_SERVER_PATH/config.d/ mkdir -p $DEST_SERVER_PATH/users.d/ mkdir -p $DEST_CLIENT_PATH -ln -sf $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/part_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ @@ -72,6 +71,13 @@ ln -sf $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/ ln -sf --backup=simple --suffix=_original.xml \ $SRC_PATH/config.d/query_masking_rules.xml $DEST_SERVER_PATH/config.d/ + +if [[ -n "$ZOOKEEPER_FAULT_INJECTION" ]] && [[ "$ZOOKEEPER_FAULT_INJECTION" -eq 1 ]]; then + ln -sf $SRC_PATH/config.d/zookeeper_fault_injection.xml $DEST_SERVER_PATH/config.d/ +else + ln -sf $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/ +fi + if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -sf $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ fi From f0270ad5d9c9719568ee6bac45a360dd0f8416d7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Oct 2021 13:16:37 +0300 Subject: [PATCH 003/582] minor fix --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 97ee50d8a3e..6629f8fba33 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -65,6 +65,9 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c if (chroot.back() == '/') chroot.pop_back(); } + + if (session_timeout_ms < 0 || operation_timeout_ms < 0 || connection_timeout_ms < 0) + throw KeeperException("Timeout cannot be negative", Coordination::Error::ZBADARGUMENTS); } ZooKeeperArgs::ZooKeeperArgs(const String & hosts_string) From 4b00c026360cf9d50d29aed76197165cb79375d5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Oct 2021 19:17:49 +0300 Subject: [PATCH 004/582] fix --- src/Common/ZooKeeper/examples/zkutil_test_async.cpp | 2 +- src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp | 3 ++- src/Common/ZooKeeper/examples/zookeeper_impl.cpp | 3 ++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/examples/zkutil_test_async.cpp b/src/Common/ZooKeeper/examples/zkutil_test_async.cpp index 17258c529ff..eafa0e27691 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_async.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_async.cpp @@ -5,7 +5,7 @@ int main(int argc, char ** argv) try { - zkutil::ZooKeeper zookeeper{"localhost:2181"}; + zkutil::ZooKeeper zookeeper{zkutil::ZooKeeperArgs("localhost:2181")}; auto nodes = zookeeper.getChildren("/tmp"); diff --git a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp index cd0f9f024fd..e770faefe1f 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp @@ -40,7 +40,8 @@ try } - ZooKeeper zk(nodes, {}, {}, {}, {5, 0}, {0, 50000}, {0, 50000}, nullptr); + zkutil::ZooKeeperArgs args; + ZooKeeper zk(nodes, args, nullptr); Poco::Event event(true); diff --git a/src/Common/ZooKeeper/examples/zookeeper_impl.cpp b/src/Common/ZooKeeper/examples/zookeeper_impl.cpp index e6ba4fe2a30..7754e239132 100644 --- a/src/Common/ZooKeeper/examples/zookeeper_impl.cpp +++ b/src/Common/ZooKeeper/examples/zookeeper_impl.cpp @@ -5,7 +5,8 @@ int main() try { - Coordination::ZooKeeper zookeeper({Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{"localhost:2181"}, false}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000}, nullptr); + zkutil::ZooKeeperArgs args; + Coordination::ZooKeeper zookeeper({Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{"localhost:2181"}, false}}, args, nullptr); zookeeper.create("/test", "hello", false, false, {}, [](const Coordination::CreateResponse & response) { From c2af5c87fa3f5980e80e00104808d436b5632ea8 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 26 Oct 2021 00:09:52 +0300 Subject: [PATCH 005/582] Update zkutil_test_commands.cpp --- src/Common/ZooKeeper/examples/zkutil_test_commands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/examples/zkutil_test_commands.cpp b/src/Common/ZooKeeper/examples/zkutil_test_commands.cpp index 490c834eab9..095a0dde2e7 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_commands.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_commands.cpp @@ -16,7 +16,7 @@ try return 1; } - ZooKeeper zk(argv[1], "", 5000); + ZooKeeper zk{zkutil::ZooKeeperArgs(argv[1])}; std::cout << "create path" << std::endl; zk.create("/test", "old", zkutil::CreateMode::Persistent); From e08f5c3d86e7dad499dc77c1fd9e3de4063bfbdc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Mar 2022 11:08:33 +0100 Subject: [PATCH 006/582] Add s3 tests with debug and tsan --- .github/workflows/pull_request.yml | 72 ++++++++++++++++++++++++++++++ tests/ci/ci_config.py | 6 +++ 2 files changed, 78 insertions(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 4a3880543c4..90b551fccd5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1221,6 +1221,76 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebugS3: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsanS3: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" FunctionalStatelessTestAarch64: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] @@ -3044,6 +3114,8 @@ jobs: - FunctionalStatefulTestMsan - FunctionalStatefulTestUBsan - FunctionalStatelessTestReleaseS3 + - FunctionalStatelessTestDebugS3 + - FunctionalStatelessTestTsanS3 - StressTestDebug - StressTestAsan - StressTestTsan diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b45a4ce90c6..a407911b56a 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -271,6 +271,12 @@ CI_CONFIG = { "Stateless tests (release, s3 storage, actions)": { "required_build": "package_release", }, + "Stateless tests (debug, s3 storage, actions)": { + "required_build": "package_debug", + }, + "Stateless tests (thread, s3 storage, actions)": { + "required_build": "package_tsan", + }, "Stress test (address, actions)": { "required_build": "package_asan", }, From 6a2620a073bd1de721f81e6573560df9653fa059 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 31 Mar 2022 17:17:16 +0300 Subject: [PATCH 007/582] Update ZooKeeperArgs.cpp --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 3723b0bd056..1d644bb6a24 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -6,6 +6,14 @@ #include #include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} +} + namespace zkutil { From 7b8dc3aa6d100356e24112d2525312cbf575bb35 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Apr 2022 12:37:53 +0200 Subject: [PATCH 008/582] Fix tests into smaller parts --- .github/workflows/pull_request.yml | 164 ++++++++++++++++++++++++++++- 1 file changed, 160 insertions(+), 4 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c32aa1f5c21..62b495d345d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1292,7 +1292,7 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebugS3: + FunctionalStatelessTestS3Debug0: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: @@ -1304,6 +1304,8 @@ jobs: CHECK_NAME=Stateless tests (debug, s3 storage, actions) REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -1327,7 +1329,81 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsanS3: + FunctionalStatelessTestS3Debug1: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Debug2: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Tsan0: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: @@ -1339,6 +1415,82 @@ jobs: CHECK_NAME=Stateless tests (tsan, s3 storage, actions) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Tsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Tsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -3265,8 +3417,12 @@ jobs: - FunctionalStatefulTestMsan - FunctionalStatefulTestUBsan - FunctionalStatelessTestReleaseS3 - - FunctionalStatelessTestDebugS3 - - FunctionalStatelessTestTsanS3 + - FunctionalStatelessTestS3Debug0 + - FunctionalStatelessTestS3Debug1 + - FunctionalStatelessTestS3Debug2 + - FunctionalStatelessTestS3Tsan0 + - FunctionalStatelessTestS3Tsan1 + - FunctionalStatelessTestS3Tsan2 - StressTestDebug - StressTestAsan - StressTestTsan From 0ab0685e1d805a36b61530e5d47e0fa1a4def6dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Apr 2022 10:16:19 +0200 Subject: [PATCH 009/582] Fix --- .github/workflows/pull_request.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 62b495d345d..f12d52573ad 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1412,7 +1412,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan, s3 storage, actions) + CHECK_NAME=Stateless tests (thread, s3 storage, actions) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1449,7 +1449,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan, s3 storage, actions) + CHECK_NAME=Stateless tests (thread, s3 storage, actions) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1486,7 +1486,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan, s3 storage, actions) + CHECK_NAME=Stateless tests (thread, s3 storage, actions) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 From 6f3373b37ed664bcfd62cea5b9cfa05242446b56 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Apr 2022 12:04:31 +0200 Subject: [PATCH 010/582] Mark as always green --- tests/ci/commit_status_helper.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a53ce6715d5..9902d0871c1 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -13,6 +13,9 @@ def override_status(status, check_name, invert=False): if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): return "success" + if "s3 storage" in check_name and ("debug" in check_name or "thread" in check_name): + return "success" + if invert: if status == "success": return "error" From 51345fa25b8e937fe79d318c729df8e9ccf23385 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 1 May 2022 19:07:02 +0200 Subject: [PATCH 011/582] Trying stress on top of S3 --- docker/test/stress/run.sh | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 0ba4f0edc21..a242edc3e9d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -27,6 +27,9 @@ export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000 +export EXPORT_S3_STORAGE_POLICIES=1 +export USE_S3_STORAGE_FOR_MERGE_TREE=1 + function install_packages() { @@ -173,23 +176,23 @@ configure start # shellcheck disable=SC2086 # No quotes because I want to split it into words. -/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS +/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS ||: chmod 777 -R /var/lib/clickhouse -clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" -clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" +clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" ||: +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" ||: stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log start -clickhouse-client --query "SHOW TABLES FROM datasets" -clickhouse-client --query "SHOW TABLES FROM test" -clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" -clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" -clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" -clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" -clickhouse-client --query "SHOW TABLES FROM test" +clickhouse-client --query "SHOW TABLES FROM datasets" ||: +clickhouse-client --query "SHOW TABLES FROM test" ||: +clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" ||: +clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" ||: +clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" ||: +clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" ||: +clickhouse-client --query "SHOW TABLES FROM test" ||: ./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \ && echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \ From 85dda249b88c4f2001da736648e9277952699a60 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 May 2022 00:11:05 +0200 Subject: [PATCH 012/582] Don't download stateful table --- docker/test/stress/run.sh | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index a242edc3e9d..421adf0d69b 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -176,23 +176,23 @@ configure start # shellcheck disable=SC2086 # No quotes because I want to split it into words. -/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS ||: +#/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS ||: chmod 777 -R /var/lib/clickhouse -clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" ||: -clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" ||: +#clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" ||: +#clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" ||: stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log start -clickhouse-client --query "SHOW TABLES FROM datasets" ||: -clickhouse-client --query "SHOW TABLES FROM test" ||: -clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" ||: -clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" ||: -clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" ||: -clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" ||: -clickhouse-client --query "SHOW TABLES FROM test" ||: +#clickhouse-client --query "SHOW TABLES FROM datasets" ||: +#clickhouse-client --query "SHOW TABLES FROM test" ||: +#clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" ||: +#clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" ||: +#clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" ||: +#clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" ||: +#clickhouse-client --query "SHOW TABLES FROM test" ||: ./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \ && echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \ From 5daf1b81a0b00c53595173e913a5e77ea9b1204b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 May 2022 22:25:09 +0300 Subject: [PATCH 013/582] Update ZooKeeperImpl.cpp --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index e2ed33c1f30..8949c883ca5 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -508,7 +508,7 @@ void ZooKeeper::receiveHandshake() read(timeout); if (timeout != args.session_timeout_ms) /// Use timeout from server. - args.session_timeout_ms = timeout * 1000; + args.session_timeout_ms = timeout; read(session_id); read(passwd); From 425b31ba1b5adf021cddaeaeed1acda03710a635 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 May 2022 15:04:12 +0300 Subject: [PATCH 014/582] Update stress --- docker/test/stress/stress | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index d78de84f60d..8c05f1029a6 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -126,7 +126,7 @@ def prepare_for_hung_check(drop_databases): for db in databases: if db == "system": continue - command = make_query_command(f'DROP DATABASE {db}') + command = make_query_command(f'DETACH DATABASE {db}') # we don't wait for drop Popen(command, shell=True) break From 0dbbca5b3e6ff8959604bfb3c69b097261c147b1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 May 2022 19:33:45 +0000 Subject: [PATCH 015/582] fix filling of missed Nested columns with multiple levels --- .../Serializations/SerializationArray.cpp | 38 +++++----- .../Serializations/SerializationArray.h | 1 + src/Interpreters/inplaceBlockConversions.cpp | 73 ++++++++++++++----- src/Interpreters/inplaceBlockConversions.h | 1 + src/Storages/MergeTree/IMergeTreeReader.cpp | 6 +- src/Storages/StorageMemory.cpp | 2 +- .../0_stateless/01825_type_json_17.reference | 7 ++ .../0_stateless/01825_type_json_17.sql | 18 +++++ 8 files changed, 108 insertions(+), 38 deletions(-) create mode 100644 tests/queries/0_stateless/01825_type_json_17.reference create mode 100644 tests/queries/0_stateless/01825_type_json_17.sql diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 30ee5e98b74..aebfb1b27b2 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -132,29 +132,29 @@ namespace offset_values.resize(i); } +} - ColumnPtr arraySizesToOffsets(const IColumn & column) - { - const auto & column_sizes = assert_cast(column); - MutableColumnPtr column_offsets = column_sizes.cloneEmpty(); - - if (column_sizes.empty()) - return column_offsets; - - const auto & sizes_data = column_sizes.getData(); - auto & offsets_data = assert_cast(*column_offsets).getData(); - - offsets_data.resize(sizes_data.size()); - - IColumn::Offset prev_offset = 0; - for (size_t i = 0, size = sizes_data.size(); i < size; ++i) - { - prev_offset += sizes_data[i]; - offsets_data[i] = prev_offset; - } +ColumnPtr arraySizesToOffsets(const IColumn & column) +{ + const auto & column_sizes = assert_cast(column); + MutableColumnPtr column_offsets = column_sizes.cloneEmpty(); + if (column_sizes.empty()) return column_offsets; + + const auto & sizes_data = column_sizes.getData(); + auto & offsets_data = assert_cast(*column_offsets).getData(); + + offsets_data.resize(sizes_data.size()); + + IColumn::Offset prev_offset = 0; + for (size_t i = 0, size = sizes_data.size(); i < size; ++i) + { + prev_offset += sizes_data[i]; + offsets_data[i] = prev_offset; } + + return column_offsets; } ColumnPtr arrayOffsetsToSizes(const IColumn & column) diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 3769f8a4513..9179988bf10 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -80,5 +80,6 @@ private: }; ColumnPtr arrayOffsetsToSizes(const IColumn & column); +ColumnPtr arraySizesToOffsets(const IColumn & column); } diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 1bde6fe5a8c..bc40d64de16 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -198,6 +199,9 @@ static bool arrayHasNoElementsRead(const IColumn & column) if (!size) return false; + if (const auto * nested_array = typeid_cast(&column_array->getData())) + return arrayHasNoElementsRead(*nested_array); + size_t data_size = column_array->getData().size(); if (data_size) return false; @@ -210,6 +214,7 @@ void fillMissingColumns( Columns & res_columns, size_t num_rows, const NamesAndTypesList & requested_columns, + const NamesAndTypesList & available_columns, StorageMetadataPtr metadata_snapshot) { size_t num_columns = requested_columns.size(); @@ -224,26 +229,35 @@ void fillMissingColumns( /// First, collect offset columns for all arrays in the block. std::unordered_map offset_columns; - auto requested_column = requested_columns.begin(); - for (size_t i = 0; i < num_columns; ++i, ++requested_column) + auto available_column = available_columns.begin(); + for (size_t i = 0; i < num_columns; ++i, ++available_column) { if (res_columns[i] == nullptr) continue; - if (const auto * array = typeid_cast(res_columns[i].get())) + auto serialization = IDataType::getSerialization(*available_column); + auto name_in_storage = Nested::extractTableName(available_column->name); + + ISerialization::SubstreamPath path; + serialization->enumerateStreams(path, [&](const auto & subpath) { - String offsets_name = Nested::extractTableName(requested_column->name); - auto & offsets_column = offset_columns[offsets_name]; + if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) + return; + + auto subname = ISerialization::getSubcolumnNameForStream(subpath); + auto & offsets_column = offset_columns[Nested::concatenateName(name_in_storage, subname)]; /// If for some reason multiple offsets columns are present for the same nested data structure, /// choose the one that is not empty. + /// TODO: more optimal if (!offsets_column || offsets_column->empty()) - offsets_column = array->getOffsetsPtr(); - } + offsets_column = arraySizesToOffsets(*subpath.back().data.column); + + }, {serialization, available_column->type, res_columns[i], nullptr}); } /// insert default values only for columns without default expressions - requested_column = requested_columns.begin(); + auto requested_column = requested_columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++requested_column) { const auto & [name, type] = *requested_column; @@ -256,19 +270,44 @@ void fillMissingColumns( if (metadata_snapshot && metadata_snapshot->getColumns().hasDefault(name)) continue; - String offsets_name = Nested::extractTableName(name); - auto offset_it = offset_columns.find(offsets_name); + std::vector current_offsets; + bool has_all_offsets = true; + const auto * array_type = typeid_cast(type.get()); - if (offset_it != offset_columns.end() && array_type) + if (array_type) { - const auto & nested_type = array_type->getNestedType(); - ColumnPtr offsets_column = offset_it->second; - size_t nested_rows = typeid_cast(*offsets_column).getData().back(); + auto serialization = IDataType::getSerialization(*requested_column); + auto name_in_storage = Nested::extractTableName(requested_column->name); - ColumnPtr nested_column = - nested_type->createColumnConstWithDefaultValue(nested_rows)->convertToFullColumnIfConst(); + ISerialization::SubstreamPath path; + serialization->enumerateStreams(path, [&](const auto & subpath) + { + if (!has_all_offsets) + return; - res_columns[i] = ColumnArray::create(nested_column, offsets_column); + if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) + return; + + auto subname = ISerialization::getSubcolumnNameForStream(subpath); + auto it = offset_columns.find(Nested::concatenateName(name_in_storage, subname)); + if (it != offset_columns.end()) + current_offsets.emplace_back(it->second); + else + has_all_offsets = false; + + }, {serialization, type, nullptr, nullptr}); + } + + if (array_type && has_all_offsets) + { + assert(!current_offsets.empty()); + auto scalar_type = getBaseTypeOfArray(type); + + size_t data_size = assert_cast(*current_offsets.back()).getData().back(); + res_columns[i] = scalar_type->createColumnConstWithDefaultValue(data_size)->convertToFullColumnIfConst(); + + for (auto it = current_offsets.rbegin(); it != current_offsets.rend(); ++it) + res_columns[i] = ColumnArray::create(res_columns[i], *it); } else { diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index b3113ddfa5c..70187d5aace 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -43,6 +43,7 @@ void fillMissingColumns( Columns & res_columns, size_t num_rows, const NamesAndTypesList & requested_columns, + const NamesAndTypesList & available_columns, StorageMetadataPtr metadata_snapshot); } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 3a823345dda..4eff1653d1e 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -66,7 +66,11 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e { try { - DB::fillMissingColumns(res_columns, num_rows, columns, metadata_snapshot); + NamesAndTypesList available_columns; + for (const auto & column : columns) + available_columns.push_back(getColumnFromPart(column)); + + DB::fillMissingColumns(res_columns, num_rows, columns, available_columns, metadata_snapshot); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index e7911125383..20ca84452e7 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -91,7 +91,7 @@ protected: ++name_and_type; } - fillMissingColumns(columns, src.rows(), column_names_and_types, /*metadata_snapshot=*/ nullptr); + fillMissingColumns(columns, src.rows(), column_names_and_types, column_names_and_types, /*metadata_snapshot=*/ nullptr); assert(std::all_of(columns.begin(), columns.end(), [](const auto & column) { return column != nullptr; })); return Chunk(std::move(columns), src.rows()); diff --git a/tests/queries/0_stateless/01825_type_json_17.reference b/tests/queries/0_stateless/01825_type_json_17.reference new file mode 100644 index 00000000000..96e58224f32 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_17.reference @@ -0,0 +1,7 @@ +Tuple(arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)), id Int8) +{"obj":{"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}],"id":1}} +{"obj":{"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}],"id":2}} +[['bbb','']] [['aaa','ccc']] +[['ddd','']] [['','']] +[[0,0]] +[[10,20]] diff --git a/tests/queries/0_stateless/01825_type_json_17.sql b/tests/queries/0_stateless/01825_type_json_17.sql new file mode 100644 index 00000000000..b34357f8ef1 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_17.sql @@ -0,0 +1,18 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json_17; +SET allow_experimental_object_type = 1; +SET output_format_json_named_tuples_as_objects = 1; + +CREATE TABLE t_json_17(obj JSON) +ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} + +SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; +SELECT obj FROM t_json_17 ORDER BY obj.id FORMAT JSONEachRow; +SELECT obj.arr.k1.k3, obj.arr.k1.k2 FROM t_json_17 ORDER BY obj.id; +SELECT obj.arr.k1.k4 FROM t_json_17 ORDER BY obj.id; + +DROP TABLE IF EXISTS t_json_17; From 56fffc86810773e0399de5147b08ced18b120822 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 12 May 2022 17:21:30 +0000 Subject: [PATCH 016/582] fix filling of missing columns --- src/Interpreters/inplaceBlockConversions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index bc40d64de16..61d51da4eab 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -232,7 +232,7 @@ void fillMissingColumns( auto available_column = available_columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++available_column) { - if (res_columns[i] == nullptr) + if (res_columns[i] == nullptr || isColumnConst(*res_columns[i])) continue; auto serialization = IDataType::getSerialization(*available_column); From 1523c9c9e5f714daff08d7811890effb25e4a87b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 17 Jun 2022 01:10:52 +0000 Subject: [PATCH 017/582] fix filling of empty Nested + small refactoring --- src/Columns/ColumnArray.cpp | 6 +- .../CompressionFactoryAdditions.cpp | 4 +- src/DataTypes/IDataType.cpp | 29 +++-- src/DataTypes/IDataType.h | 2 + .../Serializations/ISerialization.cpp | 35 ++--- src/DataTypes/Serializations/ISerialization.h | 44 ++++++- .../Serializations/SerializationArray.cpp | 123 +++++++++--------- .../Serializations/SerializationArray.h | 5 +- .../SerializationLowCardinality.cpp | 27 ++-- .../SerializationLowCardinality.h | 2 +- .../Serializations/SerializationMap.cpp | 15 +-- .../Serializations/SerializationMap.h | 2 +- .../Serializations/SerializationNamed.cpp | 12 +- .../Serializations/SerializationNamed.h | 2 +- .../Serializations/SerializationNullable.cpp | 40 +++--- .../Serializations/SerializationNullable.h | 2 +- .../Serializations/SerializationSparse.cpp | 39 +++--- .../Serializations/SerializationSparse.h | 2 +- .../Serializations/SerializationTuple.cpp | 15 +-- .../Serializations/SerializationTuple.h | 2 +- .../Serializations/SerializationWrapper.cpp | 4 +- .../Serializations/SerializationWrapper.h | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 25 ++-- src/Storages/ColumnsDescription.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 30 ++--- .../MergeTree/MergeTreeDataPartWriterWide.h | 9 +- .../MergeTree/MergeTreeReaderCompact.cpp | 23 ++-- .../MergeTree/MergeTreeReaderWide.cpp | 2 +- .../System/StorageSystemPartsColumns.cpp | 4 +- 31 files changed, 264 insertions(+), 250 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 24da9644335..7bdb66a9cc5 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -50,13 +50,15 @@ ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && if (!offsets_concrete) throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::LOGICAL_ERROR); - if (!offsets_concrete->empty() && data) + if (!offsets_concrete->empty() && data && !data->empty()) { Offset last_offset = offsets_concrete->getData().back(); /// This will also prevent possible overflow in offset. if (data->size() != last_offset) - throw Exception("offsets_column has data inconsistent with nested_column", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "offsets_column has data ({}) inconsistent with nested_column ({})", + data->size(), last_offset); } /** NOTE diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index d87d0f8b4ee..3e215076871 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -116,8 +116,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( } }; - ISerialization::SubstreamPath path; - column_type->getDefaultSerialization()->enumerateStreams(path, callback, column_type); + auto serialization = column_type->getDefaultSerialization(); + serialization->enumerateStreams(callback, column_type); if (!result_codec) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index f2bb878a533..8229946cac8 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -84,18 +84,20 @@ void IDataType::forEachSubcolumn( { for (size_t i = 0; i < subpath.size(); ++i) { - if (!subpath[i].visited && ISerialization::hasSubcolumnForPath(subpath, i + 1)) + size_t prefix_len = i + 1; + if (!subpath[i].visited && ISerialization::hasSubcolumnForPath(subpath, prefix_len)) { - auto name = ISerialization::getSubcolumnNameForStream(subpath, i + 1); - auto subdata = ISerialization::createFromPath(subpath, i); + auto name = ISerialization::getSubcolumnNameForStream(subpath, prefix_len); + auto subdata = ISerialization::createFromPath(subpath, prefix_len); callback(subpath, name, subdata); } subpath[i].visited = true; } }; - SubstreamPath path; - data.serialization->enumerateStreams(path, callback_with_data, data); + ISerialization::EnumerateStreamsSettings settings; + settings.position_independent_encoding = false; + data.serialization->enumerateStreams(settings, callback_with_data, data); } template @@ -118,33 +120,38 @@ Ptr IDataType::getForSubcolumn( return res; } +bool IDataType::hasSubcolumn(const String & subcolumn_name) const +{ + return tryGetSubcolumnType(subcolumn_name) != nullptr; +} + DataTypePtr IDataType::tryGetSubcolumnType(const String & subcolumn_name) const { - SubstreamData data = { getDefaultSerialization(), getPtr(), nullptr, nullptr }; + auto data = SubstreamData(getDefaultSerialization()).withType(getPtr()); return getForSubcolumn(subcolumn_name, data, &SubstreamData::type, false); } DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const { - SubstreamData data = { getDefaultSerialization(), getPtr(), nullptr, nullptr }; + auto data = SubstreamData(getDefaultSerialization()).withType(getPtr()); return getForSubcolumn(subcolumn_name, data, &SubstreamData::type, true); } ColumnPtr IDataType::tryGetSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const { - SubstreamData data = { getDefaultSerialization(), nullptr, column, nullptr }; + auto data = SubstreamData(getDefaultSerialization()).withColumn(column); return getForSubcolumn(subcolumn_name, data, &SubstreamData::column, false); } ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const { - SubstreamData data = { getDefaultSerialization(), nullptr, column, nullptr }; + auto data = SubstreamData(getDefaultSerialization()).withColumn(column); return getForSubcolumn(subcolumn_name, data, &SubstreamData::column, true); } SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const { - SubstreamData data = { serialization, nullptr, nullptr, nullptr }; + auto data = SubstreamData(serialization); return getForSubcolumn(subcolumn_name, data, &SubstreamData::serialization, true); } @@ -154,7 +161,7 @@ Names IDataType::getSubcolumnNames() const forEachSubcolumn([&](const auto &, const auto & name, const auto &) { res.push_back(name); - }, { getDefaultSerialization(), nullptr, nullptr, nullptr }); + }, SubstreamData(getDefaultSerialization())); return res; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 420ef61a13f..9161fd7dc7b 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -79,6 +79,8 @@ public: /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; + bool hasSubcolumn(const String & subcolumn_name) const; + DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const; DataTypePtr getSubcolumnType(const String & subcolumn_name) const; diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 7df4a956c1a..b9b3b9ac6af 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -73,24 +73,24 @@ String ISerialization::SubstreamPath::toString() const } void ISerialization::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { - path.push_back(Substream::Regular); - path.back().data = data; - callback(path); - path.pop_back(); + settings.path.push_back(Substream::Regular); + settings.path.back().data = data; + callback(settings.path); + settings.path.pop_back(); } -void ISerialization::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void ISerialization::enumerateStreams( + const StreamCallback & callback, + const DataTypePtr & type, + const ColumnPtr & column) const { - enumerateStreams(path, callback, {getPtr(), nullptr, nullptr, nullptr}); -} - -void ISerialization::enumerateStreams(SubstreamPath & path, const StreamCallback & callback, const DataTypePtr & type) const -{ - enumerateStreams(path, callback, {getPtr(), type, nullptr, nullptr}); + EnumerateStreamsSettings settings; + auto data = SubstreamData(getPtr()).withType(type).withColumn(column); + enumerateStreams(settings, callback, data); } void ISerialization::serializeBinaryBulk(const IColumn & column, WriteBuffer &, size_t, size_t) const @@ -184,7 +184,7 @@ String ISerialization::getFileNameForStream(const NameAndTypePair & column, cons return getFileNameForStream(column.getNameInStorage(), path); } -static size_t isOffsetsOfNested(const ISerialization::SubstreamPath & path) +bool isOffsetsOfNested(const ISerialization::SubstreamPath & path) { if (path.empty()) return false; @@ -288,10 +288,13 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) { - assert(prefix_len < path.size()); + assert(prefix_len <= path.size()); + if (prefix_len == 0) + return {}; - SubstreamData res = path[prefix_len].data; - for (ssize_t i = static_cast(prefix_len) - 1; i >= 0; --i) + ssize_t last_elem = prefix_len - 1; + auto res = path[last_elem].data; + for (ssize_t i = last_elem - 1; i >= 0; --i) { const auto & creator = path[i].creator; if (creator) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index b5d2082631e..1193c15b939 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -101,6 +101,30 @@ public: struct SubstreamData { + SubstreamData() = default; + SubstreamData(SerializationPtr serialization_) + : serialization(std::move(serialization_)) + { + } + + SubstreamData & withType(DataTypePtr type_) + { + type = std::move(type_); + return *this; + } + + SubstreamData & withColumn(ColumnPtr column_) + { + column = std::move(column_); + return *this; + } + + SubstreamData & withSerializationInfo(SerializationInfoPtr serialization_info_) + { + serialization_info = std::move(serialization_info_); + return *this; + } + SerializationPtr serialization; DataTypePtr type; ColumnPtr column; @@ -164,16 +188,22 @@ public: using StreamCallback = std::function; + struct EnumerateStreamsSettings + { + SubstreamPath path; + bool position_independent_encoding = true; + }; + virtual void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const; - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const; - void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } - void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } - - void enumerateStreams(SubstreamPath & path, const StreamCallback & callback, const DataTypePtr & type) const; + /// Enumerate streams with default settings. + void enumerateStreams( + const StreamCallback & callback, + const DataTypePtr & type = nullptr, + const ColumnPtr & column = nullptr) const; using OutputStreamGetter = std::function; using InputStreamGetter = std::function; @@ -375,4 +405,6 @@ State * ISerialization::checkAndGetState(const StatePtr & state) const return state_concrete; } +bool isOffsetsOfNested(const ISerialization::SubstreamPath & path); + } diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index aebfb1b27b2..85182c3a111 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -132,53 +132,53 @@ namespace offset_values.resize(i); } -} -ColumnPtr arraySizesToOffsets(const IColumn & column) -{ - const auto & column_sizes = assert_cast(column); - MutableColumnPtr column_offsets = column_sizes.cloneEmpty(); + ColumnPtr arraySizesToOffsets(const IColumn & column) + { + const auto & column_sizes = assert_cast(column); + MutableColumnPtr column_offsets = column_sizes.cloneEmpty(); + + if (column_sizes.empty()) + return column_offsets; + + const auto & sizes_data = column_sizes.getData(); + auto & offsets_data = assert_cast(*column_offsets).getData(); + + offsets_data.resize(sizes_data.size()); + + IColumn::Offset prev_offset = 0; + for (size_t i = 0, size = sizes_data.size(); i < size; ++i) + { + prev_offset += sizes_data[i]; + offsets_data[i] = prev_offset; + } - if (column_sizes.empty()) return column_offsets; - - const auto & sizes_data = column_sizes.getData(); - auto & offsets_data = assert_cast(*column_offsets).getData(); - - offsets_data.resize(sizes_data.size()); - - IColumn::Offset prev_offset = 0; - for (size_t i = 0, size = sizes_data.size(); i < size; ++i) - { - prev_offset += sizes_data[i]; - offsets_data[i] = prev_offset; } - return column_offsets; -} + ColumnPtr arrayOffsetsToSizes(const IColumn & column) + { + const auto & column_offsets = assert_cast(column); + MutableColumnPtr column_sizes = column_offsets.cloneEmpty(); -ColumnPtr arrayOffsetsToSizes(const IColumn & column) -{ - const auto & column_offsets = assert_cast(column); - MutableColumnPtr column_sizes = column_offsets.cloneEmpty(); + if (column_offsets.empty()) + return column_sizes; + + const auto & offsets_data = column_offsets.getData(); + auto & sizes_data = assert_cast(*column_sizes).getData(); + + sizes_data.resize(offsets_data.size()); + + IColumn::Offset prev_offset = 0; + for (size_t i = 0, size = offsets_data.size(); i < size; ++i) + { + auto current_offset = offsets_data[i]; + sizes_data[i] = current_offset - prev_offset; + prev_offset = current_offset; + } - if (column_offsets.empty()) return column_sizes; - - const auto & offsets_data = column_offsets.getData(); - auto & sizes_data = assert_cast(*column_sizes).getData(); - - sizes_data.resize(offsets_data.size()); - - IColumn::Offset prev_offset = 0; - for (size_t i = 0, size = offsets_data.size(); i < size; ++i) - { - auto current_offset = offsets_data[i]; - sizes_data[i] = current_offset - prev_offset; - prev_offset = current_offset; } - - return column_sizes; } DataTypePtr SerializationArray::SubcolumnCreator::create(const DataTypePtr & prev) const @@ -197,41 +197,42 @@ ColumnPtr SerializationArray::SubcolumnCreator::create(const ColumnPtr & prev) c } void SerializationArray::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { const auto * type_array = data.type ? &assert_cast(*data.type) : nullptr; const auto * column_array = data.column ? &assert_cast(*data.column) : nullptr; - auto offsets_column = column_array ? column_array->getOffsetsPtr() : nullptr; + auto offsets = column_array ? column_array->getOffsetsPtr() : nullptr; - path.push_back(Substream::ArraySizes); - path.back().data = - { + auto offsets_serialization = std::make_shared( std::make_shared>(), - "size" + std::to_string(getArrayLevel(path)), false), - data.type ? std::make_shared() : nullptr, - offsets_column ? arrayOffsetsToSizes(*offsets_column) : nullptr, - data.serialization_info, - }; + "size" + std::to_string(getArrayLevel(settings.path)), false); - callback(path); + auto offsets_column = offsets && !settings.position_independent_encoding + ? arrayOffsetsToSizes(*offsets) + : offsets; - path.back() = Substream::ArrayElements; - path.back().data = data; - path.back().creator = std::make_shared(offsets_column); + settings.path.push_back(Substream::ArraySizes); + settings.path.back().data = SubstreamData(offsets_serialization) + .withType(type_array ? std::make_shared() : nullptr) + .withColumn(std::move(offsets_column)) + .withSerializationInfo(data.serialization_info); - SubstreamData next_data = - { - nested, - type_array ? type_array->getNestedType() : nullptr, - column_array ? column_array->getDataPtr() : nullptr, - data.serialization_info, - }; + callback(settings.path); - nested->enumerateStreams(path, callback, next_data); - path.pop_back(); + settings.path.back() = Substream::ArrayElements; + settings.path.back().data = data; + settings.path.back().creator = std::make_shared(offsets); + + auto next_data = SubstreamData(nested) + .withType(type_array ? type_array->getNestedType() : nullptr) + .withColumn(column_array ? column_array->getDataPtr() : nullptr) + .withSerializationInfo(data.serialization_info); + + nested->enumerateStreams(settings, callback, next_data); + settings.path.pop_back(); } void SerializationArray::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 9179988bf10..84e37acbaad 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -36,7 +36,7 @@ public: */ void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; @@ -79,7 +79,4 @@ private: }; }; -ColumnPtr arrayOffsetsToSizes(const IColumn & column); -ColumnPtr arraySizesToOffsets(const IColumn & column); - } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index c79f588e46c..2473a74327d 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -41,30 +41,25 @@ SerializationLowCardinality::SerializationLowCardinality(const DataTypePtr & dic } void SerializationLowCardinality::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { const auto * column_lc = data.column ? &getColumnLowCardinality(*data.column) : nullptr; - SubstreamData dict_data = - { - dict_inner_serialization, - data.type ? dictionary_type : nullptr, - column_lc ? column_lc->getDictionary().getNestedColumn() : nullptr, - data.serialization_info, - }; + settings.path.push_back(Substream::DictionaryKeys); + settings.path.back().data = SubstreamData(dict_inner_serialization) + .withType(data.type ? dictionary_type : nullptr) + .withColumn(column_lc ? column_lc->getDictionary().getNestedColumn() : nullptr) + .withSerializationInfo(data.serialization_info); - path.push_back(Substream::DictionaryKeys); - path.back().data = dict_data; + dict_inner_serialization->enumerateStreams(settings, callback, settings.path.back().data); - dict_inner_serialization->enumerateStreams(path, callback, dict_data); + settings.path.back() = Substream::DictionaryIndexes; + settings.path.back().data = data; - path.back() = Substream::DictionaryIndexes; - path.back().data = data; - - callback(path); - path.pop_back(); + callback(settings.path); + settings.path.pop_back(); } struct KeysSerializationVersion diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index 0a3597e86c7..860920a2422 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -18,7 +18,7 @@ public: explicit SerializationLowCardinality(const DataTypePtr & dictionary_type); void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index ea22070b5b1..e46bb480d14 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -257,19 +257,16 @@ void SerializationMap::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c } void SerializationMap::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { - SubstreamData next_data = - { - nested, - data.type ? assert_cast(*data.type).getNestedType() : nullptr, - data.column ? assert_cast(*data.column).getNestedColumnPtr() : nullptr, - data.serialization_info, - }; + auto next_data = SubstreamData(nested) + .withType(data.type ? assert_cast(*data.type).getNestedType() : nullptr) + .withColumn(data.column ? assert_cast(*data.column).getNestedColumnPtr() : nullptr) + .withSerializationInfo(data.serialization_info); - nested->enumerateStreams(path, callback, next_data); + nested->enumerateStreams(settings, callback, next_data); } void SerializationMap::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 93b3e179499..42f99ca7991 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -32,7 +32,7 @@ public: void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/DataTypes/Serializations/SerializationNamed.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp index 097e9cedfbe..4dac4b3a922 100644 --- a/src/DataTypes/Serializations/SerializationNamed.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -4,16 +4,16 @@ namespace DB { void SerializationNamed::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { - addToPath(path); - path.back().data = data; - path.back().creator = std::make_shared(name, escape_delimiter); + addToPath(settings.path); + settings.path.back().data = data; + settings.path.back().creator = std::make_shared(name, escape_delimiter); - nested_serialization->enumerateStreams(path, callback, data); - path.pop_back(); + nested_serialization->enumerateStreams(settings, callback, data); + settings.path.pop_back(); } void SerializationNamed::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationNamed.h b/src/DataTypes/Serializations/SerializationNamed.h index 343b96c16e3..2a2c7c0dfc7 100644 --- a/src/DataTypes/Serializations/SerializationNamed.h +++ b/src/DataTypes/Serializations/SerializationNamed.h @@ -26,7 +26,7 @@ public: const String & getElementName() const { return name; } void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index a6273deaa30..47780f67800 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -38,38 +38,34 @@ ColumnPtr SerializationNullable::SubcolumnCreator::create(const ColumnPtr & prev } void SerializationNullable::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { const auto * type_nullable = data.type ? &assert_cast(*data.type) : nullptr; const auto * column_nullable = data.column ? &assert_cast(*data.column) : nullptr; - path.push_back(Substream::NullMap); - path.back().data = - { - std::make_shared(std::make_shared>(), "null", false), - type_nullable ? std::make_shared() : nullptr, - column_nullable ? column_nullable->getNullMapColumnPtr() : nullptr, - data.serialization_info, - }; + auto null_map_serialization = std::make_shared(std::make_shared>(), "null", false); - callback(path); + settings.path.push_back(Substream::NullMap); + settings.path.back().data = SubstreamData(null_map_serialization) + .withType(type_nullable ? std::make_shared() : nullptr) + .withColumn(column_nullable ? column_nullable->getNullMapColumnPtr() : nullptr) + .withSerializationInfo(data.serialization_info); - path.back() = Substream::NullableElements; - path.back().creator = std::make_shared(path.back().data.column); - path.back().data = data; + callback(settings.path); - SubstreamData next_data = - { - nested, - type_nullable ? type_nullable->getNestedType() : nullptr, - column_nullable ? column_nullable->getNestedColumnPtr() : nullptr, - data.serialization_info, - }; + settings.path.back() = Substream::NullableElements; + settings.path.back().creator = std::make_shared(settings.path.back().data.column); + settings.path.back().data = data; - nested->enumerateStreams(path, callback, next_data); - path.pop_back(); + auto next_data = SubstreamData(nested) + .withType(type_nullable ? type_nullable->getNestedType() : nullptr) + .withColumn(column_nullable ? column_nullable->getNestedColumnPtr() : nullptr) + .withSerializationInfo(data.serialization_info); + + nested->enumerateStreams(settings, callback, next_data); + settings.path.pop_back(); } void SerializationNullable::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index e6e0e4f33c2..ea3958065e7 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -14,7 +14,7 @@ public: explicit SerializationNullable(const SerializationPtr & nested_) : nested(nested_) {} void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 64db248c5fc..db194007af9 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -148,39 +148,32 @@ ColumnPtr SerializationSparse::SubcolumnCreator::create(const ColumnPtr & prev) } void SerializationSparse::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { const auto * column_sparse = data.column ? &assert_cast(*data.column) : nullptr; - size_t column_size = column_sparse ? column_sparse->size() : 0; - path.push_back(Substream::SparseOffsets); - path.back().data = - { - std::make_shared>(), - data.type ? std::make_shared() : nullptr, - column_sparse ? column_sparse->getOffsetsPtr() : nullptr, - data.serialization_info, - }; + settings.path.push_back(Substream::SparseOffsets); + settings.path.back().data = SubstreamData(std::make_shared>()) + .withType(data.type ? std::make_shared() : nullptr) + .withColumn(column_sparse ? column_sparse->getOffsetsPtr() : nullptr) + .withSerializationInfo(data.serialization_info); - callback(path); + callback(settings.path); - path.back() = Substream::SparseElements; - path.back().creator = std::make_shared(path.back().data.column, column_size); - path.back().data = data; + settings.path.back() = Substream::SparseElements; + settings.path.back().creator = std::make_shared(settings.path.back().data.column, column_size); + settings.path.back().data = data; - SubstreamData next_data = - { - nested, - data.type, - column_sparse ? column_sparse->getValuesPtr() : nullptr, - data.serialization_info, - }; + auto next_data = SubstreamData(nested) + .withType(data.type) + .withColumn(column_sparse ? column_sparse->getValuesPtr() : nullptr) + .withSerializationInfo(data.serialization_info); - nested->enumerateStreams(path, callback, next_data); - path.pop_back(); + nested->enumerateStreams(settings, callback, next_data); + settings.path.pop_back(); } void SerializationSparse::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index 54ab4853360..dc2f63c5a05 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -28,7 +28,7 @@ public: Kind getKind() const override { return Kind::SPARSE; } virtual void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 8dc15fc9841..437324d96fd 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -283,7 +283,7 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, } void SerializationTuple::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { @@ -293,15 +293,12 @@ void SerializationTuple::enumerateStreams( for (size_t i = 0; i < elems.size(); ++i) { - SubstreamData next_data = - { - elems[i], - type_tuple ? type_tuple->getElement(i) : nullptr, - column_tuple ? column_tuple->getColumnPtr(i) : nullptr, - info_tuple ? info_tuple->getElementInfo(i) : nullptr, - }; + auto next_data = SubstreamData(elems[i]) + .withType(type_tuple ? type_tuple->getElement(i) : nullptr) + .withColumn(column_tuple ? column_tuple->getColumnPtr(i) : nullptr) + .withSerializationInfo(info_tuple ? info_tuple->getElementInfo(i) : nullptr); - elems[i]->enumerateStreams(path, callback, next_data); + elems[i]->enumerateStreams(settings, callback, next_data); } } diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index e82d8473645..d1caeb73dad 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -34,7 +34,7 @@ public: /** Each sub-column in a tuple is serialized in separate stream. */ void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index 271c53dfcf1..7c50c1c6e26 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -5,11 +5,11 @@ namespace DB { void SerializationWrapper::enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const { - nested_serialization->enumerateStreams(path, callback, data); + nested_serialization->enumerateStreams(settings, callback, data); } void SerializationWrapper::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 43fc7e9914a..d010c6b5314 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -21,7 +21,7 @@ public: Kind getKind() const override { return nested_serialization->getKind(); } void enumerateStreams( - SubstreamPath & path, + EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const override; diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 9919b1272bd..0524feea1f6 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -163,7 +163,7 @@ BlockIO InterpreterDescribeQuery::execute() res_columns[6]->insertDefault(); res_columns[7]->insert(1u); - }, { type->getDefaultSerialization(), type, nullptr, nullptr }); + }, ISerialization::SubstreamData(type->getDefaultSerialization()).withType(type)); } } diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 61d51da4eab..e4f1b46fc91 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -199,11 +199,11 @@ static bool arrayHasNoElementsRead(const IColumn & column) if (!size) return false; - if (const auto * nested_array = typeid_cast(&column_array->getData())) + const auto & array_data = column_array->getData(); + if (const auto * nested_array = typeid_cast(&array_data)) return arrayHasNoElementsRead(*nested_array); - size_t data_size = column_array->getData().size(); - if (data_size) + if (!array_data.empty()) return false; size_t last_offset = column_array->getOffsets()[size - 1]; @@ -238,8 +238,7 @@ void fillMissingColumns( auto serialization = IDataType::getSerialization(*available_column); auto name_in_storage = Nested::extractTableName(available_column->name); - ISerialization::SubstreamPath path; - serialization->enumerateStreams(path, [&](const auto & subpath) + serialization->enumerateStreams([&](const auto & subpath) { if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) return; @@ -247,16 +246,15 @@ void fillMissingColumns( auto subname = ISerialization::getSubcolumnNameForStream(subpath); auto & offsets_column = offset_columns[Nested::concatenateName(name_in_storage, subname)]; - /// If for some reason multiple offsets columns are present for the same nested data structure, - /// choose the one that is not empty. - /// TODO: more optimal + /// If for some reason multiple offsets columns are present + /// for the same nested data structure, choose the one that is not empty. if (!offsets_column || offsets_column->empty()) - offsets_column = arraySizesToOffsets(*subpath.back().data.column); + offsets_column = subpath.back().data.column; - }, {serialization, available_column->type, res_columns[i], nullptr}); + }, available_column->type, res_columns[i]); } - /// insert default values only for columns without default expressions + /// Insert default values only for columns without default expressions. auto requested_column = requested_columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++requested_column) { @@ -279,8 +277,7 @@ void fillMissingColumns( auto serialization = IDataType::getSerialization(*requested_column); auto name_in_storage = Nested::extractTableName(requested_column->name); - ISerialization::SubstreamPath path; - serialization->enumerateStreams(path, [&](const auto & subpath) + serialization->enumerateStreams([&](const auto & subpath) { if (!has_all_offsets) return; @@ -295,7 +292,7 @@ void fillMissingColumns( else has_all_offsets = false; - }, {serialization, type, nullptr, nullptr}); + }, type); } if (array_type && has_all_offsets) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 7a43ae7af4b..e9f4c1b92aa 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -780,7 +780,7 @@ void ColumnsDescription::addSubcolumns(const String & name_in_storage, const Dat "Cannot add subcolumn {}: column with this name already exists", subcolumn.name); subcolumns.get<0>().insert(std::move(subcolumn)); - }, {type_in_storage->getDefaultSerialization(), type_in_storage, nullptr, nullptr}); + }, ISerialization::SubstreamData(type_in_storage->getDefaultSerialization()).withType(type_in_storage)); } void ColumnsDescription::removeSubcolumns(const String & name_in_storage) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index a4786570fcb..bbdd2734c98 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -67,8 +67,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, compressed_streams.emplace(stream_name, stream); }; - ISerialization::SubstreamPath path; - data_part->getSerialization(column)->enumerateStreams(path, callback, column.type); + data_part->getSerialization(column)->enumerateStreams(callback, column.type); } namespace diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 6610b8fc06b..7c08e127ab4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -119,8 +119,7 @@ void MergeTreeDataPartWriterWide::addStreams( settings.query_write_settings); }; - ISerialization::SubstreamPath path; - data_part->getSerialization(column)->enumerateStreams(path, callback, column.type); + data_part->getSerialization(column)->enumerateStreams(callback, column.type); } @@ -254,10 +253,9 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm void MergeTreeDataPartWriterWide::writeSingleMark( const NameAndTypePair & column, WrittenOffsetColumns & offset_columns, - size_t number_of_rows, - ISerialization::SubstreamPath & path) + size_t number_of_rows) { - StreamsWithMarks marks = getCurrentMarksForColumn(column, offset_columns, path); + StreamsWithMarks marks = getCurrentMarksForColumn(column, offset_columns); for (const auto & mark : marks) flushMarkToFile(mark, number_of_rows); } @@ -273,8 +271,7 @@ void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stre StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( const NameAndTypePair & column, - WrittenOffsetColumns & offset_columns, - ISerialization::SubstreamPath & path) + WrittenOffsetColumns & offset_columns) { StreamsWithMarks result; data_part->getSerialization(column)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) @@ -299,7 +296,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( stream_with_mark.mark.offset_in_decompressed_block = stream.compressed.offset(); result.push_back(stream_with_mark); - }, path); + }); return result; } @@ -327,7 +324,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( return; column_streams[stream_name]->compressed.nextIfAtEnd(); - }, serialize_settings.path); + }); } /// Column must not be empty. (column.size() !== 0) @@ -365,7 +362,7 @@ void MergeTreeDataPartWriterWide::writeColumn( { if (last_non_written_marks.contains(name)) throw Exception(ErrorCodes::LOGICAL_ERROR, "We have to add new mark for column, but already have non written mark. Current mark {}, total marks {}, offset {}", getCurrentMark(), index_granularity.getMarksCount(), rows_written_in_last_mark); - last_non_written_marks[name] = getCurrentMarksForColumn(name_and_type, offset_columns, serialize_settings.path); + last_non_written_marks[name] = getCurrentMarksForColumn(name_and_type, offset_columns); } writeSingleGranule( @@ -389,7 +386,7 @@ void MergeTreeDataPartWriterWide::writeColumn( } } - serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) @@ -397,7 +394,7 @@ void MergeTreeDataPartWriterWide::writeColumn( String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); offset_columns.insert(stream_name); } - }, serialize_settings.path); + }); } @@ -551,7 +548,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum } if (write_final_mark) - writeFinalMark(*it, offset_columns, serialize_settings.path); + writeFinalMark(*it, offset_columns); } } @@ -616,10 +613,9 @@ void MergeTreeDataPartWriterWide::finish(bool sync) void MergeTreeDataPartWriterWide::writeFinalMark( const NameAndTypePair & column, - WrittenOffsetColumns & offset_columns, - ISerialization::SubstreamPath & path) + WrittenOffsetColumns & offset_columns) { - writeSingleMark(column, offset_columns, 0, path); + writeSingleMark(column, offset_columns, 0); /// Memoize information about offsets data_part->getSerialization(column)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { @@ -629,7 +625,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( String stream_name = ISerialization::getFileNameForStream(column, substream_path); offset_columns.insert(stream_name); } - }, path); + }); } static void fillIndexGranularityImpl( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index b82fcd652ae..8292a41b902 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -60,8 +60,7 @@ private: /// Take offsets from column and return as MarkInCompressed file with stream name StreamsWithMarks getCurrentMarksForColumn( const NameAndTypePair & column, - WrittenOffsetColumns & offset_columns, - ISerialization::SubstreamPath & path); + WrittenOffsetColumns & offset_columns); /// Write mark to disk using stream and rows count void flushMarkToFile( @@ -72,13 +71,11 @@ private: void writeSingleMark( const NameAndTypePair & column, WrittenOffsetColumns & offset_columns, - size_t number_of_rows, - ISerialization::SubstreamPath & path); + size_t number_of_rows); void writeFinalMark( const NameAndTypePair & column, - WrittenOffsetColumns & offset_columns, - ISerialization::SubstreamPath & path); + WrittenOffsetColumns & offset_columns); void addStreams( const NameAndTypePair & column, diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index b943c3c8718..6e7a080f418 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -49,22 +49,24 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( column_positions.resize(columns_num); read_only_offsets.resize(columns_num); + auto name_and_type = columns.begin(); for (size_t i = 0; i < columns_num; ++i, ++name_and_type) { - if (name_and_type->isSubcolumn()) + auto column_from_part = getColumnFromPart(*name_and_type); + auto position = data_part->getColumnPosition(column_from_part.getNameInStorage()); + bool is_array = isArray(column_from_part.type); + + if (column_from_part.isSubcolumn()) { auto storage_column_from_part = getColumnFromPart( - {name_and_type->getNameInStorage(), name_and_type->getTypeInStorage()}); + {column_from_part.getNameInStorage(), column_from_part.getTypeInStorage()}); - if (!storage_column_from_part.type->tryGetSubcolumnType(name_and_type->getSubcolumnName())) - continue; + auto subcolumn_name = column_from_part.getSubcolumnName(); + if (!storage_column_from_part.type->hasSubcolumn(subcolumn_name)) + position.reset(); } - - auto column_from_part = getColumnFromPart(*name_and_type); - - auto position = data_part->getColumnPosition(column_from_part.getNameInStorage()); - if (!position && typeid_cast(column_from_part.type.get())) + else if (!position && is_array) { /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. @@ -221,7 +223,8 @@ void MergeTreeReaderCompact::readData( auto buffer_getter = [&](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer * { - if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != ISerialization::Substream::ArraySizes)) + bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; + if (only_offsets && !is_offsets) return nullptr; return data_buffer; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 0f5cf8de669..fa059506954 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -15,7 +15,6 @@ namespace DB namespace { - using OffsetColumns = std::map; constexpr auto DATA_FILE_EXTENSION = ".bin"; } @@ -291,6 +290,7 @@ void MergeTreeReaderWide::readData( /* seek_to_start = */false, substream_path, streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache); }; + deserialize_settings.continuous_reading = continue_reading; auto & deserialize_state = deserialize_binary_bulk_state_map[name]; diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 7f648054da2..5ffca62d9c3 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -242,7 +242,7 @@ void StorageSystemPartsColumns::processNextStorage( IDataType::forEachSubcolumn([&](const auto & subpath, const auto & name, const auto & data) { /// We count only final subcolumns, which are represented by files on disk - /// and skip intermediate suibcolumns of types Tuple and Nested. + /// and skip intermediate subcolumns of types Tuple and Nested. if (isTuple(data.type) || isNested(data.type)) return; @@ -270,7 +270,7 @@ void StorageSystemPartsColumns::processNextStorage( subcolumn_data_uncompressed_bytes.push_back(size.data_uncompressed); subcolumn_marks_bytes.push_back(size.marks); - }, { serialization, column.type, nullptr, nullptr }); + }, ISerialization::SubstreamData(serialization).withType(column.type)); if (columns_mask[src_index++]) columns[res_index++]->insert(subcolumn_names); From 3296ba25327e00940a7d262e6e46c7d3482b1241 Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Tue, 21 Jun 2022 12:17:52 +0300 Subject: [PATCH 018/582] Fix SYSTEM UNFREEZE for ordinary database --- src/Parsers/ASTSystemQuery.cpp | 4 + src/Storages/Freeze.cpp | 77 +++++++++++-------- src/Storages/Freeze.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 19 +---- src/Storages/StorageReplicatedMergeTree.h | 2 +- .../01417_freeze_partition_verbose.sh | 2 +- 8 files changed, 58 insertions(+), 58 deletions(-) diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a6ff52b74b7..ab5137d0960 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -201,6 +201,10 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, if (!filesystem_cache_path.empty()) settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_path; } + else if (type == Type::UNFREEZE) + { + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(backup_name); + } } diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 5a3056450a0..9193ed23c5b 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -9,8 +9,6 @@ namespace DB { void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage) { - is_replicated = storage.supportsReplication(); - is_remote = storage.isRemote(); replica_name = storage.getReplicaName(); zookeeper_name = storage.getZooKeeperName(); table_shared_id = storage.getTableSharedID(); @@ -26,11 +24,16 @@ void FreezeMetaData::save(DiskPtr data_disk, const String & path) const writeIntText(version, buffer); buffer.write("\n", 1); - writeBoolText(is_replicated, buffer); - buffer.write("\n", 1); - writeBoolText(is_remote, buffer); - buffer.write("\n", 1); - writeString(replica_name, buffer); + if (version == 1) { + /// is_replicated and is_remote are not used + bool is_replicated = true; + writeBoolText(is_replicated, buffer); + buffer.write("\n", 1); + bool is_remote = true; + writeBoolText(is_remote, buffer); + buffer.write("\n", 1); + } + writeString(escapeForFileName(replica_name), buffer); buffer.write("\n", 1); writeString(zookeeper_name, buffer); buffer.write("\n", 1); @@ -51,17 +54,23 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) auto metadata_str = metadata_storage->readFileToString(file_path); ReadBufferFromString buffer(metadata_str); readIntText(version, buffer); - if (version != 1) + if (version < 1 or version > 2) { LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version); return false; } DB::assertChar('\n', buffer); - readBoolText(is_replicated, buffer); - DB::assertChar('\n', buffer); - readBoolText(is_remote, buffer); - DB::assertChar('\n', buffer); - readString(replica_name, buffer); + if (version == 1) { + /// is_replicated and is_remote are not used + bool is_replicated; + readBoolText(is_replicated, buffer); + DB::assertChar('\n', buffer); + bool is_remote; + readBoolText(is_remote, buffer); + DB::assertChar('\n', buffer); + } + auto unescaped_replica_name = unescapeForFileName(replica_name); + readString(unescaped_replica_name, buffer); DB::assertChar('\n', buffer); readString(zookeeper_name, buffer); DB::assertChar('\n', buffer); @@ -89,7 +98,7 @@ String FreezeMetaData::getFileName(const String & path) BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context) { - LOG_DEBUG(log, "Unfreezing backup {}", backup_name); + LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name)); auto disks_map = local_context->getDisksMap(); Disks disks; for (auto & [name, disk]: disks_map) @@ -97,33 +106,38 @@ BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context disks.push_back(disk); } auto backup_path = fs::path(backup_directory_prefix) / escapeForFileName(backup_name); - auto store_path = backup_path / "store"; + auto store_paths = {backup_path / "store", backup_path / "data"}; PartitionCommandsResultInfo result_info; for (const auto & disk: disks) { - if (!disk->exists(store_path)) - continue; - for (auto prefix_it = disk->iterateDirectory(store_path); prefix_it->isValid(); prefix_it->next()) + for (auto store_path: store_paths) { - auto prefix_directory = store_path / prefix_it->name(); - for (auto table_it = disk->iterateDirectory(prefix_directory); table_it->isValid(); table_it->next()) + if (!disk->exists(store_path)) + continue; + for (auto prefix_it = disk->iterateDirectory(store_path); prefix_it->isValid(); prefix_it->next()) { - auto table_directory = prefix_directory / table_it->name(); - auto current_result_info = unfreezePartitionsFromTableDirectory([] (const String &) { return true; }, backup_name, {disk}, table_directory, local_context); - for (auto & command_result : current_result_info) + auto prefix_directory = store_path / prefix_it->name(); + for (auto table_it = disk->iterateDirectory(prefix_directory); table_it->isValid(); table_it->next()) { - command_result.command_type = "SYSTEM UNFREEZE"; + auto table_directory = prefix_directory / table_it->name(); + auto current_result_info = unfreezePartitionsFromTableDirectory( + [](const String &) { return true; }, backup_name, {disk}, table_directory, local_context); + for (auto & command_result : current_result_info) + { + command_result.command_type = "SYSTEM UNFREEZE"; + } + result_info.insert( + result_info.end(), + std::make_move_iterator(current_result_info.begin()), + std::make_move_iterator(current_result_info.end())); } - result_info.insert( - result_info.end(), - std::make_move_iterator(current_result_info.begin()), - std::make_move_iterator(current_result_info.end())); } } if (disk->exists(backup_path)) { + /// After unfreezing we need to clear revision.txt file and empty directories disk->removeRecursive(backup_path); } } @@ -143,11 +157,8 @@ bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const Strin FreezeMetaData meta; if (meta.load(disk, path)) { - if (meta.is_replicated) - { - FreezeMetaData::clean(disk, path); - return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context); - } + FreezeMetaData::clean(disk, path); + return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context); } } diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index 290121cfc1e..2026ac393b9 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -23,9 +23,7 @@ private: static String getFileName(const String & path); public: - int version = 1; - bool is_replicated{false}; - bool is_remote{false}; + int version = 2; String replica_name; String zookeeper_name; String table_shared_id; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c24636a56f8..9ea6cc6cea2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1890,7 +1890,7 @@ size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirecory() for (const auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names) { - removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false); + removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name); LOG_DEBUG(log, "Removed broken detached part {} due to a timeout for broken detached parts", old_name); } @@ -4575,7 +4575,7 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names) { - bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false); + bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name); LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared); old_name.clear(); } @@ -6172,7 +6172,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll( return unfreezePartitionsByMatcher([] (const String &) { return true; }, backup_name, local_context); } -bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const String &, bool) +bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const String &) { disk->removeRecursive(path); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 00a56de9142..097615c5841 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -980,7 +980,7 @@ public: /// Check shared data usage on other replicas for detached/freezed part /// Remove local files and remote files if needed - virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed); + virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name); virtual String getTableSharedID() const { return ""; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e93399918ef..df9574930aa 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8185,25 +8185,12 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode( } } -bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) +bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name) { if (disk->supportZeroCopyReplication()) { - if (is_freezed) - { - FreezeMetaData meta; - if (meta.load(disk, path)) - { - FreezeMetaData::clean(disk, path); - return removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", getContext()); - } - } - else - { - String table_id = getTableSharedID(); - - return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext()); - } + String table_id = getTableSharedID(); + return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext()); } disk->removeRecursive(path); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 73a08a2b921..fca76e42ca2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -813,7 +813,7 @@ private: int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false, const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {}); - bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) override; + bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name) override; /// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled. void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose.sh index 1f67100a4b6..38f84c2fa15 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-ordinary-database +# Tags: no-replicated-database, no-parallel # Tag no-replicated-database: Unsupported type of ALTER query CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From ef87e1207cb5f955833bce0fb9e6f5a423f5a551 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 1 Jul 2022 16:43:40 +0000 Subject: [PATCH 019/582] better support of read_in_order in case of fixed prefix of sorting key --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- ...reuseStorageOrderingForWindowFunctions.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 4 +- .../AggregatingInOrderTransform.cpp | 4 +- src/Storages/ReadInOrderOptimizer.cpp | 111 +++++++++++------- src/Storages/ReadInOrderOptimizer.h | 3 +- src/Storages/SelectQueryInfo.h | 13 +- ...02346_read_in_order_fixed_prefix.reference | 78 ++++++++++++ .../02346_read_in_order_fixed_prefix.sh | 81 +++++++++++++ 9 files changed, 238 insertions(+), 60 deletions(-) create mode 100644 tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference create mode 100755 tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cf229442442..728f03703d3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2481,7 +2481,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input auto finish_sorting_step = std::make_unique( query_plan.getCurrentDataStream(), - input_sorting_info->order_key_prefix_descr, + input_sorting_info->sort_description_for_merging, output_order_descr, settings.max_block_size, limit); diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 4d3c268ab7e..2d7b0313955 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -104,7 +104,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, if (order_info) { read_from_merge_tree->setQueryInfoInputOrderInfo(order_info); - sorting->convertToFinishSorting(order_info->order_key_prefix_descr); + sorting->convertToFinishSorting(order_info->sort_description_for_merging); } return 0; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f377709a899..8aacd2bf1ad 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -520,9 +520,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (need_preliminary_merge) { - size_t fixed_prefix_size = input_order_info->order_key_fixed_prefix_descr.size(); - size_t prefix_size = fixed_prefix_size + input_order_info->order_key_prefix_descr.size(); - + size_t prefix_size = input_order_info->used_prefix_of_sorting_key_size; auto order_key_prefix_ast = metadata_for_reading->getSortingKey().expression_list_ast->clone(); order_key_prefix_ast->children.resize(prefix_size); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index ce50ae5eeee..2c7a4e23119 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -41,13 +41,13 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( /// We won't finalize states in order to merge same states (generated due to multi-thread execution) in AggregatingSortedTransform res_header = params->getCustomHeader(/* final_= */ false); - for (size_t i = 0; i < group_by_info->order_key_prefix_descr.size(); ++i) + for (size_t i = 0; i < group_by_info->sort_description_for_merging.size(); ++i) { const auto & column_description = group_by_description_[i]; group_by_description.emplace_back(column_description, res_header.getPositionByName(column_description.column_name)); } - if (group_by_info->order_key_prefix_descr.size() < group_by_description_.size()) + if (group_by_info->sort_description_for_merging.size() < group_by_description_.size()) { group_by_key = true; /// group_by_description may contains duplicates, so we use keys_size from Aggregator::params diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 3ff4baa0b11..90d8198b42a 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -5,9 +5,11 @@ #include #include #include +#include #include #include #include +#include namespace DB { @@ -20,26 +22,43 @@ namespace ErrorCodes namespace { -ASTPtr getFixedPoint(const ASTPtr & ast) +ASTPtr getFixedPoint(const ASTPtr & ast, const ContextPtr & context) { const auto * func = ast->as(); if (!func || func->name != "equals") return nullptr; + if (!func->arguments || func->arguments->children.size() != 2) + return nullptr; + const auto & lhs = func->arguments->children[0]; const auto & rhs = func->arguments->children[1]; - if (lhs->as()) - return rhs; + if (!lhs->as() && !rhs->as()) + return nullptr; - if (rhs->as()) - return lhs; + /// Case of two literals doesn't make sense. + if (lhs->as() && rhs->as()) + return nullptr; - return nullptr; + auto argument = lhs->as() ? rhs : lhs; + while (const auto * arg_func = argument->as()) + { + if (!arg_func->arguments || arg_func->arguments->children.size() != 1) + return nullptr; + + auto func_resolver = FunctionFactory::instance().tryGet(arg_func->name, context); + if (!func_resolver || !func_resolver->isInjective({})) + return nullptr; + + argument = func->arguments->children[0]; + } + + return argument->as() ? argument : nullptr; } -size_t calculateFixedPrefixSize( - const ASTSelectQuery & query, const Names & sorting_key_columns) +NameSet getFixedSortingColumns( + const ASTSelectQuery & query, const Names & sorting_key_columns, const ContextPtr & context) { ASTPtr condition; if (query.where() && query.prewhere()) @@ -50,14 +69,15 @@ size_t calculateFixedPrefixSize( condition = query.prewhere(); if (!condition) - return 0; + return {}; /// Convert condition to CNF for more convenient analysis. auto cnf = TreeCNFConverter::tryConvertToCNF(condition); if (!cnf) - return 0; + return {}; NameSet fixed_points; + NameSet sorting_key_columns_set(sorting_key_columns.begin(), sorting_key_columns.end()); /// If we met expression like 'column = x', where 'x' is literal, /// in clause of size 1 in CNF, then we can guarantee @@ -66,22 +86,17 @@ size_t calculateFixedPrefixSize( { if (group.size() == 1 && !group.begin()->negative) { - auto fixed_point = getFixedPoint(group.begin()->ast); + auto fixed_point = getFixedPoint(group.begin()->ast, context); if (fixed_point) - fixed_points.insert(fixed_point->getColumnName()); + { + auto column_name = fixed_point->getColumnName(); + if (sorting_key_columns_set.contains(column_name)) + fixed_points.insert(column_name); + } } }); - size_t prefix_size = 0; - for (const auto & column_name : sorting_key_columns) - { - if (!fixed_points.contains(column_name)) - break; - - ++prefix_size; - } - - return prefix_size; + return fixed_points; } /// Optimize in case of exact match with order key element @@ -180,46 +195,52 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrderImpl( const StorageMetadataPtr & metadata_snapshot, const SortDescription & description, const ManyExpressionActions & actions, + const ContextPtr & context, UInt64 limit) const { auto sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); int read_direction = description.at(0).direction; - size_t fixed_prefix_size = calculateFixedPrefixSize(query, sorting_key_columns); - size_t descr_prefix_size = std::min(description.size(), sorting_key_columns.size() - fixed_prefix_size); + auto fixed_sorting_columns = getFixedSortingColumns(query, sorting_key_columns, context); - SortDescription order_key_prefix_descr; - order_key_prefix_descr.reserve(descr_prefix_size); + SortDescription sort_description_for_merging; + sort_description_for_merging.reserve(description.size()); - for (size_t i = 0; i < descr_prefix_size; ++i) + size_t desc_pos = 0; + size_t key_pos = 0; + + while (desc_pos < description.size() && key_pos < sorting_key_columns.size()) { - if (forbidden_columns.contains(description[i].column_name)) + if (forbidden_columns.contains(description[desc_pos].column_name)) break; - int current_direction = matchSortDescriptionAndKey( - actions[i]->getActions(), description[i], sorting_key_columns[i + fixed_prefix_size]); + int current_direction = matchSortDescriptionAndKey(actions[desc_pos]->getActions(), description[desc_pos], sorting_key_columns[key_pos]); + bool is_matched = current_direction && (desc_pos == 0 || current_direction == read_direction); + + if (!is_matched) + { + if (fixed_sorting_columns.contains(sorting_key_columns[key_pos])) + { + ++key_pos; + continue; + } - if (!current_direction || (i > 0 && current_direction != read_direction)) break; + } - if (i == 0) + if (desc_pos == 0) read_direction = current_direction; - order_key_prefix_descr.push_back(required_sort_description[i]); + sort_description_for_merging.push_back(description[desc_pos]); + + ++desc_pos; + ++key_pos; } - if (order_key_prefix_descr.empty()) + if (sort_description_for_merging.empty()) return {}; - SortDescription order_key_fixed_prefix_descr; - order_key_fixed_prefix_descr.reserve(fixed_prefix_size); - for (size_t i = 0; i < fixed_prefix_size; ++i) - order_key_fixed_prefix_descr.emplace_back(sorting_key_columns[i], read_direction); - - return std::make_shared( - std::move(order_key_fixed_prefix_descr), - std::move(order_key_prefix_descr), - read_direction, limit); + return std::make_shared(std::move(sort_description_for_merging), key_pos, read_direction, limit); } InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder( @@ -254,10 +275,10 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder( aliases_actions[i] = expression_analyzer.getActions(true); } - return getInputOrderImpl(metadata_snapshot, aliases_sort_description, aliases_actions, limit); + return getInputOrderImpl(metadata_snapshot, aliases_sort_description, aliases_actions, context, limit); } - return getInputOrderImpl(metadata_snapshot, required_sort_description, elements_actions, limit); + return getInputOrderImpl(metadata_snapshot, required_sort_description, elements_actions, context, limit); } } diff --git a/src/Storages/ReadInOrderOptimizer.h b/src/Storages/ReadInOrderOptimizer.h index fd8c9187ddb..161c321ae79 100644 --- a/src/Storages/ReadInOrderOptimizer.h +++ b/src/Storages/ReadInOrderOptimizer.h @@ -12,8 +12,6 @@ namespace DB * common prefix, which is needed for * performing reading in order of PK. */ -class Context; - class ReadInOrderOptimizer { public: @@ -30,6 +28,7 @@ private: const StorageMetadataPtr & metadata_snapshot, const SortDescription & description, const ManyExpressionActions & actions, + const ContextPtr & context, UInt64 limit) const; /// Actions for every element of order expression to analyze functions for monotonicity diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index bdb4c392c48..b124bb67a14 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -87,17 +87,18 @@ struct FilterDAGInfo struct InputOrderInfo { - SortDescription order_key_fixed_prefix_descr; - SortDescription order_key_prefix_descr; + SortDescription sort_description_for_merging; + size_t used_prefix_of_sorting_key_size; + int direction; UInt64 limit; InputOrderInfo( - const SortDescription & order_key_fixed_prefix_descr_, - const SortDescription & order_key_prefix_descr_, + const SortDescription & sort_description_for_merging_, + size_t used_prefix_of_sorting_key_size_, int direction_, UInt64 limit_) - : order_key_fixed_prefix_descr(order_key_fixed_prefix_descr_) - , order_key_prefix_descr(order_key_prefix_descr_) + : sort_description_for_merging(sort_description_for_merging_) + , used_prefix_of_sorting_key_size(used_prefix_of_sorting_key_size_) , direction(direction_), limit(limit_) { } diff --git a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference new file mode 100644 index 00000000000..6e14aec1896 --- /dev/null +++ b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference @@ -0,0 +1,78 @@ +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, a +OK +SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY c, d +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY b, c +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY a, b +OK +SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY b +OK diff --git a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh new file mode 100755 index 00000000000..40cbe61cff5 --- /dev/null +++ b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh @@ -0,0 +1,81 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function check_if_optimzed() +{ + query="$1" + echo $query + ! $CLICKHOUSE_CLIENT --max_threads 8 --optimize_read_in_order 1 -q "EXPLAIN PIPELINE $query" | grep -q "MergeSorting" +} + +function assert_optimized() +{ + check_if_optimzed "$1" && echo "OK" || echo "FAIL" +} + +function assert_not_optimized() +{ + ! check_if_optimzed "$1" && echo "OK" || echo "FAIL" +} + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_fixed_prefix" +$CLICKHOUSE_CLIENT -q " + CREATE TABLE t_fixed_prefix (a UInt32, b UInt32, c UInt32, d UInt32, e UInt32) + ENGINE = MergeTree ORDER BY (a, b, c, d)" + +$CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES t_fixed_prefix" +$CLICKHOUSE_CLIENT -q "INSERT INTO t_fixed_prefix SELECT number % 2, number % 10, number % 100, number % 1000, number FROM numbers(100000)" +$CLICKHOUSE_CLIENT -q "INSERT INTO t_fixed_prefix SELECT number % 2, number % 10, number % 100, number % 1000, number FROM numbers(100000)" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c, d" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a, b" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b, c" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a, c" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY b, c" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a, b" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c, d" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY c, d" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, d" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, c, d" +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c, d" + +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, a" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, c" + +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, b" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, d" + +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY c, d" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY b, c" + +assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY a, b" +assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY b" From 8b356f0e1bbfd60ec0fff2d147727dfd2a3fa83b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 4 Jul 2022 15:05:52 +0000 Subject: [PATCH 020/582] better tests and comments --- src/Storages/ReadInOrderOptimizer.cpp | 7 +- src/Storages/SelectQueryInfo.h | 15 ++ ...ggregation_in_order_fixed_prefix.reference | 8 ++ ...02346_aggregation_in_order_fixed_prefix.sh | 18 +++ .../02346_read_in_order_fixed_prefix.python | 132 ++++++++++++++++++ ...02346_read_in_order_fixed_prefix.reference | 77 ---------- .../02346_read_in_order_fixed_prefix.sh | 79 +---------- 7 files changed, 183 insertions(+), 153 deletions(-) create mode 100644 tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.reference create mode 100755 tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.sh create mode 100644 tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 90d8198b42a..bf8fea1db59 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -22,6 +22,8 @@ namespace ErrorCodes namespace { +/// Finds expression like x = 'y' or f(x) = 'y', +/// where `x` is identifier, 'y' is literal and `f` is injective functions. ASTPtr getFixedPoint(const ASTPtr & ast, const ContextPtr & context) { const auto * func = ast->as(); @@ -41,6 +43,7 @@ ASTPtr getFixedPoint(const ASTPtr & ast, const ContextPtr & context) if (lhs->as() && rhs->as()) return nullptr; + /// If indetifier is wrapped into injective functions, remove them. auto argument = lhs->as() ? rhs : lhs; while (const auto * arg_func = argument->as()) { @@ -51,7 +54,7 @@ ASTPtr getFixedPoint(const ASTPtr & ast, const ContextPtr & context) if (!func_resolver || !func_resolver->isInjective({})) return nullptr; - argument = func->arguments->children[0]; + argument = arg_func->arguments->children[0]; } return argument->as() ? argument : nullptr; @@ -219,6 +222,8 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrderImpl( if (!is_matched) { + /// If one of the sorting columns is constant after filtering, + /// skip it, because it won't affect order anymore. if (fixed_sorting_columns.contains(sorting_key_columns[key_pos])) { ++key_pos; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index b124bb67a14..9e5d8f7091b 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -87,7 +87,22 @@ struct FilterDAGInfo struct InputOrderInfo { + /// Sort description for merging of already sorted streams. + /// Always a prefix of ORDER BY or GROUP BY description specified in query. SortDescription sort_description_for_merging; + + /** Size of prefix of sorting key that is already + * sorted before execution of sorting or aggreagation. + * + * Contains both columns that scpecified in + * ORDER BY or GROUP BY clause of query + * and columns that turned out to be already sorted. + * + * E.g. if we have sorting key ORDER BY (a, b, c, d) + * and query with `WHERE a = 'x' AND b = 'y' ORDER BY c, d` clauses. + * sort_description_for_merging will be equal to (c, d) and + * used_prefix_of_sorting_key_size will be equal to 4. + */ size_t used_prefix_of_sorting_key_size; int direction; diff --git a/tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.reference b/tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.reference new file mode 100644 index 00000000000..fecc2acec7e --- /dev/null +++ b/tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.reference @@ -0,0 +1,8 @@ +1 100000 1 +1 100001 1 +1 100002 1 +ReadType: InOrder +100000 1 +100001 1 +100002 1 +ReadType: InOrder diff --git a/tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.sh b/tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.sh new file mode 100755 index 00000000000..265c7894852 --- /dev/null +++ b/tests/queries/0_stateless/02346_aggregation_in_order_fixed_prefix.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_agg" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_agg ( A Int64, B Int64 ) Engine=MergeTree() ORDER BY (A, B)" +$CLICKHOUSE_CLIENT -q "INSERT INTO test_agg SELECT intDiv(number, 1e5), number FROM numbers(1e6)" + +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order 1 -q "SELECT A, B, count() FROM test_agg where A = 1 GROUP BY A, B ORDER BY A, B LIMIT 3" +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order 1 -q "EXPLAIN actions = 1 SELECT A, B, count() FROM test_agg where A = 1 GROUP BY A, B ORDER BY A, B LIMIT 3" | grep -o "ReadType: InOrder" + +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order 1 -q "SELECT B, count() FROM test_agg where A = 1 GROUP BY B ORDER BY B LIMIT 3" +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order 1 -q "EXPLAIN actions = 1 SELECT B, count() FROM test_agg where A = 1 GROUP BY B ORDER BY B LIMIT 3" | grep -o "ReadType: InOrder" + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_agg" diff --git a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python new file mode 100644 index 00000000000..399533480a9 --- /dev/null +++ b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python @@ -0,0 +1,132 @@ +#!/usr/bin/env python3 +import os +import sys +from itertools import chain, combinations, permutations + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from pure_http_client import ClickHouseClient + +client = ClickHouseClient() + + +def powerset(iterable): + s = list(iterable) + return chain.from_iterable(combinations(s, r) for r in range(len(s) + 1)) + + +queries = [ + {"optimize": True, "where": [], "order_by": ["a"]}, + {"optimize": True, "where": [], "order_by": ["a", "b"]}, + {"optimize": True, "where": [], "order_by": ["a", "b", "c"]}, + {"optimize": True, "where": [], "order_by": ["a", "b", "c", "d"]}, + {"optimize": True, "where": ["a"], "order_by": ["a"]}, + {"optimize": True, "where": ["a"], "order_by": ["a", "b"]}, + {"optimize": True, "where": ["a"], "order_by": ["b"]}, + {"optimize": True, "where": ["a"], "order_by": ["b", "c"]}, + {"optimize": True, "where": ["b"], "order_by": ["a"]}, + {"optimize": True, "where": ["b"], "order_by": ["a", "c"]}, + {"optimize": False, "where": ["b"], "order_by": ["b", "c"]}, + {"optimize": True, "where": ["c"], "order_by": ["a"]}, + {"optimize": True, "where": ["c"], "order_by": ["a", "b"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["a"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["a", "b"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["a", "c"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["a", "b", "c"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["a", "b", "c", "d"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["b", "c"]}, + {"optimize": True, "where": ["a", "b"], "order_by": ["c", "d"]}, + {"optimize": True, "where": ["a", "c"], "order_by": ["a"]}, + {"optimize": True, "where": ["a", "c"], "order_by": ["a", "b"]}, + {"optimize": True, "where": ["a", "c"], "order_by": ["b", "d"]}, + {"optimize": True, "where": ["a", "c"], "order_by": ["a", "b", "c"]}, + {"optimize": True, "where": ["a", "c"], "order_by": ["b", "c", "d"]}, + {"optimize": True, "where": ["a", "c"], "order_by": ["a", "b", "c", "d"]}, + {"optimize": False, "where": [], "order_by": ["b"]}, + {"optimize": False, "where": [], "order_by": ["b", "a"]}, + {"optimize": False, "where": [], "order_by": ["b", "c"]}, + {"optimize": False, "where": ["a"], "order_by": ["c"]}, + {"optimize": False, "where": ["a"], "order_by": ["c", "b"]}, + {"optimize": False, "where": ["a"], "order_by": ["c", "d"]}, + {"optimize": False, "where": ["c"], "order_by": ["c", "d"]}, + {"optimize": False, "where": ["c"], "order_by": ["b", "c"]}, +] + + +client.query("DROP TABLE IF EXISTS t_fixed_prefix") +client.query( + """ +CREATE TABLE t_fixed_prefix (a UInt32, b UInt32, c UInt32, d UInt32, e UInt32) +ENGINE = MergeTree ORDER BY (a, b, c, d)""" +) + +client.query("SYSTEM STOP MERGES t_fixed_prefix") + +# create several parts +for _ in range(4): + client.query( + "INSERT INTO t_fixed_prefix SELECT number % 2, number % 10, number % 100, number % 1000, number FROM numbers(25000)" + ) + + +def check_query( + fixed_columns, order_by_columns, should_be_optimized, should_use_finish_sorting +): + where_clause = " AND ".join([f"{c} = 1" for c in fixed_columns]) + order_by_clause = ", ".join(order_by_columns) + + query = "SELECT {} FROM t_fixed_prefix".format(order_by_clause) + if len(where_clause) != 0: + query += " WHERE " + where_clause + + if len(order_by_clause) != 0: + query += " ORDER BY " + order_by_clause + + query += " SETTINGS optimize_read_in_order = {}" + + res_optimized = client.query(query.format(1)) + res_not_optimized = client.query(query.format(0)) + + if res_optimized != res_not_optimized: + print("Failed query {}. Result of queries mismatched".format(query)) + exit(1) + + res_explain = client.query("EXPLAIN PIPELINE {}".format(query.format(1))) + + is_optimized = "MergeSortingTransform" not in res_explain + uses_finish_sorting = "FinishSortingTransform" in res_explain + + if ( + is_optimized != should_be_optimized + or uses_finish_sorting != should_use_finish_sorting + ): + print( + """ +Wrong query pipeline is built for query {}: +{} +Should be optimized: {}. +Is optimized: {}. +Should use FinishSortingTransform: {}. +Uses FinishSortingTransform: {} +""".format( + query.format(1), + res_explain, + should_be_optimized, + is_optimized, + should_use_finish_sorting, + uses_finish_sorting, + ) + ) + exit(1) + + +for query in queries: + check_query(query["where"], query["order_by"], query["optimize"], False) + check_query(query["where"], query["order_by"] + ["e"], query["optimize"], query["optimize"]) + + where_columns = [f"bitNot({col})" for col in query["where"]] + check_query(where_columns, query["order_by"], query["optimize"], False) + check_query(where_columns, query["order_by"] + ["e"], query["optimize"], query["optimize"]) + +print("OK") diff --git a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference index 6e14aec1896..d86bac9de59 100644 --- a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference +++ b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.reference @@ -1,78 +1 @@ -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, a -OK -SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY c, d -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY b, c -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY a, b -OK -SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY b OK diff --git a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh index 40cbe61cff5..e1a393af0c5 100755 --- a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh +++ b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.sh @@ -1,81 +1,10 @@ #!/usr/bin/env bash +# Tags: no-fasttest, long +# Tag no-fasttest: Require python libraries like scipy, pandas and numpy CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function check_if_optimzed() -{ - query="$1" - echo $query - ! $CLICKHOUSE_CLIENT --max_threads 8 --optimize_read_in_order 1 -q "EXPLAIN PIPELINE $query" | grep -q "MergeSorting" -} - -function assert_optimized() -{ - check_if_optimzed "$1" && echo "OK" || echo "FAIL" -} - -function assert_not_optimized() -{ - ! check_if_optimzed "$1" && echo "OK" || echo "FAIL" -} - -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_fixed_prefix" -$CLICKHOUSE_CLIENT -q " - CREATE TABLE t_fixed_prefix (a UInt32, b UInt32, c UInt32, d UInt32, e UInt32) - ENGINE = MergeTree ORDER BY (a, b, c, d)" - -$CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES t_fixed_prefix" -$CLICKHOUSE_CLIENT -q "INSERT INTO t_fixed_prefix SELECT number % 2, number % 10, number % 100, number % 1000, number FROM numbers(100000)" -$CLICKHOUSE_CLIENT -q "INSERT INTO t_fixed_prefix SELECT number % 2, number % 10, number % 100, number % 1000, number FROM numbers(100000)" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY a, b, c, d" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY a, b" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY b, c" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY a, c" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE b = 1 ORDER BY b, c" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY a, b" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY a, b, c, d" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY b, a, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND b = 1 ORDER BY c, d" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, d" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY b, c, d" -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 AND c = 1 ORDER BY a, b, c, d" - -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, a" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix ORDER BY b, c" - -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, b" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 ORDER BY c, d" - -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY c, d" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE c = 1 ORDER BY b, c" - -assert_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY a, b" -assert_not_optimized "SELECT a, b, c, d, e FROM t_fixed_prefix WHERE a = 1 OR b = 1 ORDER BY b" +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02346_read_in_order_fixed_prefix.python From 2e8c530bedacd9e5d0c777ba466db98dfb9ca1f8 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:41:10 +0800 Subject: [PATCH 021/582] Move thread trace context out of ThreadStatus --- src/Common/OpenTelemetryTraceContext.h | 77 ++++++- src/Common/OpenTelemtryTraceContext.cpp | 263 ++++++++++++++++++++++++ src/Common/ThreadStatus.cpp | 1 - src/Common/ThreadStatus.h | 6 - src/Interpreters/Context.cpp | 72 ++++--- src/Interpreters/Context.h | 3 +- src/Interpreters/OpenTelemetrySpanLog.h | 22 +- src/Interpreters/ThreadStatusExt.cpp | 59 ------ 8 files changed, 393 insertions(+), 110 deletions(-) create mode 100644 src/Common/OpenTelemtryTraceContext.cpp diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 4d2fc656100..55b9d5bb49c 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -1,11 +1,44 @@ #pragma once -#include -#include +#include namespace DB { +struct OpenTelemetrySpan +{ + UUID trace_id; + UInt64 span_id; + UInt64 parent_span_id; + std::string operation_name; + UInt64 start_time_us; + UInt64 finish_time_us; + Map attributes; + + void addAttribute(const std::string& name, UInt64 value); + void addAttributeIfNotZero(const std::string& name, UInt64 value) + { + if (value != 0) + addAttribute(name, value); + } + + void addAttribute(const std::string& name, const std::string& value); + void addAttribute(const Exception & e); + void addAttribute(std::exception_ptr e); + + bool isTraceEnabled() const + { + return trace_id != UUID(); + } +}; + +struct OpenTelemetrySpanHolder; + +class Context; +using ContextPtr = std::shared_ptr; + +class OpenTelemetrySpanLog; + // The runtime info we need to create new OpenTelemetry spans. struct OpenTelemetryTraceContext { @@ -19,6 +52,46 @@ struct OpenTelemetryTraceContext // Parse/compose OpenTelemetry traceparent header. bool parseTraceparentHeader(const std::string & traceparent, std::string & error); std::string composeTraceparentHeader() const; + + bool isTraceEnabled() const + { + return trace_id != UUID(); + } }; +// tracing context kept on thread local +struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext +{ + OpenTelemetryThreadTraceContext& operator =(const OpenTelemetryTraceContext& context) + { + *(static_cast(this)) = context; + return *this; + } + + void reset(); + + static const OpenTelemetryThreadTraceContext& current(); + + std::weak_ptr span_log; +}; + +struct OpenTelemetryThreadTraceContextScope +{ + // forbidden copy ctor and assignment to make the destructor safe + OpenTelemetryThreadTraceContextScope(const OpenTelemetryThreadTraceContextScope& scope) = delete; + OpenTelemetryThreadTraceContextScope& operator =(const OpenTelemetryThreadTraceContextScope& scope) = delete; + + OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + const OpenTelemetryTraceContext& _parent_trace_context, + const std::weak_ptr& _log); + + ~OpenTelemetryThreadTraceContextScope(); + + + + OpenTelemetrySpan root_span; +}; + +using OpenTelemetryThreadTraceContextScopePtr = std::unique_ptr; + } diff --git a/src/Common/OpenTelemtryTraceContext.cpp b/src/Common/OpenTelemtryTraceContext.cpp new file mode 100644 index 00000000000..162a741f97f --- /dev/null +++ b/src/Common/OpenTelemtryTraceContext.cpp @@ -0,0 +1,263 @@ +#include "Interpreters/OpenTelemetrySpanLog.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +thread_local OpenTelemetryThreadTraceContext current_thread_trace_context; + +void OpenTelemetrySpan::addAttribute(const std::string& name, UInt64 value) +{ + if (trace_id == UUID() || name.empty()) + return; + + this->attributes.push_back(Tuple{name, toString(value)}); +} + +void OpenTelemetrySpan::addAttribute(const std::string& name, const std::string& value) +{ + if (trace_id == UUID() || name.empty() || value.empty()) + return; + + this->attributes.push_back(Tuple{name, value}); +} + +void OpenTelemetrySpan::addAttribute(const Exception & e) +{ + if (trace_id == UUID()) + return; + + this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); +} + +void OpenTelemetrySpan::addAttribute(std::exception_ptr e) +{ + if (trace_id == UUID() || e == nullptr) + return; + + this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); +} + +OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_name) +{ + if (current_thread_trace_context.trace_id == UUID()) + { + this->trace_id = 0; + this->span_id = 0; + this->parent_span_id = 0; + } + else + { + this->trace_id = current_thread_trace_context.trace_id; + this->parent_span_id = current_thread_trace_context.span_id; + this->span_id = thread_local_rng(); // create a new id for this span + this->operation_name = _operation_name; + this->start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + // set current span id to this + current_thread_trace_context.span_id = this->span_id; + } +} + +void OpenTelemetrySpanHolder::finish() +{ + if (trace_id == UUID()) + return; + + // First of all, return old value of current span. + assert(current_thread_trace_context.span_id == span_id); + current_thread_trace_context.span_id = parent_span_id; + + try + { + auto log = current_thread_trace_context.span_log.lock(); + if (!log) + { + // The log might be disabled. + return; + } + + this->finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + log->add(OpenTelemetrySpanLogElement(*this)); + } + catch (...) + { + tryLogCurrentException(__FUNCTION__); + } + + trace_id = UUID(); +} + +OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() +{ + finish(); +} + + +template +static T readHex(const char * data) +{ + T x{}; + + const char * end = data + sizeof(T) * 2; + while (data < end) + { + x *= 16; + x += unhex(*data); + ++data; + } + + return x; +} + + +bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, + std::string & error) +{ + trace_id = 0; + + // Version 00, which is the only one we can parse, is fixed width. Use this + // fact for an additional sanity check. + const int expected_length = strlen("xx-xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx-xxxxxxxxxxxxxxxx-xx"); + if (traceparent.length() != expected_length) + { + error = fmt::format("unexpected length {}, expected {}", + traceparent.length(), expected_length); + return false; + } + + const char * data = traceparent.data(); + + uint8_t version = unhex2(data); + data += 2; + + if (version != 0) + { + error = fmt::format("unexpected version {}, expected 00", version); + return false; + } + + if (*data != '-') + { + error = fmt::format("Malformed traceparant header: {}", traceparent); + return false; + } + + ++data; + UInt64 trace_id_higher_64 = unhexUInt(data); + UInt64 trace_id_lower_64 = unhexUInt(data + 16); + data += 32; + + if (*data != '-') + { + error = fmt::format("Malformed traceparant header: {}", traceparent); + return false; + } + + ++data; + UInt64 span_id_64 = unhexUInt(data); + data += 16; + + if (*data != '-') + { + error = fmt::format("Malformed traceparant header: {}", traceparent); + return false; + } + + ++data; + this->trace_flags = unhex2(data); + this->trace_id.toUnderType().items[0] = trace_id_higher_64; + this->trace_id.toUnderType().items[1] = trace_id_lower_64; + this->span_id = span_id_64; + return true; +} + + +std::string OpenTelemetryTraceContext::composeTraceparentHeader() const +{ + // This span is a parent for its children, so we specify this span_id as a + // parent id. + return fmt::format("00-{:016x}{:016x}-{:016x}-{:02x}", + trace_id.toUnderType().items[0], + trace_id.toUnderType().items[1], + span_id, + // This cast is needed because fmt is being weird and complaining that + // "mixing character types is not allowed". + static_cast(trace_flags)); +} + +const OpenTelemetryThreadTraceContext& OpenTelemetryThreadTraceContext::current() +{ + return current_thread_trace_context; +} + +void OpenTelemetryThreadTraceContext::reset() +{ + this->trace_id = 0; + this->span_id = 0; + this->trace_flags = 0; + this->tracestate = ""; + this->span_log.reset(); +} + +OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + const OpenTelemetryTraceContext& _parent_trace_context, + const std::weak_ptr& _span_log) +{ + if (_parent_trace_context.isTraceEnabled()) + { + this->root_span.trace_id = _parent_trace_context.trace_id; + this->root_span.parent_span_id = _parent_trace_context.span_id; + this->root_span.span_id = thread_local_rng(); + this->root_span.operation_name = _operation_name; + this->root_span.start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + } + else + { + this->root_span.trace_id = 0; + this->root_span.span_id = 0; + } + + // set trace context on the thread local + current_thread_trace_context = _parent_trace_context; + current_thread_trace_context.span_id = this->root_span.span_id; + current_thread_trace_context.span_log = _span_log; +} + +OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() +{ + if (this->root_span.trace_id != UUID()) + { + auto shared_span_log = current_thread_trace_context.span_log.lock(); + if (shared_span_log) + { + this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); + this->root_span.finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); + } + + this->root_span.trace_id = 0; + } + + // restore thread local variables + current_thread_trace_context.reset(); +} + + +} + diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index a4f99c1be1a..077fbcb5bda 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index c80150a8fe8..788b1cc7fcb 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -135,12 +135,6 @@ public: using Deleter = std::function; Deleter deleter; - // This is the current most-derived OpenTelemetry span for this thread. It - // can be changed throughout the query execution, whenever we enter a new - // span or exit it. See OpenTelemetrySpanHolder that is normally responsible - // for these changes. - OpenTelemetryTraceContext thread_trace_context; - protected: ThreadGroupStatusPtr thread_group; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ca6ff02b994..13d78a690b5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -134,7 +134,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; - extern const int NOT_IMPLEMENTED; } @@ -1328,29 +1327,6 @@ void Context::setCurrentQueryId(const String & query_id) random.words.a = thread_local_rng(); //-V656 random.words.b = thread_local_rng(); //-V656 - if (client_info.client_trace_context.trace_id != UUID()) - { - // Use the OpenTelemetry trace context we received from the client, and - // create a new span for the query. - query_trace_context = client_info.client_trace_context; - query_trace_context.span_id = thread_local_rng(); - } - else if (client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY) - { - // If this is an initial query without any parent OpenTelemetry trace, we - // might start the trace ourselves, with some configurable probability. - std::bernoulli_distribution should_start_trace{ - settings.opentelemetry_start_trace_probability}; - - if (should_start_trace(thread_local_rng)) - { - // Use the randomly generated default query id as the new trace id. - query_trace_context.trace_id = random.uuid; - query_trace_context.span_id = thread_local_rng(); - // Mark this trace as sampled in the flags. - query_trace_context.trace_flags = 1; - } - } String query_id_to_set = query_id; if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves. @@ -3449,4 +3425,52 @@ WriteSettings Context::getWriteSettings() const return res; } +OpenTelemetryThreadTraceContextScopePtr Context::startTracing(const std::string& name) +{ + OpenTelemetryThreadTraceContextScopePtr trace_context; + if (this->client_info.client_trace_context.trace_id != UUID()) + { + // Use the OpenTelemetry trace context we received from the client, and + // initialize the tracing context for this query on current thread + return std::make_unique(name, + this->client_info.client_trace_context, + this->getOpenTelemetrySpanLog()); + } + + // start the trace ourselves, with some configurable probability. + std::bernoulli_distribution should_start_trace{settings.opentelemetry_start_trace_probability}; + if (!should_start_trace(thread_local_rng)) + { + return trace_context; + } + + /// 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 + + OpenTelemetryTraceContext query_trace_context; + query_trace_context.trace_id = random.uuid; + query_trace_context.span_id = 0; + // Mark this trace as sampled in the flags. + query_trace_context.trace_flags = 1; + + return std::make_unique(name, + query_trace_context, + this->getOpenTelemetrySpanLog()); + +} + } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 83193dd589b..37b831ff18f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -361,8 +361,7 @@ private: inline static ContextPtr global_context_instance; public: - // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. - OpenTelemetryTraceContext query_trace_context; + OpenTelemetryThreadTraceContextScopePtr startTracing(const std::string& name); private: using SampleBlockCache = std::unordered_map; diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 34f4765c8c4..2f2cc7a1bea 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -1,29 +1,15 @@ #pragma once #include -#include -#include +#include namespace DB { -struct OpenTelemetrySpan -{ - UUID trace_id; - UInt64 span_id; - UInt64 parent_span_id; - std::string operation_name; - UInt64 start_time_us; - UInt64 finish_time_us; - Map attributes; - // I don't understand how Links work, namely, which direction should they - // point to, and how they are related with parent_span_id, so no Links for now. -}; - struct OpenTelemetrySpanLogElement : public OpenTelemetrySpan { OpenTelemetrySpanLogElement() = default; - explicit OpenTelemetrySpanLogElement(const OpenTelemetrySpan & span) + OpenTelemetrySpanLogElement(const OpenTelemetrySpan & span) : OpenTelemetrySpan(span) {} static std::string name() { return "OpenTelemetrySpanLog"; } @@ -41,14 +27,18 @@ public: using SystemLog::SystemLog; }; +typedef std::shared_ptr OpenTelemetrySpanLogPtr; + struct OpenTelemetrySpanHolder : public OpenTelemetrySpan { + OpenTelemetrySpanHolder(const OpenTelemetryTraceContext& _trace_context, OpenTelemetrySpanLogPtr _span_log, const std::string & _operation_name); explicit OpenTelemetrySpanHolder(const std::string & _operation_name); void addAttribute(const std::string& name, UInt64 value); void addAttribute(const std::string& name, const std::string& value); void addAttribute(const Exception & e); void addAttribute(std::exception_ptr e); + void finish(); ~OpenTelemetrySpanHolder(); }; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 53d7fd0457a..d2371a6b192 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -84,15 +84,6 @@ void ThreadStatus::attachQueryContext(ContextPtr query_context_) thread_group->global_context = global_context; } - // Generate new span for thread manually here, because we can't depend - // on OpenTelemetrySpanHolder due to link order issues. - // FIXME why and how is this different from setupState()? - thread_trace_context = query_context_->query_trace_context; - if (thread_trace_context.trace_id != UUID()) - { - thread_trace_context.span_id = thread_local_rng(); - } - applyQuerySettings(); } @@ -132,18 +123,6 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) if (auto query_context_ptr = query_context.lock()) { applyQuerySettings(); - - // Generate new span for thread manually here, because we can't depend - // on OpenTelemetrySpanHolder due to link order issues. - thread_trace_context = query_context_ptr->query_trace_context; - if (thread_trace_context.trace_id != UUID()) - { - thread_trace_context.span_id = thread_local_rng(); - } - } - else - { - thread_trace_context.trace_id = 0; } initPerformanceCounters(); @@ -353,42 +332,6 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); - std::shared_ptr opentelemetry_span_log; - auto query_context_ptr = query_context.lock(); - if (thread_trace_context.trace_id != UUID() && query_context_ptr) - { - opentelemetry_span_log = query_context_ptr->getOpenTelemetrySpanLog(); - } - - if (opentelemetry_span_log) - { - // Log the current thread span. - // We do this manually, because we can't use OpenTelemetrySpanHolder as a - // ThreadStatus member, because of linking issues. This file is linked - // separately, so we can reference OpenTelemetrySpanLog here, but if we had - // the span holder as a field, we would have to reference it in the - // destructor, which is in another library. - OpenTelemetrySpanLogElement span; - - span.trace_id = thread_trace_context.trace_id; - // All child span holders should be finished by the time we detach this - // thread, so the current span id should be the thread span id. If not, - // an assertion for a proper parent span in ~OpenTelemetrySpanHolder() - // is going to fail, because we're going to reset it to zero later in - // this function. - span.span_id = thread_trace_context.span_id; - assert(query_context_ptr); - span.parent_span_id = query_context_ptr->query_trace_context.span_id; - span.operation_name = getThreadName(); - span.start_time_us = query_start_time_microseconds; - span.finish_time_us = - std::chrono::duration_cast( - std::chrono::system_clock::now().time_since_epoch()).count(); - span.attributes.push_back(Tuple{"clickhouse.thread_id", toString(thread_id)}); - - opentelemetry_span_log->add(span); - } - finalizeQueryProfiler(); finalizePerformanceCounters(); @@ -404,8 +347,6 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) query_id.clear(); query_context.reset(); - thread_trace_context.trace_id = 0; - thread_trace_context.span_id = 0; thread_group.reset(); thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; From 0a44a44feb470986fcf459cc9cacc4f2a9ee6e93 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:41:27 +0800 Subject: [PATCH 022/582] Start tracing at HTTP entry --- src/Server/HTTPHandler.cpp | 41 +++++++++++++++++++++++++------------- 1 file changed, 27 insertions(+), 14 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index cdf856e87d5..ec825fdd5e8 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -533,20 +533,7 @@ void HTTPHandler::processQuery( session->makeSessionContext(session_id, session_timeout, session_check == "1"); } - // Parse the OpenTelemetry traceparent header. - ClientInfo client_info = session->getClientInfo(); - if (request.has("traceparent")) - { - std::string opentelemetry_traceparent = request.get("traceparent"); - std::string error; - if (!client_info.client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) - { - LOG_DEBUG(log, "Failed to parse OpenTelemetry traceparent header '{}': {}", opentelemetry_traceparent, error); - } - client_info.client_trace_context.tracestate = request.get("tracestate", ""); - } - - auto context = session->makeQueryContext(std::move(client_info)); + auto context = session->makeQueryContext(session->getClientInfo()); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -944,6 +931,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// In case of exception, send stack trace to client. bool with_stacktrace = false; + OpenTelemetryThreadTraceContextScopePtr thread_trace_context; try { if (request.getMethod() == HTTPServerRequest::HTTP_OPTIONS) @@ -951,6 +939,26 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse processOptionsRequest(response, server.config()); return; } + + // Parse the OpenTelemetry traceparent header. + ClientInfo& client_info = session->getClientInfo(); + if (request.has("traceparent")) + { + std::string opentelemetry_traceparent = request.get("traceparent"); + std::string error; + if (!client_info.client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) + { + LOG_DEBUG(log, "Failed to parse OpenTelemetry traceparent header '{}': {}", opentelemetry_traceparent, error); + } + client_info.client_trace_context.tracestate = request.get("tracestate", ""); + } + + thread_trace_context = request_context->startTracing("HTTPHandler::handleRequest()"); + if (thread_trace_context) + { + thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); + } + response.setContentType("text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); /// For keep-alive to work. @@ -1001,6 +1009,11 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse trySendExceptionToClient(exception_message, exception_code, request, response, used_output); } + if (thread_trace_context) + { + thread_trace_context->root_span.addAttribute("clickhouse.http_status", response.getStatus()); + } + used_output.finalize(); } From 32931857f0bc24ed6530bd722ed0922cd02e34e1 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:41:43 +0800 Subject: [PATCH 023/582] Propagate tracing context across threads --- src/Common/OpenTelemetryTraceContext.h | 8 ++++++++ src/Common/ThreadPool.cpp | 26 +++++++++++++++++++++++++- src/Common/ThreadPool.h | 5 +++-- 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 55b9d5bb49c..76119c8f538 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -85,6 +85,14 @@ struct OpenTelemetryThreadTraceContextScope const OpenTelemetryTraceContext& _parent_trace_context, const std::weak_ptr& _log); + OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + const OpenTelemetryThreadTraceContext& _parent_thread_trace_context) + : OpenTelemetryThreadTraceContextScope(_operation_name, + _parent_thread_trace_context, + _parent_thread_trace_context.span_log) + { + } + ~OpenTelemetryThreadTraceContextScope(); diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 3f5091af0c9..5981b80f346 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -149,7 +150,10 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti } } - jobs.emplace(std::move(job), priority); + // this scheduleImpl is called in the parent thread, + // the tracing context on this thread is used as parent context for the sub-thread that runs the job + auto& current_thread_context = DB::OpenTelemetryThreadTraceContext::current(); + jobs.emplace(std::move(job), priority, current_thread_context); ++scheduled_jobs; new_job_or_shutdown.notify_one(); } @@ -248,6 +252,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ setThreadName("ThreadPool"); Job job; + DB::OpenTelemetryThreadTraceContext parent_thead_trace_context; // A copy of parent trace context bool need_shutdown = false; { @@ -260,6 +265,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// boost::priority_queue does not provide interface for getting non-const reference to an element /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); + parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); jobs.pop(); } else @@ -272,6 +278,10 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!need_shutdown) { + // set up tracing context for this thread by its parent context + DB::OpenTelemetryThreadTraceContextScope thread_trace_context("ThreadPool::worker()" , + parent_thead_trace_context); + try { ALLOW_ALLOCATIONS_IN_SCOPE; @@ -279,15 +289,29 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ std::is_same_v ? CurrentMetrics::GlobalThreadActive : CurrentMetrics::LocalThreadActive); job(); + + if (thread_trace_context.root_span.isTraceEnabled()) + { + // Use the thread name as operation name so that the tracing log will be more clear. + // the thread name is usually set in the jobs, we can only get the name after the job finishes + std::string thread_name = getThreadName(); + if (!thread_name.empty()) + thread_trace_context.root_span.operation_name = thread_name; + } + /// job should be reset before decrementing scheduled_jobs to /// ensure that the Job destroyed before wait() returns. job = {}; + parent_thead_trace_context.reset(); } catch (...) { + thread_trace_context.root_span.addAttribute(std::current_exception()); + /// job should be reset before decrementing scheduled_jobs to /// ensure that the Job destroyed before wait() returns. job = {}; + parent_thead_trace_context.reset(); { std::lock_guard lock(mutex); diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 4bd1cf391ef..5e53dfd3fa9 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -96,9 +96,10 @@ private: { Job job; int priority; + DB::OpenTelemetryThreadTraceContext thread_trace_context; - JobWithPriority(Job job_, int priority_) - : job(job_), priority(priority_) {} + JobWithPriority(Job job_, int priority_, const DB::OpenTelemetryThreadTraceContext& thread_trace_context_) + : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} bool operator< (const JobWithPriority & rhs) const { From 8c044561480aad5145bebc6af4a2673a79c1c9b2 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:41:58 +0800 Subject: [PATCH 024/582] Fix query span at executeQuery layer --- src/Interpreters/executeQuery.cpp | 76 ++++++++++--------------------- src/Server/HTTPHandler.cpp | 4 ++ 2 files changed, 29 insertions(+), 51 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 85c4ea261a0..baded3c1a21 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -232,7 +232,7 @@ inline UInt64 time_in_seconds(std::chrono::time_point return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); } -static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast) +static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, std::shared_ptr& query_span) { /// Exception before the query execution. if (auto quota = context->getQuota()) @@ -290,30 +290,11 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr if (auto query_log = context->getQueryLog()) query_log->add(elem); - if (auto opentelemetry_span_log = context->getOpenTelemetrySpanLog(); - context->query_trace_context.trace_id != UUID() - && opentelemetry_span_log) - { - OpenTelemetrySpanLogElement span; - span.trace_id = context->query_trace_context.trace_id; - span.span_id = context->query_trace_context.span_id; - span.parent_span_id = context->getClientInfo().client_trace_context.span_id; - span.operation_name = "query"; - span.start_time_us = current_time_us; - span.finish_time_us = time_in_microseconds(std::chrono::system_clock::now()); - span.attributes.reserve(6); - span.attributes.push_back(Tuple{"clickhouse.query_status", "ExceptionBeforeStart"}); - span.attributes.push_back(Tuple{"db.statement", elem.query}); - span.attributes.push_back(Tuple{"clickhouse.query_id", elem.client_info.current_query_id}); - span.attributes.push_back(Tuple{"clickhouse.exception", elem.exception}); - span.attributes.push_back(Tuple{"clickhouse.exception_code", toString(elem.exception_code)}); - if (!context->query_trace_context.tracestate.empty()) - { - span.attributes.push_back(Tuple{"clickhouse.tracestate", context->query_trace_context.tracestate}); - } - - opentelemetry_span_log->add(span); - } + query_span->attribute_names.addAttribute("clickhouse.exception_code", elem.exception_code); + query_span->attribute_names.addAttribute("clickhouse.exception", elem.exception); + query_span->attribute_names.addAttribute("db.statement", elem.query); + query_span->attribute_names.addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->finish(); ProfileEvents::increment(ProfileEvents::FailedQuery); @@ -840,7 +821,8 @@ static std::tuple executeQueryImpl( log_processors_profiles = settings.log_processors_profiles, status_info_to_query_log, implicit_txn_control, - pulling_pipeline = pipeline.pulling()](QueryPipeline & query_pipeline) mutable + pulling_pipeline = pipeline.pulling(), + query_span](QueryPipeline & query_pipeline) mutable { QueryStatus * process_list_elem = context->getProcessListElement(); @@ -931,29 +913,14 @@ static std::tuple executeQueryImpl( } } - if (auto opentelemetry_span_log = context->getOpenTelemetrySpanLog(); - context->query_trace_context.trace_id != UUID() - && opentelemetry_span_log) - { - OpenTelemetrySpanLogElement span; - span.trace_id = context->query_trace_context.trace_id; - span.span_id = context->query_trace_context.span_id; - span.parent_span_id = context->getClientInfo().client_trace_context.span_id; - span.operation_name = "query"; - span.start_time_us = elem.query_start_time_microseconds; - span.finish_time_us = time_in_microseconds(finish_time); - - span.attributes.reserve(4); - span.attributes.push_back(Tuple{"clickhouse.query_status", "QueryFinish"}); - span.attributes.push_back(Tuple{"db.statement", elem.query}); - span.attributes.push_back(Tuple{"clickhouse.query_id", elem.client_info.current_query_id}); - if (!context->query_trace_context.tracestate.empty()) - { - span.attributes.push_back(Tuple{"clickhouse.tracestate", context->query_trace_context.tracestate}); - } - - opentelemetry_span_log->add(span); - } + query_span->addAttribute("db.statement", elem.query); + query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); + query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); + query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); + query_span->addAttributeIfNotZero("clickhouse.written_bytes", elem.written_bytes); + query_span->addAttributeIfNotZero("clickhouse.memory_usage", elem.memory_usage); + query_span->finish(); if (implicit_txn_control) { @@ -980,7 +947,8 @@ static std::tuple executeQueryImpl( log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), quota(quota), status_info_to_query_log, - implicit_txn_control]() mutable + implicit_txn_control, + query_span]() mutable { if (implicit_txn_control) { @@ -1037,6 +1005,12 @@ static std::tuple executeQueryImpl( { ProfileEvents::increment(ProfileEvents::FailedInsertQuery); } + + query_span->addAttribute("db.statement", elem.query); + query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttribute("clickhouse.exception", elem.exception); + query_span->addAttribute("clickhouse.exception_code", elem.exception_code); + query_span->finish(); }; res.finish_callback = std::move(finish_callback); @@ -1060,7 +1034,7 @@ static std::tuple executeQueryImpl( if (query_for_logging.empty()) query_for_logging = prepareQueryForLogging(query, context); - onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast); + onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast, query_span); } throw; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ec825fdd5e8..036ae7951a9 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -956,6 +956,10 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse thread_trace_context = request_context->startTracing("HTTPHandler::handleRequest()"); if (thread_trace_context) { + if (!client_info.client_trace_context.tracestate.empty()) + { + thread_trace_context->root_span.addAttribute("clickhouse.tracestate", client_info.client_trace_context.tracestate); + } thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); } From 19f3f3e6caa67c84587d9d1784a715c46ce72bd0 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:42:22 +0800 Subject: [PATCH 025/582] Initialize thread tracing context for TCPServer --- src/Server/TCPHandler.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d1d762d3c61..8bb39d881d1 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -197,6 +197,7 @@ void TCPHandler::runImpl() /// Initialized later. std::optional query_scope; + OpenTelemetryThreadTraceContextScopePtr thread_trace_context; /** An exception during the execution of request (it must be sent over the network to the client). * The client will be able to accept it, if it did not happen while sending another packet and the client has not disconnected yet. @@ -222,6 +223,8 @@ void TCPHandler::runImpl() if (state.empty() && state.part_uuids_to_ignore && !receivePacket()) continue; + thread_trace_context = query_context->startTracing("TCPHandler"); + query_scope.emplace(query_context); /// If query received, then settings in query_context has been updated. @@ -396,6 +399,7 @@ void TCPHandler::runImpl() /// (i.e. deallocations from the Aggregator with two-level aggregation) state.reset(); query_scope.reset(); + thread_trace_context.reset(); } catch (const Exception & e) { @@ -458,6 +462,9 @@ void TCPHandler::runImpl() { if (exception) { + if (thread_trace_context) + thread_trace_context->root_span.addAttribute(exception.value()); + try { /// Try to send logs to client, but it could be risky too @@ -501,6 +508,7 @@ void TCPHandler::runImpl() /// (i.e. deallocations from the Aggregator with two-level aggregation) state.reset(); query_scope.reset(); + thread_trace_context.reset(); } catch (...) { From 57a7e4a7c96afa6e5863169d1cb1ee06fb339322 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 16 Mar 2022 16:45:17 +0800 Subject: [PATCH 026/582] Remove old API reference --- src/Client/Connection.cpp | 17 +++++++++++++++++ src/Common/ThreadStatus.h | 1 - src/Interpreters/executeQuery.cpp | 2 +- src/QueryPipeline/RemoteInserter.cpp | 16 ---------------- src/QueryPipeline/RemoteQueryExecutor.cpp | 4 ---- src/Storages/StorageURL.cpp | 21 ++++++++++----------- 6 files changed, 28 insertions(+), 33 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index df37d1c98a4..3dc9cb2f21d 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -24,6 +24,7 @@ #include #include "Core/Block.h" #include +#include #include #include #include @@ -473,6 +474,22 @@ void Connection::sendQuery( bool with_pending_data, std::function) { + OpenTelemetrySpanHolder span("Connection::sendQuery()"); + span.addAttribute("clickhouse.query_id", query_id_); + span.addAttribute("clickhouse.query", query); + span.addAttribute("target", []() { return this->getHost() + ":" + std::to_string(this->getPort()); } ); + + ClientInfo new_client_info; + auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); + if (client_info && current_trace_context.isTraceEnabled()) + { + // use current span as the parent of remote span + new_client_info = *client_info; + new_client_info.client_trace_context = current_trace_context; + + client_info = &new_client_info; + } + if (!connected) connect(timeouts); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 788b1cc7fcb..50e6ca1183b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -33,7 +33,6 @@ class ThreadStatus; class QueryProfilerReal; class QueryProfilerCPU; class QueryThreadLog; -struct OpenTelemetrySpanHolder; class TasksStatsCounters; struct RUsageCounters; struct PerfEventsCounters; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index baded3c1a21..17b031d2771 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -665,7 +665,7 @@ static std::tuple executeQueryImpl( { std::unique_ptr span; - if (context->query_trace_context.trace_id != UUID()) + if (query_span->isTraceEnabled()) { auto * raw_interpreter_ptr = interpreter.get(); std::string class_name(demangle(typeid(*raw_interpreter_ptr).name())); diff --git a/src/QueryPipeline/RemoteInserter.cpp b/src/QueryPipeline/RemoteInserter.cpp index ce2ba23576d..546ac43165c 100644 --- a/src/QueryPipeline/RemoteInserter.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -30,22 +30,6 @@ RemoteInserter::RemoteInserter( { ClientInfo modified_client_info = client_info_; modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - if (CurrentThread::isInitialized()) - { - auto& thread_trace_context = CurrentThread::get().thread_trace_context; - - if (thread_trace_context.trace_id != UUID()) - { - // overwrite the trace context only if current thread trace context is available - modified_client_info.client_trace_context = thread_trace_context; - } - else - { - // if the trace on the thread local is not enabled(for example running in a background thread) - // we should not clear the trace context on the client info because the client info may hold trace context - // and this trace context should be propagated to the remote server so that the tracing of distributed table insert is complete. - } - } Settings settings = settings_; /// With current protocol it is impossible to avoid deadlock in case of send_logs_level!=none. diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 44e844fbe40..e42b0141a27 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -242,10 +242,6 @@ void RemoteQueryExecutor::sendQuery(ClientInfo::QueryKind query_kind) auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); ClientInfo modified_client_info = context->getClientInfo(); modified_client_info.query_kind = query_kind; - if (CurrentThread::isInitialized()) - { - modified_client_info.client_trace_context = CurrentThread::get().thread_trace_context; - } { std::lock_guard lock(duplicated_part_uuids_mutex); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 15ae23305f3..fb1d3e798d7 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -101,20 +101,19 @@ namespace ReadWriteBufferFromHTTP::HTTPHeaderEntries getHeaders(const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_) { ReadWriteBufferFromHTTP::HTTPHeaderEntries headers(headers_.begin(), headers_.end()); - // Propagate OpenTelemetry trace context, if any, downstream. - if (CurrentThread::isInitialized()) - { - const auto & thread_trace_context = CurrentThread::get().thread_trace_context; - if (thread_trace_context.trace_id != UUID()) - { - headers.emplace_back("traceparent", thread_trace_context.composeTraceparentHeader()); - if (!thread_trace_context.tracestate.empty()) - { - headers.emplace_back("tracestate", thread_trace_context.tracestate); - } + // Propagate OpenTelemetry trace context, if any, downstream. + auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); + if (current_trace_context.isTraceEnabled()) + { + header.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); + + if (!current_trace_context.tracestate.empty()) + { + header.emplace_back("tracestate",current_trace_context.tracestate); } } + return headers; } From 495555f04fa7743e64b553d5a09b30e56b75a102 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:42:53 +0800 Subject: [PATCH 027/582] Remove useless span --- src/Interpreters/InterpreterFactory.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 6b081467ae7..9e067b28b6d 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -112,8 +112,6 @@ namespace ErrorCodes std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) { - OpenTelemetrySpanHolder span("InterpreterFactory::get()"); - ProfileEvents::increment(ProfileEvents::Query); if (query->as()) From d3d89f59ca5e722277088c70291c70c6252c2574 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:43:09 +0800 Subject: [PATCH 028/582] Add tracing support to distributed insert --- src/Storages/Distributed/DirectoryMonitor.cpp | 25 +++++++++++++++++++ src/Storages/Distributed/DistributedSink.cpp | 4 +-- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 5e9da48fc68..fec3d825964 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -609,6 +609,8 @@ bool StorageDistributedDirectoryMonitor::processFiles(const std::mapgetSettingsRef()); @@ -623,6 +625,10 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa formatReadableQuantity(distributed_header.rows), formatReadableSizeWithBinarySuffix(distributed_header.bytes)); + thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + this->storage.getContext()->getOpenTelemetrySpanLog()); + auto connection = pool->get(timeouts, &distributed_header.insert_settings); RemoteInserter remote{*connection, timeouts, distributed_header.insert_query, @@ -634,10 +640,20 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa } catch (Exception & e) { + if (thread_trace_context) + thread_trace_context->root_span.addAttribute(std::current_exception()); + e.addMessage(fmt::format("While sending {}", file_path)); maybeMarkAsBroken(file_path, e); throw; } + catch(...) + { + if (thread_trace_context) + thread_trace_context->root_span.addAttribute(std::current_exception()); + + throw; + } auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); markAsSend(file_path); @@ -849,6 +865,10 @@ private: ReadBufferFromFile in(file_path->second); const auto & distributed_header = readDistributedHeader(in, parent.log); + OpenTelemetryThreadTraceContextScope thread_trace_context(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + parent.storage.getContext()->getOpenTelemetrySpanLog()); + if (!remote) { remote = std::make_unique(connection, timeouts, @@ -883,6 +903,11 @@ private: ReadBufferFromFile in(file_path->second); const auto & distributed_header = readDistributedHeader(in, parent.log); + // this function is called in a separated thread, so we set up the trace context from the file + OpenTelemetryThreadTraceContextScope thread_trace_context(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + parent.storage.getContext()->getOpenTelemetrySpanLog()); + RemoteInserter remote(connection, timeouts, distributed_header.insert_query, distributed_header.insert_settings, diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 798de060768..37555b07507 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -737,11 +737,11 @@ void DistributedSink::writeToShard(const Block & block, const std::vectorgetSettingsRef().write(header_buf); - if (context->getClientInfo().client_trace_context.trace_id != UUID() && CurrentThread::isInitialized()) + if (OpenTelemetryThreadTraceContext::current().isTraceEnabled()) { // if the distributed tracing is enabled, use the trace context in current thread as parent of next span auto client_info = context->getClientInfo(); - client_info.client_trace_context = CurrentThread::get().thread_trace_context; + client_info.client_trace_context = OpenTelemetryThreadTraceContext::current(); client_info.write(header_buf, DBMS_TCP_PROTOCOL_VERSION); } else From 8ce5bae2ddb225da0e9c481246bcdfbaf116d4f3 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:43:52 +0800 Subject: [PATCH 029/582] Add support to GRPCServer --- src/Server/GRPCServer.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index e1c73b7ebbb..18c016c45a3 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -662,6 +662,7 @@ namespace std::optional session; ContextMutablePtr query_context; std::optional query_scope; + OpenTelemetryThreadTraceContextScopePtr thread_trace_context; String query_text; ASTPtr ast; ASTInsertQuery * insert_query = nullptr; @@ -828,6 +829,8 @@ namespace query_context = session->makeQueryContext(std::move(client_info)); + + /// Prepare settings. SettingsChanges settings_changes; for (const auto & [key, value] : query_info.settings()) @@ -840,6 +843,8 @@ namespace query_context->setCurrentQueryId(query_info.query_id()); query_scope.emplace(query_context); + thread_trace_context = query_context->startTracing("GRPCServer"); + /// Prepare for sending exceptions and logs. const Settings & settings = query_context->getSettingsRef(); send_exception_with_stacktrace = settings.calculate_text_stack_trace; @@ -1358,6 +1363,7 @@ namespace io = {}; query_scope.reset(); query_context.reset(); + thread_trace_context.reset(); session.reset(); } From c0a67dc6b9fdd231aa41d00a36c367abc96b07f8 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:44:05 +0800 Subject: [PATCH 030/582] Add API to simplify user code --- src/Common/OpenTelemetryTraceContext.h | 1 + src/Common/OpenTelemtryTraceContext.cpp | 13 +++++++++++++ 2 files changed, 14 insertions(+) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 76119c8f538..fff7d3b1dd7 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -23,6 +23,7 @@ struct OpenTelemetrySpan } void addAttribute(const std::string& name, const std::string& value); + void addAttribute(const std::string& name, std::function value_supplier); void addAttribute(const Exception & e); void addAttribute(std::exception_ptr e); diff --git a/src/Common/OpenTelemtryTraceContext.cpp b/src/Common/OpenTelemtryTraceContext.cpp index 162a741f97f..610f542bcf8 100644 --- a/src/Common/OpenTelemtryTraceContext.cpp +++ b/src/Common/OpenTelemtryTraceContext.cpp @@ -35,6 +35,19 @@ void OpenTelemetrySpan::addAttribute(const std::string& name, const std::string& this->attributes.push_back(Tuple{name, value}); } +void OpenTelemetrySpan::addAttribute(const std::string& name, std::function value_supplier) +{ + if (!this->isTraceEnabled() || !value_supplier) + return; + + std::string value = value_supplier(); + if (value.empty()) + return; + + this->attribute_names.push_back(name); + this->attribute_values.push_back(value); +} + void OpenTelemetrySpan::addAttribute(const Exception & e) { if (trace_id == UUID()) From 93dc109e36a5b7e1f382e436bf570c13521da881 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:44:19 +0800 Subject: [PATCH 031/582] Fix code --- src/Client/Connection.cpp | 2 +- src/Common/OpenTelemetryTraceContext.h | 40 ++++- src/Common/OpenTelemtryTraceContext.cpp | 113 ++++++------- src/Common/ThreadPool.cpp | 1 + src/Interpreters/Context.h | 3 - src/Interpreters/OpenTelemetrySpanLog.cpp | 190 ---------------------- src/Interpreters/OpenTelemetrySpanLog.h | 18 +- src/Interpreters/Session.h | 2 + src/Interpreters/executeQuery.cpp | 8 +- src/Server/GRPCServer.cpp | 6 +- src/Server/HTTPHandler.cpp | 31 ++-- src/Server/TCPHandler.cpp | 6 +- src/Storages/StorageURL.cpp | 4 +- 13 files changed, 132 insertions(+), 292 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 3dc9cb2f21d..05458fae737 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -477,7 +477,7 @@ void Connection::sendQuery( OpenTelemetrySpanHolder span("Connection::sendQuery()"); span.addAttribute("clickhouse.query_id", query_id_); span.addAttribute("clickhouse.query", query); - span.addAttribute("target", []() { return this->getHost() + ":" + std::to_string(this->getPort()); } ); + span.addAttribute("target", [this]() { return this->getHost() + ":" + std::to_string(this->getPort()); } ); ClientInfo new_client_info; auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index fff7d3b1dd7..a2219607ca4 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -23,7 +24,7 @@ struct OpenTelemetrySpan } void addAttribute(const std::string& name, const std::string& value); - void addAttribute(const std::string& name, std::function value_supplier); + void addAttribute(const std::string& name, std::function value_supplier); void addAttribute(const Exception & e); void addAttribute(std::exception_ptr e); @@ -84,23 +85,54 @@ struct OpenTelemetryThreadTraceContextScope OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, const OpenTelemetryTraceContext& _parent_trace_context, - const std::weak_ptr& _log); + const std::weak_ptr& _log) + : OpenTelemetryThreadTraceContextScope(_operation_name, + _parent_trace_context, + nullptr, + _log) + { + } + /// Initialize a tracing context on a child thread based on the context from the parent thread OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, const OpenTelemetryThreadTraceContext& _parent_thread_trace_context) : OpenTelemetryThreadTraceContextScope(_operation_name, _parent_thread_trace_context, + nullptr, _parent_thread_trace_context.span_log) { } + /// For Servers like HTTP/TCP/GRPC to initialize tracing context on thread that process requests from clients + OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + OpenTelemetryTraceContext _parent_trace_context, + const Settings& _settings, + const std::weak_ptr& _log) + : OpenTelemetryThreadTraceContextScope(_operation_name, + _parent_trace_context, + &_settings, + _log) + { + } + + OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + OpenTelemetryTraceContext _parent_trace_context, + const Settings* settings_ptr, + const std::weak_ptr& _log); + ~OpenTelemetryThreadTraceContextScope(); - - OpenTelemetrySpan root_span; }; using OpenTelemetryThreadTraceContextScopePtr = std::unique_ptr; +struct OpenTelemetrySpanHolder : public OpenTelemetrySpan +{ + OpenTelemetrySpanHolder(const std::string & _operation_name); + + void finish(); + ~OpenTelemetrySpanHolder(); +}; + } diff --git a/src/Common/OpenTelemtryTraceContext.cpp b/src/Common/OpenTelemtryTraceContext.cpp index 610f542bcf8..3048e9c0087 100644 --- a/src/Common/OpenTelemtryTraceContext.cpp +++ b/src/Common/OpenTelemtryTraceContext.cpp @@ -1,25 +1,16 @@ #include "Interpreters/OpenTelemetrySpanLog.h" #include -#include -#include -#include -#include -#include -#include -#include - -#include +#include #include -#include - +#include namespace DB { thread_local OpenTelemetryThreadTraceContext current_thread_trace_context; -void OpenTelemetrySpan::addAttribute(const std::string& name, UInt64 value) +void OpenTelemetrySpan::addAttribute(const std::string & name, UInt64 value) { if (trace_id == UUID() || name.empty()) return; @@ -27,7 +18,7 @@ void OpenTelemetrySpan::addAttribute(const std::string& name, UInt64 value) this->attributes.push_back(Tuple{name, toString(value)}); } -void OpenTelemetrySpan::addAttribute(const std::string& name, const std::string& value) +void OpenTelemetrySpan::addAttribute(const std::string & name, const std::string & value) { if (trace_id == UUID() || name.empty() || value.empty()) return; @@ -35,7 +26,7 @@ void OpenTelemetrySpan::addAttribute(const std::string& name, const std::string& this->attributes.push_back(Tuple{name, value}); } -void OpenTelemetrySpan::addAttribute(const std::string& name, std::function value_supplier) +void OpenTelemetrySpan::addAttribute(const std::string & name, std::function value_supplier) { if (!this->isTraceEnabled() || !value_supplier) return; @@ -78,7 +69,8 @@ OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_ this->parent_span_id = current_thread_trace_context.span_id; this->span_id = thread_local_rng(); // create a new id for this span this->operation_name = _operation_name; - this->start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + this->start_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); // set current span id to this current_thread_trace_context.span_id = this->span_id; @@ -103,7 +95,8 @@ void OpenTelemetrySpanHolder::finish() return; } - this->finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + this->finish_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); log->add(OpenTelemetrySpanLogElement(*this)); } @@ -121,25 +114,7 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() } -template -static T readHex(const char * data) -{ - T x{}; - - const char * end = data + sizeof(T) * 2; - while (data < end) - { - x *= 16; - x += unhex(*data); - ++data; - } - - return x; -} - - -bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, - std::string & error) +bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, std::string & error) { trace_id = 0; @@ -148,8 +123,7 @@ bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & trace const int expected_length = strlen("xx-xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx-xxxxxxxxxxxxxxxx-xx"); if (traceparent.length() != expected_length) { - error = fmt::format("unexpected length {}, expected {}", - traceparent.length(), expected_length); + error = fmt::format("unexpected length {}, expected {}", traceparent.length(), expected_length); return false; } @@ -204,7 +178,8 @@ std::string OpenTelemetryTraceContext::composeTraceparentHeader() const { // This span is a parent for its children, so we specify this span_id as a // parent id. - return fmt::format("00-{:016x}{:016x}-{:016x}-{:02x}", + return fmt::format( + "00-{:016x}{:016x}-{:016x}-{:02x}", trace_id.toUnderType().items[0], trace_id.toUnderType().items[1], span_id, @@ -213,39 +188,59 @@ std::string OpenTelemetryTraceContext::composeTraceparentHeader() const static_cast(trace_flags)); } -const OpenTelemetryThreadTraceContext& OpenTelemetryThreadTraceContext::current() +const OpenTelemetryThreadTraceContext & OpenTelemetryThreadTraceContext::current() { return current_thread_trace_context; } void OpenTelemetryThreadTraceContext::reset() { - this->trace_id = 0; + this->trace_id = {}; this->span_id = 0; this->trace_flags = 0; this->tracestate = ""; this->span_log.reset(); } -OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, - const OpenTelemetryTraceContext& _parent_trace_context, - const std::weak_ptr& _span_log) +OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( + const std::string & _operation_name, + OpenTelemetryTraceContext _parent_trace_context, + const Settings * settings_ptr, + const std::weak_ptr & _span_log) { - if (_parent_trace_context.isTraceEnabled()) + this->root_span.trace_id = {}; + this->root_span.span_id = 0; + + if (!_parent_trace_context.isTraceEnabled()) { - this->root_span.trace_id = _parent_trace_context.trace_id; - this->root_span.parent_span_id = _parent_trace_context.span_id; - this->root_span.span_id = thread_local_rng(); - this->root_span.operation_name = _operation_name; - this->root_span.start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - } - else - { - this->root_span.trace_id = 0; - this->root_span.span_id = 0; + if (settings_ptr == nullptr) + /// skip tracing context initialization on current thread + return; + + // start the trace ourselves, with some configurable probability. + std::bernoulli_distribution should_start_trace{settings_ptr->opentelemetry_start_trace_probability}; + if (!should_start_trace(thread_local_rng)) + /// skip tracing context initialization on current thread + return; + + while(_parent_trace_context.trace_id == UUID()) + { + // make sure the random generated trace_id is not 0 which is an invalid id + _parent_trace_context.trace_id.toUnderType().items[0] = thread_local_rng(); //-V656 + _parent_trace_context.trace_id.toUnderType().items[1] = thread_local_rng(); //-V656 + } + _parent_trace_context.span_id = 0; + _parent_trace_context.trace_flags = 1; } - // set trace context on the thread local + this->root_span.trace_id = _parent_trace_context.trace_id; + this->root_span.parent_span_id = _parent_trace_context.span_id; + this->root_span.span_id = thread_local_rng(); + this->root_span.operation_name = _operation_name; + this->root_span.start_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + /// set up trace context on current thread current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; current_thread_trace_context.span_log = _span_log; @@ -253,18 +248,19 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope(const OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() { - if (this->root_span.trace_id != UUID()) + if (this->root_span.isTraceEnabled()) { auto shared_span_log = current_thread_trace_context.span_log.lock(); if (shared_span_log) { this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); - this->root_span.finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + this->root_span.finish_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); } - this->root_span.trace_id = 0; + this->root_span.trace_id = {}; } // restore thread local variables @@ -273,4 +269,3 @@ OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() } - diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 5981b80f346..9c7b41b2dba 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 37b831ff18f..f4a1e79a993 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -360,9 +360,6 @@ private: inline static ContextPtr global_context_instance; -public: - OpenTelemetryThreadTraceContextScopePtr startTracing(const std::string& name); - private: using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index 2683a5f7955..57d5c11ad97 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -68,195 +68,5 @@ void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(attributes); } - -OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_name) -{ - trace_id = 0; - - if (!CurrentThread::isInitialized()) - { - // There may be no thread context if we're running inside the - // clickhouse-client, e.g. reading an external table provided with the - // `--external` option. - return; - } - - auto & thread = CurrentThread::get(); - - trace_id = thread.thread_trace_context.trace_id; - if (trace_id == UUID()) - return; - - parent_span_id = thread.thread_trace_context.span_id; - span_id = thread_local_rng(); - operation_name = _operation_name; - start_time_us = std::chrono::duration_cast( - std::chrono::system_clock::now().time_since_epoch()).count(); - - thread.thread_trace_context.span_id = span_id; } - -OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() -{ - try - { - if (trace_id == UUID()) - return; - - // First of all, return old value of current span. - auto & thread = CurrentThread::get(); - assert(thread.thread_trace_context.span_id == span_id); - thread.thread_trace_context.span_id = parent_span_id; - - // Not sure what's the best way to access the log from here. - auto * thread_group = CurrentThread::getGroup().get(); - // Not sure whether and when this can be null. - if (!thread_group) - return; - - ContextPtr context; - { - std::lock_guard lock(thread_group->mutex); - context = thread_group->query_context.lock(); - } - - if (!context) - { - // Both global and query contexts can be null when executing a - // background task, and global context can be null for some - // queries. - return; - } - - auto log = context->getOpenTelemetrySpanLog(); - if (!log) - { - // The log might be disabled. - return; - } - - finish_time_us = std::chrono::duration_cast( - std::chrono::system_clock::now().time_since_epoch()).count(); - - log->add(OpenTelemetrySpanLogElement( - static_cast(*this))); - } - catch (...) - { - tryLogCurrentException(__FUNCTION__); - } -} - -void OpenTelemetrySpanHolder::addAttribute(const std::string& name, UInt64 value) -{ - if (trace_id == UUID()) - return; - - this->attributes.push_back(Tuple{name, toString(value)}); -} - -void OpenTelemetrySpanHolder::addAttribute(const std::string& name, const std::string& value) -{ - if (trace_id == UUID()) - return; - - this->attributes.push_back(Tuple{name, value}); -} - -void OpenTelemetrySpanHolder::addAttribute(const Exception & e) -{ - if (trace_id == UUID()) - return; - - this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); -} - -void OpenTelemetrySpanHolder::addAttribute(std::exception_ptr e) -{ - if (trace_id == UUID() || e == nullptr) - return; - - this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); -} - -bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, - std::string & error) -{ - trace_id = 0; - - // Version 00, which is the only one we can parse, is fixed width. Use this - // fact for an additional sanity check. - const int expected_length = strlen("xx-xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx-xxxxxxxxxxxxxxxx-xx"); - if (traceparent.length() != expected_length) - { - error = fmt::format("unexpected length {}, expected {}", - traceparent.length(), expected_length); - return false; - } - - const char * data = traceparent.data(); - - uint8_t version = unhex2(data); - data += 2; - - if (version != 0) - { - error = fmt::format("unexpected version {}, expected 00", version); - return false; - } - - if (*data != '-') - { - error = fmt::format("Malformed traceparant header: {}", traceparent); - return false; - } - - ++data; - UInt64 trace_id_higher_64 = unhexUInt(data); - UInt64 trace_id_lower_64 = unhexUInt(data + 16); - data += 32; - - if (*data != '-') - { - error = fmt::format("Malformed traceparant header: {}", traceparent); - return false; - } - - ++data; - UInt64 span_id_64 = unhexUInt(data); - data += 16; - - if (*data != '-') - { - error = fmt::format("Malformed traceparant header: {}", traceparent); - return false; - } - - ++data; - this->trace_flags = unhex2(data); - - // store the 128-bit trace id in big-endian order - this->trace_id.toUnderType().items[0] = trace_id_higher_64; - this->trace_id.toUnderType().items[1] = trace_id_lower_64; - this->span_id = span_id_64; - return true; -} - - -std::string OpenTelemetryTraceContext::composeTraceparentHeader() const -{ - // This span is a parent for its children, so we specify this span_id as a - // parent id. - return fmt::format("00-{:016x}{:016x}-{:016x}-{:02x}", - // Output the trace id in network byte order - trace_id.toUnderType().items[0], - trace_id.toUnderType().items[1], - span_id, - // This cast is needed because fmt is being weird and complaining that - // "mixing character types is not allowed". - static_cast(trace_flags)); -} - - -} diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 2f2cc7a1bea..671fd01128b 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -1,7 +1,8 @@ #pragma once #include -#include +#include +#include namespace DB { @@ -27,19 +28,4 @@ public: using SystemLog::SystemLog; }; -typedef std::shared_ptr OpenTelemetrySpanLogPtr; - -struct OpenTelemetrySpanHolder : public OpenTelemetrySpan -{ - OpenTelemetrySpanHolder(const OpenTelemetryTraceContext& _trace_context, OpenTelemetrySpanLogPtr _span_log, const std::string & _operation_name); - explicit OpenTelemetrySpanHolder(const std::string & _operation_name); - void addAttribute(const std::string& name, UInt64 value); - void addAttribute(const std::string& name, const std::string& value); - void addAttribute(const Exception & e); - void addAttribute(std::exception_ptr e); - - void finish(); - ~OpenTelemetrySpanHolder(); -}; - } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 8de76349b7d..e7b7c4c2b8d 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -65,6 +65,8 @@ public: ContextMutablePtr sessionContext() { return session_context; } ContextPtr sessionContext() const { return session_context; } + ContextPtr sessionOrGlobalContext() { return session_context ? session_context : global_context; } + /// Makes a query context, can be used multiple times, with or without makeSession() called earlier. /// The query context will be created from a copy of a session context if it exists, or from a copy of /// a global context otherwise. In the latter case the function also assigns an user to this context. diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 17b031d2771..cb7f1d347c9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -290,10 +290,10 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr if (auto query_log = context->getQueryLog()) query_log->add(elem); - query_span->attribute_names.addAttribute("clickhouse.exception_code", elem.exception_code); - query_span->attribute_names.addAttribute("clickhouse.exception", elem.exception); - query_span->attribute_names.addAttribute("db.statement", elem.query); - query_span->attribute_names.addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttribute("clickhouse.exception_code", elem.exception_code); + query_span->addAttribute("clickhouse.exception", elem.exception); + query_span->addAttribute("db.statement", elem.query); + query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); query_span->finish(); ProfileEvents::increment(ProfileEvents::FailedQuery); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 18c016c45a3..c03c5c2d9e5 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -843,7 +843,11 @@ namespace query_context->setCurrentQueryId(query_info.query_id()); query_scope.emplace(query_context); - thread_trace_context = query_context->startTracing("GRPCServer"); + /// Set up tracing context for this query on current thread + thread_trace_context = std::make_unique("GRPCServer", + query_context->getClientInfo().client_trace_context, + query_context->getSettingsRef(), + query_context->getOpenTelemetrySpanLog()); /// Prepare for sending exceptions and logs. const Settings & settings = query_context->getSettingsRef(); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 036ae7951a9..3cfe0d02f53 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -932,6 +932,12 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse bool with_stacktrace = false; OpenTelemetryThreadTraceContextScopePtr thread_trace_context; + SCOPE_EXIT({ + // make sure the response status is recorded + if (thread_trace_context) + thread_trace_context->root_span.addAttribute("clickhouse.http_status", response.getStatus()); + }); + try { if (request.getMethod() == HTTPServerRequest::HTTP_OPTIONS) @@ -953,15 +959,17 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse client_info.client_trace_context.tracestate = request.get("tracestate", ""); } - thread_trace_context = request_context->startTracing("HTTPHandler::handleRequest()"); - if (thread_trace_context) + // Setup tracing context for this thread + auto context = session->sessionOrGlobalContext(); + thread_trace_context = std::make_unique("HTTPHandler::handleRequest()", + client_info.client_trace_context, + context->getSettingsRef(), + context->getOpenTelemetrySpanLog()); + if (!client_info.client_trace_context.tracestate.empty()) { - if (!client_info.client_trace_context.tracestate.empty()) - { - thread_trace_context->root_span.addAttribute("clickhouse.tracestate", client_info.client_trace_context.tracestate); - } - thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); + thread_trace_context->root_span.addAttribute("clickhouse.tracestate", client_info.client_trace_context.tracestate); } + thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); response.setContentType("text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); @@ -998,6 +1006,9 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// cannot write in it anymore. So, just log this exception. if (used_output.isFinalized()) { + if (thread_trace_context) + thread_trace_context->root_span.addAttribute("clickhouse.exception", "Cannot flush data to client"); + tryLogCurrentException(log, "Cannot flush data to client"); return; } @@ -1011,11 +1022,9 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse int exception_code = getCurrentExceptionCode(); trySendExceptionToClient(exception_message, exception_code, request, response, used_output); - } - if (thread_trace_context) - { - thread_trace_context->root_span.addAttribute("clickhouse.http_status", response.getStatus()); + if (thread_trace_context) + thread_trace_context->root_span.addAttribute("clickhouse.exception_code", exception_code); } used_output.finalize(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 8bb39d881d1..78fac253c64 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -223,7 +223,11 @@ void TCPHandler::runImpl() if (state.empty() && state.part_uuids_to_ignore && !receivePacket()) continue; - thread_trace_context = query_context->startTracing("TCPHandler"); + /// Set up tracing context for this query on current thread + thread_trace_context = std::make_unique("TCPHandler", + query_context->getClientInfo().client_trace_context, + query_context->getSettingsRef(), + query_context->getOpenTelemetrySpanLog()); query_scope.emplace(query_context); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fb1d3e798d7..d473cdc5d35 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -106,11 +106,11 @@ namespace auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); if (current_trace_context.isTraceEnabled()) { - header.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); + headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); if (!current_trace_context.tracestate.empty()) { - header.emplace_back("tracestate",current_trace_context.tracestate); + headers.emplace_back("tracestate", current_trace_context.tracestate); } } From 90efbc323d9cd20f367f9ad09cfa5a0536c89cb2 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:44:32 +0800 Subject: [PATCH 032/582] Fix conflicts --- src/Interpreters/Context.h | 1 + src/Server/TCPHandler.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f4a1e79a993..d3708245fe7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -550,6 +550,7 @@ public: const String & projection_name = {}, const String & view_name = {}); + OpenTelemetryThreadTraceContextScopePtr startTracing(const std::string& name); /// Supported factories for records in query_log enum class QueryLogFactories diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 78fac253c64..c5f95cd89b2 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -467,7 +467,7 @@ void TCPHandler::runImpl() if (exception) { if (thread_trace_context) - thread_trace_context->root_span.addAttribute(exception.value()); + thread_trace_context->root_span.addAttribute(*exception); try { From 4ef85164aa57d8ba54acfe8c25fb208ec502f56c Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:44:50 +0800 Subject: [PATCH 033/582] Fix file name --- ...OpenTelemtryTraceContext.cpp => OpenTelemetryTraceContext.cpp} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/Common/{OpenTelemtryTraceContext.cpp => OpenTelemetryTraceContext.cpp} (100%) diff --git a/src/Common/OpenTelemtryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp similarity index 100% rename from src/Common/OpenTelemtryTraceContext.cpp rename to src/Common/OpenTelemetryTraceContext.cpp From 47da9722777031f4b1a3654c81d858b730359d84 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 17:45:08 +0800 Subject: [PATCH 034/582] Allows re-entering on thread context to make code more robust. --- src/Common/OpenTelemetryTraceContext.cpp | 41 +++++++++++++++++++++--- src/Common/OpenTelemetryTraceContext.h | 15 ++++++--- 2 files changed, 47 insertions(+), 9 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 3048e9c0087..93ff842e6e1 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -195,7 +195,7 @@ const OpenTelemetryThreadTraceContext & OpenTelemetryThreadTraceContext::current void OpenTelemetryThreadTraceContext::reset() { - this->trace_id = {}; + this->trace_id = UUID(); this->span_id = 0; this->trace_flags = 0; this->tracestate = ""; @@ -208,7 +208,31 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( const Settings * settings_ptr, const std::weak_ptr & _span_log) { - this->root_span.trace_id = {}; + if (current_thread_trace_context.isTraceEnabled()) + { + /// + /// This is not the normal case, + /// it means that construction of current object is not at the start of current thread. + /// Usually this is due to: + /// 1. bad design + /// 2. right design but code changes so that original point where this object is constructing is not the new start execution of current thread + /// + /// In such case, we should use current context as parent of this new constructing object, + /// So this branch ensures this class can be instantiated multiple times on one same thread safely. + /// + this->is_context_owner = false; + this->root_span.trace_id = current_thread_trace_context.trace_id; + this->root_span.parent_span_id = current_thread_trace_context.span_id; + this->root_span.span_id = thread_local_rng(); + this->root_span.operation_name = _operation_name; + this->root_span.start_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + current_thread_trace_context.span_id = this->root_span.span_id; + return; + } + + this->root_span.trace_id = UUID(); this->root_span.span_id = 0; if (!_parent_trace_context.isTraceEnabled()) @@ -260,11 +284,18 @@ OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); } - this->root_span.trace_id = {}; + this->root_span.trace_id = UUD(); } - // restore thread local variables - current_thread_trace_context.reset(); + if (this->is_context_owner) + { + // clear the context on current thread + current_thread_trace_context.reset(); + } + else + { + current_thread_trace_context.span_id = this->root_span.parent_span_id; + } } diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index a2219607ca4..87a67c5a6c9 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -34,8 +34,6 @@ struct OpenTelemetrySpan } }; -struct OpenTelemetrySpanHolder; - class Context; using ContextPtr = std::shared_ptr; @@ -61,7 +59,7 @@ struct OpenTelemetryTraceContext } }; -// tracing context kept on thread local +/// tracing context kept on thread local struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext { OpenTelemetryThreadTraceContext& operator =(const OpenTelemetryTraceContext& context) @@ -77,6 +75,11 @@ struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext std::weak_ptr span_log; }; +/// A scoped tracing context, is used to hold the tracing context at the beginning of each thread execution and clear the context automatically when the scope exists. +/// It should be the root of all span logs for one tracing. +/// +/// It's SAFE to construct this object multiple times on one same thread, +/// but it's not encourage to do so because this is only a protection in case of code changes. struct OpenTelemetryThreadTraceContextScope { // forbidden copy ctor and assignment to make the destructor safe @@ -123,16 +126,20 @@ struct OpenTelemetryThreadTraceContextScope ~OpenTelemetryThreadTraceContextScope(); OpenTelemetrySpan root_span; + +private: + bool is_context_owner = true; }; using OpenTelemetryThreadTraceContextScopePtr = std::unique_ptr; +/// A span holder is usually used in a function scope struct OpenTelemetrySpanHolder : public OpenTelemetrySpan { OpenTelemetrySpanHolder(const std::string & _operation_name); + ~OpenTelemetrySpanHolder(); void finish(); - ~OpenTelemetrySpanHolder(); }; } From 6ae2504e577ca190e851b696bfdee8adcc4ffdb1 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 7 Jul 2022 18:17:30 +0800 Subject: [PATCH 035/582] Fix code --- src/Common/OpenTelemetryTraceContext.cpp | 9 ++++----- src/Interpreters/executeQuery.cpp | 4 +++- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 93ff842e6e1..cde3866cead 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -15,7 +15,7 @@ void OpenTelemetrySpan::addAttribute(const std::string & name, UInt64 value) if (trace_id == UUID() || name.empty()) return; - this->attributes.push_back(Tuple{name, toString(value)}); + this->attributes.push_back(Tuple{name, std::to_string(value)}); } void OpenTelemetrySpan::addAttribute(const std::string & name, const std::string & value) @@ -35,8 +35,7 @@ void OpenTelemetrySpan::addAttribute(const std::string & name, std::functionattribute_names.push_back(name); - this->attribute_values.push_back(value); + this->attributes.push_back(Tuple{name, value}); } void OpenTelemetrySpan::addAttribute(const Exception & e) @@ -254,7 +253,6 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( _parent_trace_context.trace_id.toUnderType().items[1] = thread_local_rng(); //-V656 } _parent_trace_context.span_id = 0; - _parent_trace_context.trace_flags = 1; } this->root_span.trace_id = _parent_trace_context.trace_id; @@ -267,6 +265,7 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( /// set up trace context on current thread current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; + current_thread_trace_context.trace_flags = 1; current_thread_trace_context.span_log = _span_log; } @@ -284,7 +283,7 @@ OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); } - this->root_span.trace_id = UUD(); + this->root_span.trace_id = UUID(); } if (this->is_context_owner) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cb7f1d347c9..8283a3b0ae0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -344,6 +344,8 @@ static std::tuple executeQueryImpl( QueryProcessingStage::Enum stage, ReadBuffer * istr) { + std::shared_ptr query_span = std::make_shared("query"); + const auto current_time = std::chrono::system_clock::now(); auto & client_info = context->getClientInfo(); @@ -445,7 +447,7 @@ static std::tuple executeQueryImpl( if (!internal) { - onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast); + onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast, query_span); } throw; From 97115593914553ac96e077be32b0cc3ee7a6466e Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 8 Jul 2022 18:56:07 +0800 Subject: [PATCH 036/582] Remove extra code --- src/Interpreters/Context.cpp | 48 ------------------------------------ src/Interpreters/Context.h | 2 -- 2 files changed, 50 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 13d78a690b5..9b52a2d46eb 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3425,52 +3425,4 @@ WriteSettings Context::getWriteSettings() const return res; } -OpenTelemetryThreadTraceContextScopePtr Context::startTracing(const std::string& name) -{ - OpenTelemetryThreadTraceContextScopePtr trace_context; - if (this->client_info.client_trace_context.trace_id != UUID()) - { - // Use the OpenTelemetry trace context we received from the client, and - // initialize the tracing context for this query on current thread - return std::make_unique(name, - this->client_info.client_trace_context, - this->getOpenTelemetrySpanLog()); - } - - // start the trace ourselves, with some configurable probability. - std::bernoulli_distribution should_start_trace{settings.opentelemetry_start_trace_probability}; - if (!should_start_trace(thread_local_rng)) - { - return trace_context; - } - - /// 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 - - OpenTelemetryTraceContext query_trace_context; - query_trace_context.trace_id = random.uuid; - query_trace_context.span_id = 0; - // Mark this trace as sampled in the flags. - query_trace_context.trace_flags = 1; - - return std::make_unique(name, - query_trace_context, - this->getOpenTelemetrySpanLog()); - -} - } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d3708245fe7..1a87a0a82a3 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -550,8 +550,6 @@ public: const String & projection_name = {}, const String & view_name = {}); - OpenTelemetryThreadTraceContextScopePtr startTracing(const std::string& name); - /// Supported factories for records in query_log enum class QueryLogFactories { From da57a993e40b3bd94f2760fa5a1ed4800aef836d Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Sat, 9 Jul 2022 13:37:13 +0800 Subject: [PATCH 037/582] Fix CI --- src/Client/Connection.cpp | 2 +- src/Common/OpenTelemetryTraceContext.cpp | 2 +- src/Common/ThreadPool.cpp | 1 - src/Interpreters/Context.cpp | 1 + src/Server/GRPCServer.cpp | 2 -- src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- 6 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 05458fae737..1365b661e79 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -477,7 +477,7 @@ void Connection::sendQuery( OpenTelemetrySpanHolder span("Connection::sendQuery()"); span.addAttribute("clickhouse.query_id", query_id_); span.addAttribute("clickhouse.query", query); - span.addAttribute("target", [this]() { return this->getHost() + ":" + std::to_string(this->getPort()); } ); + span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); } ); ClientInfo new_client_info; auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index cde3866cead..84141ce267a 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -246,7 +246,7 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( /// skip tracing context initialization on current thread return; - while(_parent_trace_context.trace_id == UUID()) + while (_parent_trace_context.trace_id == UUID()) { // make sure the random generated trace_id is not 0 which is an invalid id _parent_trace_context.trace_id.toUnderType().items[0] = thread_local_rng(); //-V656 diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 9c7b41b2dba..5981b80f346 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9b52a2d46eb..4529766fe9e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -134,6 +134,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; + extern const int NOT_IMPLEMENTED; } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index c03c5c2d9e5..3deefb32389 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -829,8 +829,6 @@ namespace query_context = session->makeQueryContext(std::move(client_info)); - - /// Prepare settings. SettingsChanges settings_changes; for (const auto & [key, value] : query_info.settings()) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index fec3d825964..3c651f7faa9 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -647,7 +647,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa maybeMarkAsBroken(file_path, e); throw; } - catch(...) + catch (...) { if (thread_trace_context) thread_trace_context->root_span.addAttribute(std::current_exception()); From c2ae762a5ad38abe28219c9009e58064d1a04042 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Sat, 9 Jul 2022 13:58:00 +0800 Subject: [PATCH 038/582] Fix tests --- src/Client/Connection.cpp | 2 +- src/Interpreters/executeQuery.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 1365b661e79..cb1fbe08d67 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -477,7 +477,7 @@ void Connection::sendQuery( OpenTelemetrySpanHolder span("Connection::sendQuery()"); span.addAttribute("clickhouse.query_id", query_id_); span.addAttribute("clickhouse.query", query); - span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); } ); + span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); }); ClientInfo new_client_info; auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8283a3b0ae0..ad16b8082eb 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -917,6 +917,8 @@ static std::tuple executeQueryImpl( query_span->addAttribute("db.statement", elem.query); query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttribute("clickhouse.query_status", "QueryFinish"); + query_span->addAttribute("clickhouse.tracestate", context->query_trace_context.tracestate}); query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); From c559370ec916f6df7e4c683fb5eff647f4332eb6 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Sat, 9 Jul 2022 20:20:04 +0800 Subject: [PATCH 039/582] Fix test cases --- src/Interpreters/executeQuery.cpp | 2 +- src/Server/HTTPHandler.cpp | 6 +----- .../01455_opentelemetry_distributed.reference | 2 +- .../01455_opentelemetry_distributed.sh | 18 ++++++------------ 4 files changed, 9 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ad16b8082eb..ed2c6ca7f9a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -917,8 +917,8 @@ static std::tuple executeQueryImpl( query_span->addAttribute("db.statement", elem.query); query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttribute("clickhouse.tracestate", OpenTelemetryThreadTraceContext::current().tracestate); query_span->addAttribute("clickhouse.query_status", "QueryFinish"); - query_span->addAttribute("clickhouse.tracestate", context->query_trace_context.tracestate}); query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 3cfe0d02f53..9a11130fc20 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -961,14 +961,10 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); - thread_trace_context = std::make_unique("HTTPHandler::handleRequest()", + thread_trace_context = std::make_unique("HTTPHandler", client_info.client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog()); - if (!client_info.client_trace_context.tracestate.empty()) - { - thread_trace_context->root_span.addAttribute("clickhouse.tracestate", client_info.client_trace_context.tracestate); - } thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); response.setContentType("text/plain; charset=UTF-8"); diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference index e70506599ec..d48b3738bc2 100644 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference @@ -15,7 +15,7 @@ {"query":"select 1 format Null\n","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1} {"query":"select 1 format Null\n","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1} {"query":"select * from url('http:\/\/127.0.0.2:8123\/?query=select%201%20format%20Null', CSV, 'a int')","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1} -{"total spans":"3","unique spans":"3","unique non-zero parent spans":"2"} +{"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"} {"initial query spans with proper parent":"1"} {"unique non-empty tracestate values":"1"} ===sampled=== diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh index 95d99449837..b2b5ae89105 100755 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh @@ -48,21 +48,16 @@ select count(*) "'"'"total spans"'"'", ; -- Also check that the initial query span in ClickHouse has proper parent span. +-- the first span should be child of input trace context +-- the 2nd span should be the 'query' span select count(*) "'"'"initial query spans with proper parent"'"'" - from - (select *, attribute_name, attribute_value - from system.opentelemetry_span_log - array join mapKeys(attribute) as attribute_name, - mapValues(attribute) as attribute_value) o - join system.query_log on query_id = o.attribute_value + from system.opentelemetry_span_log where trace_id = UUIDNumToString(toFixedString(unhex('$trace_id'), 16)) - and current_database = currentDatabase() and operation_name = 'query' - and parent_span_id = reinterpretAsUInt64(unhex('73')) - and o.attribute_name = 'clickhouse.query_id' - and is_initial_query - and type = 'QueryFinish' + and parent_span_id in ( + select span_id from system.opentelemetry_span_log where trace_id = UUIDNumToString(toFixedString(unhex('$trace_id'), 16)) and parent_span_id = reinterpretAsUInt64(unhex('73')) + ) ; -- Check that the tracestate header was propagated. It must have exactly the @@ -136,7 +131,6 @@ ${CLICKHOUSE_CLIENT} -q " select if(2 <= count() and count() <= 18, 'OK', 'Fail') from system.opentelemetry_span_log where operation_name = 'query' - and parent_span_id = 0 -- only account for the initial queries and attribute['clickhouse.query_id'] like '$query_id-%' ; " From 85bf02225848b7e48321a835b3cda5cdaa53ef5c Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Thu, 21 Jul 2022 16:57:28 +0300 Subject: [PATCH 040/582] review zookeeper and partition matching --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Storages/Freeze.cpp | 26 ++++++++++++++------- src/Storages/Freeze.h | 9 ++++--- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++-- src/Storages/StorageReplicatedMergeTree.h | 2 +- 6 files changed, 29 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 0fc6753b6df..b50ff4d32c3 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -501,7 +501,7 @@ BlockIO InterpreterSystemQuery::execute() { getContext()->checkAccess(AccessType::SYSTEM_UNFREEZE); /// The result contains information about deleted parts as a table. It is for compatibility with ALTER TABLE UNFREEZE query. - result = Unfreezer().unfreeze(query.backup_name, getContext()); + result = Unfreezer(getContext()).unfreeze(query.backup_name); break; } default: diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 9193ed23c5b..6a5d908226c 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_PARTITION_VALUE; +} + void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage) { replica_name = storage.getReplicaName(); @@ -96,10 +101,10 @@ String FreezeMetaData::getFileName(const String & path) return fs::path(path) / "frozen_metadata.txt"; } -BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context) +BlockIO Unfreezer::unfreeze(const String & backup_name) { LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name)); - auto disks_map = local_context->getDisksMap(); + auto disks_map = local_context_->getDisksMap(); Disks disks; for (auto & [name, disk]: disks_map) { @@ -123,7 +128,7 @@ BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context { auto table_directory = prefix_directory / table_it->name(); auto current_result_info = unfreezePartitionsFromTableDirectory( - [](const String &) { return true; }, backup_name, {disk}, table_directory, local_context); + [](const String &) { return true; }, backup_name, {disk}, table_directory, std::nullopt); for (auto & command_result : current_result_info) { command_result.command_type = "SYSTEM UNFREEZE"; @@ -150,7 +155,7 @@ BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context return result; } -bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context) +bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper) { if (disk->supportZeroCopyReplication()) { @@ -158,7 +163,7 @@ bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const Strin if (meta.load(disk, path)) { FreezeMetaData::clean(disk, path); - return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context); + return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context, zookeeper); } } @@ -167,7 +172,7 @@ bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const Strin return false; } -PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, ContextPtr local_context) +PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, std::optional format_version) { PartitionCommandsResultInfo result; @@ -179,8 +184,13 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg for (auto it = disk->iterateDirectory(table_directory); it->isValid(); it->next()) { const auto & partition_directory = it->name(); + + int count_underscores = std::count_if(partition_directory.begin(), partition_directory.end(), []( char c ){return c =='_';}); + if ((format_version.has_value() && format_version == 0) || count_underscores == 4) { + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Can not complete unfreeze query because part directory name is obsolete: " + partition_directory); + } - /// Partition ID is prefix of part directory name: _ + /// For format_version == 1, partition ID is prefix of part directory name: _ auto found = partition_directory.find('_'); if (found == std::string::npos) continue; @@ -191,7 +201,7 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg const auto & path = it->path(); - bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context); + bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context_, zookeeper_); result.push_back(PartitionCommandResultInfo{ .partition_id = partition_id, diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index 2026ac393b9..5fb719293cf 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -32,12 +32,15 @@ public: class Unfreezer { public: - PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, ContextPtr local_context); - BlockIO unfreeze(const String & backup_name, ContextPtr local_context); + Unfreezer(ContextPtr local_context) : local_context_(local_context) { zookeeper_ = local_context->getZooKeeper(); } + PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, std::optional format_version); + BlockIO unfreeze(const String & backup_name); private: + ContextPtr local_context_; + zkutil::ZooKeeperPtr zookeeper_; Poco::Logger * log = &Poco::Logger::get("Unfreezer"); static constexpr std::string_view backup_directory_prefix = "shadow"; - static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context); + static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper); }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9ea6cc6cea2..dfdd67d56de 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6187,7 +6187,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn auto disks = getStoragePolicy()->getDisks(); - return Unfreezer().unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path, local_context); + return Unfreezer(local_context).unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path, format_version); } bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index df9574930aa..6adc40d1823 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8190,7 +8190,7 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & if (disk->supportZeroCopyReplication()) { String table_id = getTableSharedID(); - return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext()); + return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext(), current_zookeeper); } disk->removeRecursive(path); @@ -8200,11 +8200,10 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, - const String &, const String & detached_replica_name, const String & detached_zookeeper_path, ContextPtr local_context) + const String &, const String & detached_replica_name, const String & detached_zookeeper_path, ContextPtr local_context, const zkutil::ZooKeeperPtr & zookeeper) { bool keep_shared = false; - zkutil::ZooKeeperPtr zookeeper = local_context->getZooKeeper(); NameSet files_not_to_remove; fs::path checksums = fs::path(path) / IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index fca76e42ca2..8c19874cfba 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -318,7 +318,7 @@ public: void checkBrokenDisks(); static bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, - const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context); + const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context, const zkutil::ZooKeeperPtr & zookeeper); private: std::atomic_bool are_restoring_replica {false}; From 8f14c60932496c1bb5b89688b92536593c2cf127 Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Fri, 22 Jul 2022 13:10:08 +0300 Subject: [PATCH 041/582] remove exception for old partitions, add server configuration option to enable system unfreeze --- src/Storages/Freeze.cpp | 15 +++++---------- src/Storages/Freeze.h | 19 +++++++++++++++---- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 6a5d908226c..24f99e0ed2a 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -104,7 +104,7 @@ String FreezeMetaData::getFileName(const String & path) BlockIO Unfreezer::unfreeze(const String & backup_name) { LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name)); - auto disks_map = local_context_->getDisksMap(); + auto disks_map = local_context->getDisksMap(); Disks disks; for (auto & [name, disk]: disks_map) { @@ -128,7 +128,7 @@ BlockIO Unfreezer::unfreeze(const String & backup_name) { auto table_directory = prefix_directory / table_it->name(); auto current_result_info = unfreezePartitionsFromTableDirectory( - [](const String &) { return true; }, backup_name, {disk}, table_directory, std::nullopt); + [](const String &) { return true; }, backup_name, {disk}, table_directory); for (auto & command_result : current_result_info) { command_result.command_type = "SYSTEM UNFREEZE"; @@ -172,7 +172,7 @@ bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const Strin return false; } -PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, std::optional format_version) +PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory) { PartitionCommandsResultInfo result; @@ -184,13 +184,8 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg for (auto it = disk->iterateDirectory(table_directory); it->isValid(); it->next()) { const auto & partition_directory = it->name(); - - int count_underscores = std::count_if(partition_directory.begin(), partition_directory.end(), []( char c ){return c =='_';}); - if ((format_version.has_value() && format_version == 0) || count_underscores == 4) { - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Can not complete unfreeze query because part directory name is obsolete: " + partition_directory); - } - /// For format_version == 1, partition ID is prefix of part directory name: _ + /// Partition ID is prefix of part directory name: _ auto found = partition_directory.find('_'); if (found == std::string::npos) continue; @@ -201,7 +196,7 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg const auto & path = it->path(); - bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context_, zookeeper_); + bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context, zookeeper); result.push_back(PartitionCommandResultInfo{ .partition_id = partition_id, diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index 5fb719293cf..eed1900fbd7 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -6,6 +6,10 @@ namespace DB { +namespace ErrorCodes { + extern const int SUPPORT_IS_DISABLED; +} + /// Special metadata used during freeze table. Required for zero-copy /// replication. struct FreezeMetaData @@ -32,12 +36,19 @@ public: class Unfreezer { public: - Unfreezer(ContextPtr local_context) : local_context_(local_context) { zookeeper_ = local_context->getZooKeeper(); } - PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, std::optional format_version); + Unfreezer(ContextPtr context) : local_context(context), zookeeper() { + const auto & config = local_context->getConfigRef(); + static constexpr auto config_key = "enable_system_unfreeze"; + if (!config.getBool(config_key, false)) { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); + } + zookeeper = this->local_context->getZooKeeper(); + } + PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory); BlockIO unfreeze(const String & backup_name); private: - ContextPtr local_context_; - zkutil::ZooKeeperPtr zookeeper_; + ContextPtr local_context; + zkutil::ZooKeeperPtr zookeeper; Poco::Logger * log = &Poco::Logger::get("Unfreezer"); static constexpr std::string_view backup_directory_prefix = "shadow"; static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index dfdd67d56de..d4692b14326 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6187,7 +6187,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn auto disks = getStoragePolicy()->getDisks(); - return Unfreezer(local_context).unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path, format_version); + return Unfreezer(local_context).unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path); } bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const From 9fc3addea9d41e455a4c9010ed2b4c3736ae88f4 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Sun, 31 Jul 2022 23:36:20 +0300 Subject: [PATCH 042/582] Fixed using `column_after_join` for handling `WINDOW` expressions in `ExpressionAnalyzer`, shall be `aggregated_columns`. --- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- .../reuseStorageOrderingForWindowFunctions.cpp | 2 +- ...optimizations_optimize_read_in_window_order.reference | 9 +++++++++ ...5_plan_optimizations_optimize_read_in_window_order.sh | 9 +++++++++ 4 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 23258c60099..b6d26e3c27c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -793,7 +793,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, with_alias->getColumnName(), 1 /* direction */, 1 /* nulls_direction */)); - auto actions_dag = std::make_shared(columns_after_join); + auto actions_dag = std::make_shared(aggregated_columns); getRootActions(column_ast, false, actions_dag); desc.partition_by_actions.push_back(std::move(actions_dag)); } @@ -814,7 +814,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, order_by_element.direction, order_by_element.nulls_direction)); - auto actions_dag = std::make_shared(columns_after_join); + auto actions_dag = std::make_shared(aggregated_columns); getRootActions(column_ast, false, actions_dag); desc.order_by_actions.push_back(std::move(actions_dag)); } diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 4d3c268ab7e..a5b1068097c 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -29,7 +29,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, { /// Find the following sequence of steps, add InputOrderInfo and apply prefix sort description to /// SortingStep: - /// WindowStep <- SortingStep <- [Expression] <- [SettingQuotaAndLimits] <- ReadFromMergeTree + /// WindowStep <- SortingStep <- [Expression] <- ReadFromMergeTree auto * window_node = parent_node; auto * window = typeid_cast(window_node->step.get()); diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference index 7fcd29b5faf..00eb03bd5f0 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference @@ -10,3 +10,12 @@ No sorting plan optimize_read_in_window_order=1 Prefix sort description: n ASC, x ASC Result sort description: n ASC, x ASC +Complex ORDER BY + optimize_read_in_window_order=0 +3 3 1 +4 5 2 +5 7 3 + optimize_read_in_window_order=1 +3 3 1 +4 5 2 +5 7 3 diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh index 418baea8113..328d181fadd 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh @@ -31,6 +31,15 @@ $CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OV echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" +echo 'Complex ORDER BY' +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${name}_complex (unique1 Int32, unique2 Int32, ten Int32) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192" +$CLICKHOUSE_CLIENT -q "INSERT INTO ${name}_complex VALUES (1, 2, 3), (2, 3, 4), (3, 4, 5)" +echo ' optimize_read_in_window_order=0' +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=0" +echo ' optimize_read_in_window_order=1' +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=1" + $CLICKHOUSE_CLIENT -q "drop table ${name}" $CLICKHOUSE_CLIENT -q "drop table ${name}_n" $CLICKHOUSE_CLIENT -q "drop table ${name}_n_x" +$CLICKHOUSE_CLIENT -q "drop table ${name}_complex" From f205b5480d01a93e12e68a0e03b60a82b54e6f09 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 2 Aug 2022 09:50:40 +0800 Subject: [PATCH 043/582] Fix test cases --- src/Server/HTTPHandler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 9a11130fc20..0d35e8740c0 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -533,7 +533,8 @@ void HTTPHandler::processQuery( session->makeSessionContext(session_id, session_timeout, session_check == "1"); } - auto context = session->makeQueryContext(session->getClientInfo()); + auto client_info = session->getClientInfo(); + auto context = session->makeQueryContext(std::move(client_info)); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); From 8eb254c0c82f1f4b913814d8f9b09f6a3c5e8719 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 2 Aug 2022 10:23:51 +0800 Subject: [PATCH 044/582] Fix merge problem --- src/Storages/Distributed/DirectoryMonitor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 8c0efe52882..6338ef4c9b3 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -632,7 +632,6 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa distributed_header.client_info.client_trace_context, this->storage.getContext()->getOpenTelemetrySpanLog()); - auto connection = pool->get(timeouts, &distributed_header.insert_settings); RemoteInserter remote{*connection, timeouts, distributed_header.insert_query, distributed_header.insert_settings, From 7d8a8f26cddf545d32ed42b6d877eeb83f2e68f7 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 2 Aug 2022 13:31:36 +0800 Subject: [PATCH 045/582] Fix build check --- src/Client/Connection.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index cb1fbe08d67..1855c43a675 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -480,7 +480,7 @@ void Connection::sendQuery( span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); }); ClientInfo new_client_info; - auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); + const auto ¤t_trace_context = OpenTelemetryThreadTraceContext::current(); if (client_info && current_trace_context.isTraceEnabled()) { // use current span as the parent of remote span diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d473cdc5d35..d228c1b80ff 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -103,7 +103,7 @@ namespace ReadWriteBufferFromHTTP::HTTPHeaderEntries headers(headers_.begin(), headers_.end()); // Propagate OpenTelemetry trace context, if any, downstream. - auto& current_trace_context = OpenTelemetryThreadTraceContext::current(); + const auto ¤t_trace_context = OpenTelemetryThreadTraceContext::current(); if (current_trace_context.isTraceEnabled()) { headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); From 57dde8c250c0a8841a3a75820cb24aee882506a2 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 2 Aug 2022 15:43:19 +0800 Subject: [PATCH 046/582] Clean up header inclusion Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 1 + src/Common/OpenTelemetryTraceContext.h | 3 ++- src/Common/ThreadPool.h | 1 + src/Common/ThreadStatus.h | 1 - 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 84141ce267a..e2ac0934069 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 87a67c5a6c9..abb69880b3a 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -1,11 +1,12 @@ #pragma once #include -#include namespace DB { +struct Settings; + struct OpenTelemetrySpan { UUID trace_id; diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 0c0690887c6..0c54600c862 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -14,6 +14,7 @@ #include #include +#include #include /** Very simple thread pool similar to boost::threadpool. diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 9e9b363b301..b40cb7b9fcc 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include From b4c4448f0feb27abe7665ab2b3e570f75b03ac8c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 2 Aug 2022 12:07:38 +0200 Subject: [PATCH 047/582] fix --- src/Common/ZooKeeper/ZooKeeper.cpp | 4 ++-- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index cbca13e49c0..6fcd3b52f16 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -564,7 +564,7 @@ Coordination::Error ZooKeeper::syncImpl(const std::string & path, std::string & { auto future_result = asyncTrySyncNoThrow(path); - if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Sync), path)); return Coordination::Error::ZOPERATIONTIMEOUT; @@ -769,7 +769,7 @@ void ZooKeeper::waitForEphemeralToDisappearIfAny(const std::string & path) if (!tryGet(path, content, nullptr, eph_node_disappeared)) return; - int32_t timeout_ms = 3 * session_timeout_ms; + int32_t timeout_ms = 3 * args.session_timeout_ms; if (!eph_node_disappeared->tryWait(timeout_ms)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Ephemeral node {} still exists after {}s, probably it's owned by someone else. " diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index accdb6f101b..9c323c41eab 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1090,7 +1090,7 @@ void ZooKeeper::initApiVersion() }; get(keeper_api_version_path, std::move(callback), {}); - if (future.wait_for(std::chrono::milliseconds(operation_timeout.totalMilliseconds())) != std::future_status::ready) + if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { LOG_TRACE(log, "Failed to get API version: timeout"); return; From 035896fa16131e5b3011933f498e466349e8955e Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 2 Aug 2022 20:18:42 +0800 Subject: [PATCH 048/582] Fix build check --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 5981b80f346..738e2a26a4d 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -152,7 +152,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti // this scheduleImpl is called in the parent thread, // the tracing context on this thread is used as parent context for the sub-thread that runs the job - auto& current_thread_context = DB::OpenTelemetryThreadTraceContext::current(); + const auto ¤t_thread_context = DB::OpenTelemetryThreadTraceContext::current(); jobs.emplace(std::move(job), priority, current_thread_context); ++scheduled_jobs; new_job_or_shutdown.notify_one(); From 02b0fb2c687fdbf7f962b16d7d23e1d755d0e57c Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 3 Aug 2022 11:26:59 +0800 Subject: [PATCH 049/582] Empty push to trigger CI From 4e1ef89c645e9512347e90145191852295af40cd Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 3 Aug 2022 19:44:11 +0800 Subject: [PATCH 050/582] Fix query in test case to match new behaviour --- tests/integration/test_grpc_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 469113cd68e..52c583973d0 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -744,7 +744,7 @@ def test_opentelemetry_context_propagation(): assert ( node.query( f"SELECT attribute['db.statement'], attribute['clickhouse.tracestate'] FROM system.opentelemetry_span_log " - f"WHERE trace_id='{trace_id}' AND parent_span_id={parent_span_id}" + f"WHERE trace_id='{trace_id}' AND operation_name='query'" ) == "SELECT 1\tsome custom state\n" ) From c222eec756e5b81a6b1cf0087876faaca2bb73b4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 3 Aug 2022 14:23:06 +0200 Subject: [PATCH 051/582] Update pull_request.yml --- .github/workflows/pull_request.yml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8c20de76ced..5d3e9d65966 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1263,7 +1263,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, s3 storage, actions) + CHECK_NAME=Stateless tests (debug, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1300,7 +1300,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, s3 storage, actions) + CHECK_NAME=Stateless tests (debug, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1337,7 +1337,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, s3 storage, actions) + CHECK_NAME=Stateless tests (debug, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1374,7 +1374,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, s3 storage, actions) + CHECK_NAME=Stateless tests (thread, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1411,7 +1411,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, s3 storage, actions) + CHECK_NAME=Stateless tests (thread, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1448,7 +1448,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, s3 storage, actions) + CHECK_NAME=Stateless tests (thread, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 From 2d5446a86bb779d349ef9f45d7d1f2de84e9af4a Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Fri, 5 Aug 2022 14:11:52 +0800 Subject: [PATCH 052/582] add FunctionsUniqTheta --- src/AggregateFunctions/ThetaSketchData.h | 54 ++++++ src/Functions/CMakeLists.txt | 5 + src/Functions/UniqTheta/CMakeLists.txt | 9 + .../UniqTheta/FunctionsUniqTheta.cpp | 19 ++ src/Functions/UniqTheta/FunctionsUniqTheta.h | 164 ++++++++++++++++++ 5 files changed, 251 insertions(+) create mode 100644 src/Functions/UniqTheta/CMakeLists.txt create mode 100644 src/Functions/UniqTheta/FunctionsUniqTheta.cpp create mode 100644 src/Functions/UniqTheta/FunctionsUniqTheta.h diff --git a/src/AggregateFunctions/ThetaSketchData.h b/src/AggregateFunctions/ThetaSketchData.h index f46836ad189..0bce5bf1d5b 100644 --- a/src/AggregateFunctions/ThetaSketchData.h +++ b/src/AggregateFunctions/ThetaSketchData.h @@ -9,6 +9,8 @@ #include #include #include +#include +#include namespace DB @@ -80,6 +82,58 @@ public: u->update(rhs.sk_union->get_result()); } + void intersect(const ThetaSketchData & rhs) + { + datasketches::theta_union * u = getSkUnion(); + + if (sk_update) + { + u->update(*sk_update); + sk_update.reset(nullptr); + } + + datasketches::theta_intersection theta_intersection; + + theta_intersection.update(u->get_result()); + + if (rhs.sk_update) + theta_intersection.update(*rhs.sk_update); + else if (rhs.sk_union) + theta_intersection.update(rhs.sk_union->get_result()); + + sk_union.reset(nullptr); + u = getSkUnion(); + u->update(theta_intersection.get_result()); + } + + void a_not_b(const ThetaSketchData & rhs) + { + datasketches::theta_union * u = getSkUnion(); + + if (sk_update) + { + u->update(*sk_update); + sk_update.reset(nullptr); + } + + datasketches::theta_a_not_b a_not_b; + + if (rhs.sk_update) + { + datasketches::compact_theta_sketch result = a_not_b.compute(u->get_result(), *rhs.sk_update); + sk_union.reset(nullptr); + u = getSkUnion(); + u->update(result); + } + else if (rhs.sk_union) + { + datasketches::compact_theta_sketch result = a_not_b.compute(u->get_result(), rhs.sk_union->get_result()); + sk_union.reset(nullptr); + u = getSkUnion(); + u->update(result); + } + } + /// You can only call for an empty object. void read(DB::ReadBuffer & in) { diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 1bc95b49dbe..0387cc86d48 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -92,6 +92,11 @@ list (APPEND OBJECT_LIBS $) add_subdirectory(array) list (APPEND OBJECT_LIBS $) +if (TARGET ch_contrib::datasketches) + add_subdirectory(UniqTheta) + list (APPEND OBJECT_LIBS $) +endif() + add_subdirectory(JSONPath) list (APPEND PRIVATE_LIBS clickhouse_functions_jsonpath) diff --git a/src/Functions/UniqTheta/CMakeLists.txt b/src/Functions/UniqTheta/CMakeLists.txt new file mode 100644 index 00000000000..27e23eb3881 --- /dev/null +++ b/src/Functions/UniqTheta/CMakeLists.txt @@ -0,0 +1,9 @@ +include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") + +add_library(clickhouse_functions_uniqtheta FunctionsUniqTheta.cpp) + +target_link_libraries(clickhouse_functions_uniqtheta PRIVATE dbms) + +if (TARGET ch_contrib::datasketches) + target_link_libraries (clickhouse_functions_uniqtheta PRIVATE ch_contrib::datasketches) +endif () diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.cpp b/src/Functions/UniqTheta/FunctionsUniqTheta.cpp new file mode 100644 index 00000000000..d3925952b00 --- /dev/null +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.cpp @@ -0,0 +1,19 @@ +#include + +#include "FunctionsUniqTheta.h" + +#if USE_DATASKETCHES + +namespace DB +{ + +REGISTER_FUNCTION(UniqTheta) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} + +} + +#endif diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.h b/src/Functions/UniqTheta/FunctionsUniqTheta.h new file mode 100644 index 00000000000..bd9ad11b9b3 --- /dev/null +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.h @@ -0,0 +1,164 @@ +#pragma once + +#include + +#if USE_DATASKETCHES + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + + + namespace ErrorCodes + { + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; + } + + struct UniqThetaIntersectImpl + { + static void apply(AggregateFunctionUniqThetaData & sketch_data_1, const AggregateFunctionUniqThetaData & sketch_data_2) + { + sketch_data_1.set.intersect(sketch_data_2.set); + } + }; + + struct UniqThetaUnionImpl + { + static void apply(AggregateFunctionUniqThetaData & sketch_data_1, const AggregateFunctionUniqThetaData & sketch_data_2) + { + sketch_data_1.set.merge(sketch_data_2.set); + } + }; + + struct UniqThetaNotImpl + { + static void apply(AggregateFunctionUniqThetaData & sketch_data_1, const AggregateFunctionUniqThetaData & sketch_data_2) + { + sketch_data_1.set.a_not_b(sketch_data_2.set); + } + }; + + template + class FunctionUniqTheta : public IFunction + { + public: + static constexpr auto name = Name::name; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const auto * sketch_type0 = typeid_cast(arguments[0].get()); + if (!(sketch_type0 && sketch_type0->getFunctionName() == "uniqTheta")) + throw Exception( + "First argument for function " + getName() + " must be a uniqTheta but it has type " + arguments[0]->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * sketch_type1 = typeid_cast(arguments[1].get()); + if (!(sketch_type1 && sketch_type1->getFunctionName() == "uniqTheta")) + throw Exception( + "Second argument for function " + getName() + " must be a uniqTheta but it has type " + arguments[1]->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + //todo:check + if (sketch_type0->getArgumentsDataTypes()[0]->getTypeId() != sketch_type1->getArgumentsDataTypes()[0]->getTypeId()) + throw Exception( + "The nested type in uniqThetas must be the same, but one is " + sketch_type0->getArgumentsDataTypes()[0]->getName() + + ", and the other is " + sketch_type1->getArgumentsDataTypes()[0]->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const ColumnAggregateFunction * column_ptrs[2]; + bool is_column_const[2]; + for (size_t i = 0; i < 2; ++i) + { + if (const auto * argument_column_const = typeid_cast(arguments[i].column.get())) + { + column_ptrs[i] = typeid_cast(argument_column_const->getDataColumnPtr().get()); + is_column_const[i] = true; + } + else + { + column_ptrs[i] = typeid_cast(arguments[i].column.get()); + is_column_const[i] = false; + } + } + + auto col_to = ColumnAggregateFunction::create(column_ptrs[0]->getAggregateFunction()); + + col_to->reserve(input_rows_count); + + const PaddedPODArray & container0 = column_ptrs[0]->getData(); + const PaddedPODArray & container1 = column_ptrs[1]->getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + const AggregateDataPtr data_ptr_0 = is_column_const[0] ? container0[0] : container0[i]; + const AggregateDataPtr data_ptr_1 = is_column_const[1] ? container1[0] : container1[i]; + + col_to->insertFrom(data_ptr_0); + AggregateFunctionUniqThetaData & sketch_data_1 = *reinterpret_cast(col_to->getData()[i]); + const AggregateFunctionUniqThetaData & sketch_data_2 + = *reinterpret_cast(data_ptr_1); + Impl::apply(sketch_data_1, sketch_data_2); + } + return col_to; + } + }; + + struct NameUniqThetaIntersect + { + static constexpr auto name = "uniqThetaIntersect"; + }; + + struct NameUniqThetaUnion + { + static constexpr auto name = "uniqThetaUnion"; + }; + + struct NameUniqThetaNot + { + static constexpr auto name = "uniqThetaNot"; + }; + + using FunctionUniqThetaIntersect = FunctionUniqTheta; + using FunctionUniqThetaUnion = FunctionUniqTheta; + using FunctionUniqThetaNot = FunctionUniqTheta; + +} + + +#endif From 40e0192bbcaaaa87cb125b8b0d2d888922c6ff03 Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Fri, 5 Aug 2022 16:22:17 +0800 Subject: [PATCH 053/582] add test --- ...8_uniq_theta_union_intersect_not.reference | 18 ++++ .../01798_uniq_theta_union_intersect_not.sql | 88 +++++++++++++++++++ 2 files changed, 106 insertions(+) create mode 100644 tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.reference create mode 100644 tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql diff --git a/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.reference b/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.reference new file mode 100644 index 00000000000..ff12af23d43 --- /dev/null +++ b/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.reference @@ -0,0 +1,18 @@ +uniqTheta union test +0 0 0 +4 2 3 +4 3 2 +uniqTheta intersect test +0 0 0 +1 2 3 +1 3 2 +uniqTheta union test +0 0 0 +1 2 3 +2 3 2 +uniqTheta retention test +4 9 4 +uniqTheta retention with AggregatingMergeTree test +0.5 2 4 +uniqTheta retention with MergeTree test +0.5 2 4 diff --git a/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql b/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql new file mode 100644 index 00000000000..cc414b06403 --- /dev/null +++ b/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql @@ -0,0 +1,88 @@ +SELECT 'uniqTheta union test'; + +select finalizeAggregation(uniqThetaUnion(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[]) as a, arrayReduce('uniqThetaState',[]) as b ); + +select finalizeAggregation(uniqThetaUnion(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[1,2]) as a, arrayReduce('uniqThetaState',[2,3,4]) as b ); + +select finalizeAggregation(uniqThetaUnion(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[2,3,4]) as a, arrayReduce('uniqThetaState',[1,2]) as b ); + +SELECT 'uniqTheta intersect test'; + +select finalizeAggregation(uniqThetaIntersect(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[]) as a, arrayReduce('uniqThetaState',[]) as b ); + +select finalizeAggregation(uniqThetaIntersect(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[1,2]) as a, arrayReduce('uniqThetaState',[2,3,4]) as b ); + +select finalizeAggregation(uniqThetaIntersect(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[2,3,4]) as a, arrayReduce('uniqThetaState',[1,2]) as b ); + +SELECT 'uniqTheta union test'; + +select finalizeAggregation(uniqThetaNot(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[]) as a, arrayReduce('uniqThetaState',[]) as b ); + +select finalizeAggregation(uniqThetaNot(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[1,2]) as a, arrayReduce('uniqThetaState',[2,3,4]) as b ); + +select finalizeAggregation(uniqThetaNot(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[2,3,4]) as a, arrayReduce('uniqThetaState',[1,2]) as b ); + +SELECT 'uniqTheta retention test'; + +select finalizeAggregation(uniqThetaIntersect(a,b)), finalizeAggregation(a),finalizeAggregation(b) from +( +select (uniqThetaStateIf(number, number>0)) as a, (uniqThetaStateIf(number, number>5)) as b +from +(select number FROM system.numbers LIMIT 10) +); + +SELECT 'uniqTheta retention with AggregatingMergeTree test'; +DROP TABLE IF EXISTS test1; + +CREATE TABLE test1 +( + `year` String , + `uv` AggregateFunction(uniqTheta, Int64) +) +ENGINE = AggregatingMergeTree() +ORDER BY (year); + +INSERT INTO TABLE test1(year, uv) select '2021',uniqThetaState(toInt64(1)); +INSERT INTO TABLE test1(year, uv) select '2021',uniqThetaState(toInt64(2)); +INSERT INTO TABLE test1(year, uv) select '2021',uniqThetaState(toInt64(3)); +INSERT INTO TABLE test1(year, uv) select '2021',uniqThetaState(toInt64(4)); +INSERT INTO TABLE test1(year, uv) select '2022',uniqThetaState(toInt64(1)); +INSERT INTO TABLE test1(year, uv) select '2022',uniqThetaState(toInt64(3)); + +select finalizeAggregation(uniqThetaIntersect(uv2021,uv2022))/finalizeAggregation(uv2021),finalizeAggregation(uniqThetaIntersect(uv2021,uv2022)),finalizeAggregation(uv2021) +from +( +select uniqThetaMergeStateIf(uv,year='2021') as uv2021, uniqThetaMergeStateIf(uv,year='2022') as uv2022 +from test1 +); + +DROP TABLE IF EXISTS test1; + +SELECT 'uniqTheta retention with MergeTree test'; +DROP TABLE IF EXISTS test2; + +CREATE TABLE test2 +( + `year` String , + `uv` Int64 +) +ENGINE = MergeTree() +ORDER BY (year); + +INSERT INTO TABLE test2(year, uv) select '2021',1; +INSERT INTO TABLE test2(year, uv) select '2021',2; +INSERT INTO TABLE test2(year, uv) select '2021',3; +INSERT INTO TABLE test2(year, uv) select '2021',4; +INSERT INTO TABLE test2(year, uv) select '2022',1; +INSERT INTO TABLE test2(year, uv) select '2022',3; + +select finalizeAggregation(uniqThetaIntersect(uv2021,uv2022))/finalizeAggregation(uv2021),finalizeAggregation(uniqThetaIntersect(uv2021,uv2022)),finalizeAggregation(uv2021) +from +( +select uniqThetaStateIf(uv,year='2021') as uv2021, uniqThetaStateIf(uv,year='2022') as uv2022 +from test2 +); + + + +DROP TABLE IF EXISTS test2; From 7ddadd25a3a0217f55ad8dcb8adfbf8e643b3d0c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 26 Jul 2022 09:08:55 +0000 Subject: [PATCH 054/582] Initial implementation of KeeperMap --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Storages/KVStorageUtils.cpp | 175 ++++++++++ src/Storages/KVStorageUtils.h | 47 +++ .../RocksDB/StorageEmbeddedRocksDB.cpp | 186 +--------- src/Storages/StorageKeeperMap.cpp | 329 ++++++++++++++++++ src/Storages/StorageKeeperMap.h | 52 +++ src/Storages/registerStorages.cpp | 3 + 7 files changed, 608 insertions(+), 186 deletions(-) create mode 100644 src/Storages/KVStorageUtils.cpp create mode 100644 src/Storages/KVStorageUtils.h create mode 100644 src/Storages/StorageKeeperMap.cpp create mode 100644 src/Storages/StorageKeeperMap.h diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7a00bbf524c..ea11a1b23ac 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -720,7 +720,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } /// We can have queries like "CREATE TABLE ENGINE=" if /// supports schema inference (will determine table structure in it's constructor). - else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) // NOLINT + else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name) && create.storage->engine->name != "KeeperMap") // NOLINT throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); /// Even if query has list of columns, canonicalize it (unfold Nested columns). diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp new file mode 100644 index 00000000000..8c99d593f40 --- /dev/null +++ b/src/Storages/KVStorageUtils.cpp @@ -0,0 +1,175 @@ +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ +// returns keys may be filter by condition +bool traverseASTFilter( + const std::string & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, const ContextPtr & context, FieldVectorPtr & res) +{ + const auto * function = elem->as(); + if (!function) + return false; + + if (function->name == "and") + { + // one child has the key filter condition is ok + for (const auto & child : function->arguments->children) + if (traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) + return true; + return false; + } + else if (function->name == "or") + { + // make sure every child has the key filter condition + for (const auto & child : function->arguments->children) + if (!traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) + return false; + return true; + } + else if (function->name == "equals" || function->name == "in") + { + const auto & args = function->arguments->as(); + const ASTIdentifier * ident; + std::shared_ptr value; + + if (args.children.size() != 2) + return false; + + if (function->name == "in") + { + ident = args.children.at(0)->as(); + if (!ident) + return false; + + if (ident->name() != primary_key) + return false; + value = args.children.at(1); + + PreparedSetKey set_key; + if ((value->as() || value->as())) + set_key = PreparedSetKey::forSubquery(*value); + else + set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + + auto set_it = sets.find(set_key); + if (set_it == sets.end()) + return false; + SetPtr prepared_set = set_it->second; + + if (!prepared_set->hasExplicitSetElements()) + return false; + + prepared_set->checkColumnsNumber(1); + const auto & set_column = *prepared_set->getSetElements()[0]; + for (size_t row = 0; row < set_column.size(); ++row) + res->push_back(set_column[row]); + return true; + } + else + { + if ((ident = args.children.at(0)->as())) + value = args.children.at(1); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0); + else + return false; + + if (ident->name() != primary_key) + return false; + + const auto node = evaluateConstantExpressionAsLiteral(value, context); + /// function->name == "equals" + if (const auto * literal = node->as()) + { + auto converted_field = convertFieldToType(literal->value, *primary_key_type); + if (!converted_field.isNull()) + res->push_back(converted_field); + return true; + } + } + } + return false; +} +} + +std::pair getFilterKeys( + const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context) +{ + const auto & select = query_info.query->as(); + if (!select.where()) + return {{}, true}; + + FieldVectorPtr res = std::make_shared(); + auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.sets, context, res); + return std::make_pair(res, !matched_keys); +} + +std::vector serializeKeysToRawString( + FieldVector::const_iterator & it, + FieldVector::const_iterator end, + DataTypePtr key_column_type, + size_t max_block_size) +{ + size_t num_keys = end - it; + + std::vector result; + result.reserve(num_keys); + + size_t rows_processed = 0; + while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) + { + std::string & serialized_key = result.emplace_back(); + WriteBufferFromString wb(serialized_key); + key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); + wb.finalize(); + + ++it; + ++rows_processed; + } + return result; +} + +std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys) +{ + if (!keys.column) + return {}; + + size_t num_keys = keys.column->size(); + std::vector result; + result.reserve(num_keys); + + for (size_t i = 0; i < num_keys; ++i) + { + std::string & serialized_key = result.emplace_back(); + WriteBufferFromString wb(serialized_key); + Field field; + keys.column->get(i, field); + /// TODO(@vdimir): use serializeBinaryBulk + keys.type->getDefaultSerialization()->serializeBinary(field, wb); + wb.finalize(); + } + return result; +} + +/// In current implementation rocks db can have key with only one column. +size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) +{ + if (primary_key.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); + return header.getPositionByName(primary_key[0]); +} + +} diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h new file mode 100644 index 00000000000..ca6aecd1374 --- /dev/null +++ b/src/Storages/KVStorageUtils.h @@ -0,0 +1,47 @@ +#pragma once + +#include + +#include +#include + +#include + +namespace DB +{ + +using FieldVectorPtr = std::shared_ptr; + +class IDataType; +using DataTypePtr = std::shared_ptr; + +/** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. + * TODO support key like search + */ +std::pair getFilterKeys( + const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context); + +template +void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) +{ + ReadBufferFromString key_buffer(key); + ReadBufferFromString value_buffer(value); + for (size_t i = 0; i < header.columns(); ++i) + { + const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); + serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); + } +} + +std::vector serializeKeysToRawString( + FieldVector::const_iterator & it, + FieldVector::const_iterator end, + DataTypePtr key_column_type, + size_t max_block_size); + +std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys); + +/// In current implementation key with only column is supported. +size_t getPrimaryKeyPos(const Block & header, const Names & primary_key); + +} diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 2774c52fe7c..dda27c281cb 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -3,30 +3,17 @@ #include -#include #include +#include -#include -#include -#include -#include -#include -#include #include -#include -#include - #include #include #include #include -#include -#include #include -#include -#include #include #include @@ -75,177 +62,6 @@ static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfigurati return options; } -// returns keys may be filter by condition -static bool traverseASTFilter( - const String & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, const ContextPtr & context, FieldVectorPtr & res) -{ - const auto * function = elem->as(); - if (!function) - return false; - - if (function->name == "and") - { - // one child has the key filter condition is ok - for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) - return true; - return false; - } - else if (function->name == "or") - { - // make sure every child has the key filter condition - for (const auto & child : function->arguments->children) - if (!traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) - return false; - return true; - } - else if (function->name == "equals" || function->name == "in") - { - const auto & args = function->arguments->as(); - const ASTIdentifier * ident; - std::shared_ptr value; - - if (args.children.size() != 2) - return false; - - if (function->name == "in") - { - ident = args.children.at(0)->as(); - if (!ident) - return false; - - if (ident->name() != primary_key) - return false; - value = args.children.at(1); - - PreparedSetKey set_key; - if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(*value); - else - set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); - - auto set_it = sets.find(set_key); - if (set_it == sets.end()) - return false; - SetPtr prepared_set = set_it->second; - - if (!prepared_set->hasExplicitSetElements()) - return false; - - prepared_set->checkColumnsNumber(1); - const auto & set_column = *prepared_set->getSetElements()[0]; - for (size_t row = 0; row < set_column.size(); ++row) - res->push_back(set_column[row]); - return true; - } - else - { - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; - - if (ident->name() != primary_key) - return false; - - const auto node = evaluateConstantExpressionAsLiteral(value, context); - /// function->name == "equals" - if (const auto * literal = node->as()) - { - auto converted_field = convertFieldToType(literal->value, *primary_key_type); - if (!converted_field.isNull()) - res->push_back(converted_field); - return true; - } - } - } - return false; -} - -/** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. - * TODO support key like search - */ -static std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context) -{ - const auto & select = query_info.query->as(); - if (!select.where()) - return {{}, true}; - - FieldVectorPtr res = std::make_shared(); - auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.sets, context, res); - return std::make_pair(res, !matched_keys); -} - -template -static void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) -{ - ReadBufferFromString key_buffer(key); - ReadBufferFromString value_buffer(value); - for (size_t i = 0; i < header.columns(); ++i) - { - const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); - serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); - } -} - -static std::vector serializeKeysToRawString( - FieldVector::const_iterator & it, - FieldVector::const_iterator end, - DataTypePtr key_column_type, - size_t max_block_size) -{ - size_t num_keys = end - it; - - std::vector result; - result.reserve(num_keys); - - size_t rows_processed = 0; - while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) - { - std::string & serialized_key = result.emplace_back(); - WriteBufferFromString wb(serialized_key); - key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); - wb.finalize(); - - ++it; - ++rows_processed; - } - return result; -} - -static std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys) -{ - if (!keys.column) - return {}; - - size_t num_keys = keys.column->size(); - std::vector result; - result.reserve(num_keys); - - for (size_t i = 0; i < num_keys; ++i) - { - std::string & serialized_key = result.emplace_back(); - WriteBufferFromString wb(serialized_key); - Field field; - keys.column->get(i, field); - /// TODO(@vdimir): use serializeBinaryBulk - keys.type->getDefaultSerialization()->serializeBinary(field, wb); - wb.finalize(); - } - return result; -} - -/// In current implementation rocks db can have key with only one column. -static size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) -{ - if (primary_key.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); - return header.getPositionByName(primary_key[0]); -} - class EmbeddedRocksDBSource : public ISource { public: diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp new file mode 100644 index 00000000000..1208ac31477 --- /dev/null +++ b/src/Storages/StorageKeeperMap.cpp @@ -0,0 +1,329 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Parsers/ASTExpressionList.h" +#include "Parsers/ASTFunction.h" +#include +#include +#include "Parsers/ASTSelectQuery.h" +#include "Storages/MergeTree/IMergeTreeDataPart.h" +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +StorageKeeperMap::StorageKeeperMap(std::string_view keeper_path_, ContextPtr context, const StorageID & table_id) + : IStorage(table_id), keeper_path(keeper_path_), zookeeper_client(context->getZooKeeper()->startNewSession()) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription{getNamesAndTypes()}); + setInMemoryMetadata(storage_metadata); + + if (keeper_path.empty()) + throw Exception("keeper_path should not be empty", ErrorCodes::BAD_ARGUMENTS); + if (!keeper_path.starts_with('/')) + throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); + + if (keeper_path != "/") + { + LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); + + size_t cur_pos = 0; + do + { + size_t search_start = cur_pos + 1; + cur_pos = keeper_path.find('/', search_start); + if (search_start == cur_pos) + throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + + auto path = keeper_path.substr(0, cur_pos); + auto status = getClient()->tryCreate(path, "", zkutil::CreateMode::Persistent); + if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(status, path); + } while (cur_pos != std::string_view::npos); + } +} + +NamesAndTypesList StorageKeeperMap::getNamesAndTypes() +{ + return {{"key", std::make_shared()}, {"value", std::make_shared()}}; +} + +class StorageKeeperMapSink : public SinkToStorage +{ + StorageKeeperMap & storage; + std::unordered_map new_values; + +public: + StorageKeeperMapSink(const Block & header, StorageKeeperMap & storage_) : SinkToStorage(header), storage(storage_) { } + + std::string getName() const override { return "StorageKeeperMapSink"; } + + void consume(Chunk chunk) override + { + auto block = getHeader().cloneWithColumns(chunk.getColumns()); + + size_t rows = block.rows(); + for (size_t i = 0; i < rows; ++i) + { + std::string key = block.getByPosition(0).column->getDataAt(i).toString(); + + if (key.find('/') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key cannot contain '/'. Key: '{}'", key); + + std::string value = block.getByPosition(1).column->getDataAt(i).toString(); + + new_values[std::move(key)] = std::move(value); + } + } + + void onFinish() override + { + auto & zookeeper = storage.getClient(); + Coordination::Requests requests; + for (const auto & [key, value] : new_values) + { + auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); + + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + } + + zookeeper->multi(requests); + } +}; + +enum class FilterType +{ + EXACT = 0 +}; + +struct KeyFilter +{ + std::string filter; + FilterType type; +}; + +class StorageKeeperMapSource : public ISource +{ + StorageKeeperMap & storage; + std::vector keys; + size_t current_idx = 0; + Block sample_block; + Names column_names; + bool has_value_column{false}; + size_t max_block_size; + std::optional filter; + + Chunk generateSingleKey() + { + assert(filter && filter->type == FilterType::EXACT); + static bool processed = false; + + if (processed) + return {}; + + auto zookeeper = storage.getClient(); + + std::string value; + auto path = fmt::format("{}/{}", storage.rootKeeperPath(), filter->filter); + auto res = zookeeper->tryGet(path, value); + if (!res) + return {}; + + MutableColumns columns(sample_block.cloneEmptyColumns()); + insertRowForKey(columns, filter->filter, value); + processed = true; + + return Chunk{std::move(columns), 1}; + } + + bool insertRowForKey(MutableColumns & columns, const std::string & key, const std::string & value) + { + + for (size_t column_index = 0; column_index < column_names.size(); ++column_index) + { + if (column_names[column_index] == "key") + assert_cast(*columns[column_index]).insertData(key.data(), key.size()); + else if (column_names[column_index] == "value") + assert_cast(*columns[column_index]).insertData(value.data(), value.size()); + } + return true; + } + +public: + StorageKeeperMapSource(const Block & sample_block_, StorageKeeperMap & storage_, size_t max_block_size_, std::optional filter_) + : ISource(sample_block_) + , storage(storage_) + , sample_block(sample_block_.cloneEmpty()) + , column_names(sample_block_.getNames()) + , max_block_size(max_block_size_) + , filter(std::move(filter_)) + { + has_value_column = std::any_of(column_names.begin(), column_names.end(), [](const auto & name) { return name == "value"; }); + + // TODO(antonio2368): Do it lazily in generate + if (!filter || filter->type != FilterType::EXACT) + { + auto zookeeper = storage.getClient(); + keys = zookeeper->getChildren(storage.rootKeeperPath()); + } + } + + std::string getName() const override { return "StorageKeeperMapSource"; } + + Chunk generate() override + { + if (filter && filter->type == FilterType::EXACT) + return generateSingleKey(); + + auto zookeeper = storage.getClient(); + + MutableColumns columns(sample_block.cloneEmptyColumns()); + size_t num_rows = 0; + for (; num_rows < max_block_size && current_idx != keys.size(); ++current_idx) + { + const auto & key = keys[current_idx]; + std::string value; + if (has_value_column) + { + auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); + auto res = zookeeper->tryGet(path, value); + if (!res) + continue; + } + + if (insertRowForKey(columns, key, value)) + ++num_rows; + } + + if (num_rows == 0) + return {}; + + return {std::move(columns), num_rows}; + } +}; + +std::optional tryGetKeyFilter(const IAST & elem, const ContextPtr context) +{ + const auto * function = elem.as(); + if (!function) + return std::nullopt; + + if (function->name != "equals") + return std::nullopt; + + const auto & args = function->arguments->as(); + const ASTIdentifier * ident; + ASTPtr value; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0); + else + return std::nullopt; + + if (ident->name() != "key") + return std::nullopt; + + auto evaluated = evaluateConstantExpressionAsLiteral(value, context); + const auto * literal = evaluated->as(); + if (!literal) + return std::nullopt; + + if (literal->value.getType() != Field::Types::String) + return std::nullopt; + + return KeyFilter{literal->value.safeGet(), FilterType::EXACT}; +} + +Pipe StorageKeeperMap::read( + const Names & column_names, + const StorageSnapshotPtr & /*storage_snapshot*/, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + unsigned /*num_streams*/) +{ + Block sample_block; + for (const std::string & column_name : column_names) + { + sample_block.insert({std::make_shared(), column_name}); + } + + const auto & select = query_info.query->as(); + std::optional key_filter; + if (select.where()) + key_filter = tryGetKeyFilter(*select.where(), context); + + return Pipe(std::make_shared(sample_block, *this, max_block_size, std::move(key_filter))); +} + +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +{ + auto columns = getNamesAndTypes(); + Block write_header; + for (const auto & [name, type] : columns) + { + write_header.insert(ColumnWithTypeAndName(type, name)); + } + + return std::make_shared(write_header, *this); +} + +zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() +{ + if (zookeeper_client->expired()) + zookeeper_client = zookeeper_client->startNewSession(); + + return zookeeper_client; +} + +const std::string & StorageKeeperMap::rootKeeperPath() const +{ + return keeper_path; +} + +void registerStorageKeeperMap(StorageFactory & factory) +{ + factory.registerStorage( + "KeeperMap", + [](const StorageFactory::Arguments & args) + { + if (!args.attach && !args.columns.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Storage KeeperMap does not accept column definition as it has predefined columns (key String, value String)"); + + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1 argument: " + "keeper_path, path in the Keeper where the values will be stored"); + + auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + + return std::make_shared(keeper_path, args.getContext(), args.table_id); + }, + {}); +} + +} diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h new file mode 100644 index 00000000000..32542a6ecf6 --- /dev/null +++ b/src/Storages/StorageKeeperMap.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +// KV store using (Zoo|CH)Keeper +class StorageKeeperMap final : public IStorage +{ +public: + // TODO(antonio2368): add setting to control creating if keeper_path doesn't exist + StorageKeeperMap( + std::string_view keeper_path_, + ContextPtr context, + const StorageID & table_id + ); + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr context) override; + + std::string getName() const override + { + return "KeeperMap"; + } + + static NamesAndTypesList getNamesAndTypes(); + + zkutil::ZooKeeperPtr & getClient(); + + const std::string & rootKeeperPath() const; +private: + + std::string keeper_path; + zkutil::ZooKeeperPtr zookeeper_client; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 575b3de7ae2..055270be4ae 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -88,6 +88,7 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif +void registerStorageKeeperMap(StorageFactory & factory); void registerStorages() { @@ -171,6 +172,8 @@ void registerStorages() #if USE_SQLITE registerStorageSQLite(factory); #endif + + registerStorageKeeperMap(factory); } } From 034807030d199a3cac5bd3e9af292c93d0cc65ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 27 Jul 2022 13:20:45 +0000 Subject: [PATCH 055/582] Support multiple columns --- src/Storages/KVStorageUtils.cpp | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 1 - src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 1 + src/Storages/StorageKeeperMap.cpp | 587 ++++++++++-------- src/Storages/StorageKeeperMap.h | 52 +- 5 files changed, 369 insertions(+), 274 deletions(-) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 8c99d593f40..91b7e7feda6 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -168,7 +168,7 @@ std::vector serializeKeysToRawString(const ColumnWithTypeAndName & size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) { if (primary_key.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one primary key is supported"); return header.getPositionByName(primary_key[0]); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index dda27c281cb..9df60ede024 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -404,7 +404,6 @@ Chunk StorageEmbeddedRocksDB::getBySerializedKeys( for (const auto & key : keys) slices_keys.emplace_back(key); - auto statuses = multiGet(slices_keys, values); if (null_map) { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 62c9a0eeae7..169b6c1e1bc 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1208ac31477..11c733dd0bb 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -1,37 +1,202 @@ +#include + #include + +#include + #include + +#include + +#include +#include +#include +#include +#include + #include #include + +#include +#include +#include #include -#include -#include +#include +#include + #include #include -#include "Parsers/ASTExpressionList.h" -#include "Parsers/ASTFunction.h" -#include -#include -#include "Parsers/ASTSelectQuery.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" -#include -#include -#include +#include + +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int BAD_ARGUMENTS; +extern const int KEEPER_EXCEPTION; } -StorageKeeperMap::StorageKeeperMap(std::string_view keeper_path_, ContextPtr context, const StorageID & table_id) - : IStorage(table_id), keeper_path(keeper_path_), zookeeper_client(context->getZooKeeper()->startNewSession()) +namespace { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(ColumnsDescription{getNamesAndTypes()}); - setInMemoryMetadata(storage_metadata); + +std::string base64Encode(const std::string & decoded) +{ + std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ostr.exceptions(std::ios::failbit); + Poco::Base64Encoder encoder(ostr, Poco::BASE64_URL_ENCODING); + encoder.rdbuf()->setLineLength(0); + encoder << decoded; + encoder.close(); + return ostr.str(); +} + +std::string base64Decode(const std::string & encoded) +{ + std::string decoded; + Poco::MemoryInputStream istr(encoded.data(), encoded.size()); + Poco::Base64Decoder decoder(istr, Poco::BASE64_URL_ENCODING); + Poco::StreamCopier::copyToString(decoder, decoded); + return decoded; +} + +} + +class StorageKeeperMapSink : public SinkToStorage +{ + StorageKeeperMap & storage; + std::unordered_map new_values; + size_t primary_key_pos; + +public: + StorageKeeperMapSink(StorageKeeperMap & storage_, const StorageMetadataPtr & metadata_snapshot) + : SinkToStorage(metadata_snapshot->getSampleBlock()), storage(storage_) + { + auto primary_key = storage.getPrimaryKey(); + assert(primary_key.size() == 1); + primary_key_pos = getHeader().getPositionByName(storage.getPrimaryKey()[0]); + } + + std::string getName() const override { return "StorageKeeperMapSink"; } + + void consume(Chunk chunk) override + { + auto rows = chunk.getNumRows(); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + + WriteBufferFromOwnString wb_key; + WriteBufferFromOwnString wb_value; + + for (size_t i = 0; i < rows; ++i) + { + wb_key.restart(); + wb_value.restart(); + + size_t idx = 0; + for (const auto & elem : block) + { + elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); + ++idx; + } + + auto key = base64Encode(wb_key.str()); + new_values[std::move(key)] = std::move(wb_value.str()); + } + } + + void onFinish() override + { + auto & zookeeper = storage.getClient(); + Coordination::Requests requests; + for (const auto & [key, value] : new_values) + { + auto path = storage.fullPathForKey(key); + + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + } + + zookeeper->multi(requests); + } +}; + +template +class StorageKeeperMapSource : public ISource +{ + const StorageKeeperMap & storage; + size_t max_block_size; + + using KeyContainerPtr = std::shared_ptr; + KeyContainerPtr container; + using KeyContainerIter = typename KeyContainer::const_iterator; + KeyContainerIter it; + KeyContainerIter end; + +public: + StorageKeeperMapSource( + const StorageKeeperMap & storage_, + const Block & header, + size_t max_block_size_, + KeyContainerPtr container_, + KeyContainerIter begin_, + KeyContainerIter end_) + : ISource(header), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_) + { + } + + std::string getName() const override { return "StorageKeeperMapSource"; } + + Chunk generate() override + { + if (it >= end) + { + it = {}; + return {}; + } + + using KeyType = typename KeyContainer::value_type; + if constexpr (std::same_as) + { + const auto & sample_block = getPort().getHeader(); + const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey().at(0)).type; + auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); + + for (auto & raw_key : raw_keys) + raw_key = base64Encode(raw_key); + + return storage.getBySerializedKeys(raw_keys, nullptr); + } + else + { + size_t elem_num = std::min(max_block_size, static_cast(end - it)); + auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr); + it += elem_num; + return chunk; + } + } +}; + +StorageKeeperMap::StorageKeeperMap( + ContextPtr context, + const StorageID & table_id, + const StorageInMemoryMetadata & metadata, + std::string_view primary_key_, + std::string_view keeper_path_) + : IKeyValueStorage(table_id) + , keeper_path(keeper_path_) + , primary_key(primary_key_) + , zookeeper_client(context->getZooKeeper()->startNewSession()) +{ + setInMemoryMetadata(metadata); if (keeper_path.empty()) throw Exception("keeper_path should not be empty", ErrorCodes::BAD_ARGUMENTS); @@ -58,237 +223,66 @@ StorageKeeperMap::StorageKeeperMap(std::string_view keeper_path_, ContextPtr con } } -NamesAndTypesList StorageKeeperMap::getNamesAndTypes() -{ - return {{"key", std::make_shared()}, {"value", std::make_shared()}}; -} - -class StorageKeeperMapSink : public SinkToStorage -{ - StorageKeeperMap & storage; - std::unordered_map new_values; - -public: - StorageKeeperMapSink(const Block & header, StorageKeeperMap & storage_) : SinkToStorage(header), storage(storage_) { } - - std::string getName() const override { return "StorageKeeperMapSink"; } - - void consume(Chunk chunk) override - { - auto block = getHeader().cloneWithColumns(chunk.getColumns()); - - size_t rows = block.rows(); - for (size_t i = 0; i < rows; ++i) - { - std::string key = block.getByPosition(0).column->getDataAt(i).toString(); - - if (key.find('/') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key cannot contain '/'. Key: '{}'", key); - - std::string value = block.getByPosition(1).column->getDataAt(i).toString(); - - new_values[std::move(key)] = std::move(value); - } - } - - void onFinish() override - { - auto & zookeeper = storage.getClient(); - Coordination::Requests requests; - for (const auto & [key, value] : new_values) - { - auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); - - if (zookeeper->exists(path)) - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - else - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); - } - - zookeeper->multi(requests); - } -}; - -enum class FilterType -{ - EXACT = 0 -}; - -struct KeyFilter -{ - std::string filter; - FilterType type; -}; - -class StorageKeeperMapSource : public ISource -{ - StorageKeeperMap & storage; - std::vector keys; - size_t current_idx = 0; - Block sample_block; - Names column_names; - bool has_value_column{false}; - size_t max_block_size; - std::optional filter; - - Chunk generateSingleKey() - { - assert(filter && filter->type == FilterType::EXACT); - static bool processed = false; - - if (processed) - return {}; - - auto zookeeper = storage.getClient(); - - std::string value; - auto path = fmt::format("{}/{}", storage.rootKeeperPath(), filter->filter); - auto res = zookeeper->tryGet(path, value); - if (!res) - return {}; - - MutableColumns columns(sample_block.cloneEmptyColumns()); - insertRowForKey(columns, filter->filter, value); - processed = true; - - return Chunk{std::move(columns), 1}; - } - - bool insertRowForKey(MutableColumns & columns, const std::string & key, const std::string & value) - { - - for (size_t column_index = 0; column_index < column_names.size(); ++column_index) - { - if (column_names[column_index] == "key") - assert_cast(*columns[column_index]).insertData(key.data(), key.size()); - else if (column_names[column_index] == "value") - assert_cast(*columns[column_index]).insertData(value.data(), value.size()); - } - return true; - } - -public: - StorageKeeperMapSource(const Block & sample_block_, StorageKeeperMap & storage_, size_t max_block_size_, std::optional filter_) - : ISource(sample_block_) - , storage(storage_) - , sample_block(sample_block_.cloneEmpty()) - , column_names(sample_block_.getNames()) - , max_block_size(max_block_size_) - , filter(std::move(filter_)) - { - has_value_column = std::any_of(column_names.begin(), column_names.end(), [](const auto & name) { return name == "value"; }); - - // TODO(antonio2368): Do it lazily in generate - if (!filter || filter->type != FilterType::EXACT) - { - auto zookeeper = storage.getClient(); - keys = zookeeper->getChildren(storage.rootKeeperPath()); - } - } - - std::string getName() const override { return "StorageKeeperMapSource"; } - - Chunk generate() override - { - if (filter && filter->type == FilterType::EXACT) - return generateSingleKey(); - - auto zookeeper = storage.getClient(); - - MutableColumns columns(sample_block.cloneEmptyColumns()); - size_t num_rows = 0; - for (; num_rows < max_block_size && current_idx != keys.size(); ++current_idx) - { - const auto & key = keys[current_idx]; - std::string value; - if (has_value_column) - { - auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); - auto res = zookeeper->tryGet(path, value); - if (!res) - continue; - } - - if (insertRowForKey(columns, key, value)) - ++num_rows; - } - - if (num_rows == 0) - return {}; - - return {std::move(columns), num_rows}; - } -}; - -std::optional tryGetKeyFilter(const IAST & elem, const ContextPtr context) -{ - const auto * function = elem.as(); - if (!function) - return std::nullopt; - - if (function->name != "equals") - return std::nullopt; - - const auto & args = function->arguments->as(); - const ASTIdentifier * ident; - ASTPtr value; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return std::nullopt; - - if (ident->name() != "key") - return std::nullopt; - - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return std::nullopt; - - if (literal->value.getType() != Field::Types::String) - return std::nullopt; - - return KeyFilter{literal->value.safeGet(), FilterType::EXACT}; -} Pipe StorageKeeperMap::read( const Names & column_names, - const StorageSnapshotPtr & /*storage_snapshot*/, + const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned /*num_streams*/) + unsigned num_streams) { - Block sample_block; - for (const std::string & column_name : column_names) + storage_snapshot->check(column_names); + + FieldVectorPtr filtered_keys; + bool all_scan; + + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + auto primary_key_type = sample_block.getByName(primary_key).type; + std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info); + + const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { - sample_block.insert({std::make_shared(), column_name}); - } + if (keys->empty()) + return {}; - const auto & select = query_info.query->as(); - std::optional key_filter; - if (select.where()) - key_filter = tryGetKeyFilter(*select.where(), context); + ::sort(keys->begin(), keys->end()); + keys->erase(std::unique(keys->begin(), keys->end()), keys->end()); - return Pipe(std::make_shared(sample_block, *this, max_block_size, std::move(key_filter))); + Pipes pipes; + + size_t num_keys = keys->size(); + size_t num_threads = std::min(num_streams, keys->size()); + + assert(num_keys <= std::numeric_limits::max()); + assert(num_threads <= std::numeric_limits::max()); + + for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) + { + size_t begin = num_keys * thread_idx / num_threads; + size_t end = num_keys * (thread_idx + 1) / num_threads; + + using KeyContainer = typename KeyContainerPtr::element_type; + pipes.emplace_back(std::make_shared>( + *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end)); + } + return Pipe::unitePipes(std::move(pipes)); + }; + + auto & client = getClient(); + if (all_scan) + return process_keys(std::make_shared>(client->getChildren(keeper_path))); + + return process_keys(std::move(filtered_keys)); } -SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { - auto columns = getNamesAndTypes(); - Block write_header; - for (const auto & [name, type] : columns) - { - write_header.insert(ColumnWithTypeAndName(type, name)); - } - - return std::make_shared(write_header, *this); + return std::make_shared(*this, metadata_snapshot); } -zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() +zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { if (zookeeper_client->expired()) zookeeper_client = zookeeper_client->startNewSession(); @@ -301,29 +295,118 @@ const std::string & StorageKeeperMap::rootKeeperPath() const return keeper_path; } +std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const +{ + return fmt::format("{}/{}", keeper_path, key); +} + +Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const +{ + if (keys.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageKeeperMap supports only one key, got: {}", keys.size()); + + auto raw_keys = serializeKeysToRawString(keys[0]); + + if (raw_keys.size() != keys[0].column->size()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); + + return getBySerializedKeys(raw_keys, &null_map); +} + +Chunk StorageKeeperMap::getBySerializedKeys(const std::span keys, PaddedPODArray * null_map) const +{ + Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); + MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t primary_key_pos = getPrimaryKeyPos(sample_block, getPrimaryKey()); + + if (null_map) + { + null_map->clear(); + null_map->resize_fill(keys.size(), 1); + } + + auto client = getClient(); + + std::vector> values; + values.reserve(keys.size()); + + for (const auto & key : keys) + { + values.emplace_back(client->asyncTryGet(fullPathForKey(key))); + } + + auto wait_until = std::chrono::system_clock::now() + std::chrono::milliseconds(Coordination::DEFAULT_OPERATION_TIMEOUT_MS); + + for (size_t i = 0; i < keys.size(); ++i) + { + auto & value = values[i]; + if (value.wait_until(wait_until) != std::future_status::ready) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fetch values: timeout"); + + auto response = value.get(); + Coordination::Error code = response.error; + + if (code == Coordination::Error::ZOK) + { + fillColumns(base64Decode(keys[i]), response.data, primary_key_pos, sample_block, columns); + } + else if (code == Coordination::Error::ZNONODE) + { + if (null_map) + { + (*null_map)[i] = 0; + for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx) + columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault()); + } + } + else + { + throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch value: {}", code); + } + } + + size_t num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); +} + +namespace +{ +StoragePtr create(const StorageFactory::Arguments & args) +{ + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1 argument: " + "keeper_path, path in the Keeper where the values will be stored"); + + auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->primary_key) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + return std::make_shared(args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path); +} +} + void registerStorageKeeperMap(StorageFactory & factory) { factory.registerStorage( "KeeperMap", - [](const StorageFactory::Arguments & args) + create, { - if (!args.attach && !args.columns.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Storage KeeperMap does not accept column definition as it has predefined columns (key String, value String)"); - - ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 1) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1 argument: " - "keeper_path, path in the Keeper where the values will be stored"); - - auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); - - return std::make_shared(keeper_path, args.getContext(), args.table_id); - }, - {}); + .supports_sort_order = true, + .supports_parallel_insert = true, + }); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 32542a6ecf6..bde76e958ba 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -1,23 +1,29 @@ #pragma once -#include #include #include +#include #include +#include +#include "Common/PODArray_fwd.h" +#include + +#include namespace DB { -// KV store using (Zoo|CH)Keeper -class StorageKeeperMap final : public IStorage +// KV store using (Zoo|CH)Keeper +class StorageKeeperMap final : public IKeyValueStorage { public: // TODO(antonio2368): add setting to control creating if keeper_path doesn't exist StorageKeeperMap( - std::string_view keeper_path_, - ContextPtr context, - const StorageID & table_id - ); + ContextPtr context, + const StorageID & table_id, + const StorageInMemoryMetadata & metadata, + std::string_view primary_key_, + std::string_view keeper_path_); Pipe read( const Names & column_names, @@ -27,26 +33,32 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; - std::string getName() const override + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + + std::string getName() const override { return "KeeperMap"; } + Names getPrimaryKey() const override { return {primary_key}; } + + Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const override; + Chunk getBySerializedKeys(std::span keys, PaddedPODArray * null_map) const; + + bool supportsParallelInsert() const override { return true; } + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn( + const ASTPtr & node, ContextPtr /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override { - return "KeeperMap"; + return node->getColumnName() == primary_key; } - static NamesAndTypesList getNamesAndTypes(); - - zkutil::ZooKeeperPtr & getClient(); - + zkutil::ZooKeeperPtr & getClient() const; const std::string & rootKeeperPath() const; -private: + std::string fullPathForKey(std::string_view key) const; +private: std::string keeper_path; - zkutil::ZooKeeperPtr zookeeper_client; + std::string primary_key; + + mutable zkutil::ZooKeeperPtr zookeeper_client; }; } From 85927c18c06624f4ae42d76ac0f108ceae79ee4c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 13:34:14 +0000 Subject: [PATCH 056/582] Add more engine arguments --- src/Storages/StorageKeeperMap.cpp | 99 ++++++++++++++++++++++--------- src/Storages/StorageKeeperMap.h | 4 +- 2 files changed, 75 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 11c733dd0bb..8e297e63273 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -39,9 +39,9 @@ namespace DB namespace ErrorCodes { -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int BAD_ARGUMENTS; -extern const int KEEPER_EXCEPTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int KEEPER_EXCEPTION; } namespace @@ -67,6 +67,8 @@ std::string base64Decode(const std::string & encoded) return decoded; } +constexpr std::string_view default_host = "default"; + } class StorageKeeperMapSink : public SinkToStorage @@ -185,16 +187,28 @@ public: } }; +namespace +{ + +zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) +{ + if (hosts == default_host) + return context->getZooKeeper()->startNewSession(); + + return std::make_shared(hosts); +} + +} + StorageKeeperMap::StorageKeeperMap( ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata, std::string_view primary_key_, - std::string_view keeper_path_) - : IKeyValueStorage(table_id) - , keeper_path(keeper_path_) - , primary_key(primary_key_) - , zookeeper_client(context->getZooKeeper()->startNewSession()) + std::string_view keeper_path_, + const std::string & hosts, + bool create_missing_root_path) + : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) { setInMemoryMetadata(metadata); @@ -203,23 +217,34 @@ StorageKeeperMap::StorageKeeperMap( if (!keeper_path.starts_with('/')) throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); - if (keeper_path != "/") + auto client = getClient(); + if (keeper_path != "/" && !client->exists(keeper_path)) { - LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); - - size_t cur_pos = 0; - do + if (!create_missing_root_path) { - size_t search_start = cur_pos + 1; - cur_pos = keeper_path.find('/', search_start); - if (search_start == cur_pos) - throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path '{}' doesn't exist. Please create it or set 'create_missing_root_path' to true'", + keeper_path_); + } + else + { + LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); - auto path = keeper_path.substr(0, cur_pos); - auto status = getClient()->tryCreate(path, "", zkutil::CreateMode::Persistent); - if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(status, path); - } while (cur_pos != std::string_view::npos); + size_t cur_pos = 0; + do + { + size_t search_start = cur_pos + 1; + cur_pos = keeper_path.find('/', search_start); + if (search_start == cur_pos) + throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + + auto path = keeper_path.substr(0, cur_pos); + auto status = client->tryCreate(path, "", zkutil::CreateMode::Persistent); + if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(status, path); + } while (cur_pos != std::string_view::npos); + } } } @@ -285,7 +310,10 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { if (zookeeper_client->expired()) + { zookeeper_client = zookeeper_client->startNewSession(); + zookeeper_client->sync("/"); + } return zookeeper_client; } @@ -374,13 +402,29 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 1) + if (engine_args.empty() || engine_args.size() > 3) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1 argument: " - "keeper_path, path in the Keeper where the values will be stored"); + "Storage KeeperMap requires 1-4 arguments:\n" + "keeper_path: path in the Keeper where the values will be stored (required)\n" + "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" + "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" + "create_missing_root_path: true if the root path should be created if it's missing (default: 1)", + default_host); - auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + + std::string hosts = "default"; + if (engine_args.size() > 1) + hosts = checkAndGetLiteralArgument(engine_args[1], "hosts"); + + [[maybe_unused]] size_t keys_limit = 0; + if (engine_args.size() > 2) + keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); + + bool create_missing_root_path = true; + if (engine_args.size() > 3) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[3], "create_missing_root_path"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -394,7 +438,8 @@ StoragePtr create(const StorageFactory::Arguments & args) if (primary_key_names.size() != 1) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - return std::make_shared(args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path); + return std::make_shared( + args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index bde76e958ba..6e2d6298559 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -23,7 +23,9 @@ public: const StorageID & table_id, const StorageInMemoryMetadata & metadata, std::string_view primary_key_, - std::string_view keeper_path_); + std::string_view keeper_path_, + const std::string & hosts, + bool create_missing_root_path); Pipe read( const Names & column_names, From 0651af8ac67826a6cbcfc347fa07243c8b4ff2a1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 14:02:14 +0000 Subject: [PATCH 057/582] Define error codes --- src/Storages/KVStorageUtils.cpp | 5 +++++ src/Storages/StorageKeeperMap.cpp | 1 + 2 files changed, 6 insertions(+) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 91b7e7feda6..c16ec561291 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { // returns keys may be filter by condition diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8e297e63273..6f0a9c01424 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int KEEPER_EXCEPTION; + extern const int LOGICAL_ERROR; } namespace From bc5eda1bc785686352d015615db9904f82089cc9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 8 Aug 2022 09:43:29 +0000 Subject: [PATCH 058/582] Key limit --- src/Storages/StorageKeeperMap.cpp | 202 ++++++++++++++++++++++++++---- src/Storages/StorageKeeperMap.h | 9 +- 2 files changed, 188 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 6f0a9c01424..1fb0279b404 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -24,6 +24,7 @@ #include #include +#include "Common/ZooKeeper/ZooKeeper.h" #include #include #include @@ -70,6 +71,65 @@ std::string base64Decode(const std::string & encoded) constexpr std::string_view default_host = "default"; +std::string_view getBaseName(const std::string_view path) +{ + auto last_slash = path.find_last_of('/'); + if (last_slash == std::string_view::npos) + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); + + return path.substr(last_slash + 1); +} + +struct ZooKeeperLock +{ + explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_) + : lock_path(std::move(lock_path_)), client(std::move(client_)) + { + lock(); + } + + ~ZooKeeperLock() + { + if (locked) + unlock(); + } + + void lock() + { + assert(!locked); + sequence_path = client->create(std::filesystem::path(lock_path) / "lock-", "", zkutil::CreateMode::EphemeralSequential); + auto node_name = getBaseName(sequence_path); + + while (true) + { + auto children = client->getChildren(lock_path); + assert(!children.empty()); + ::sort(children.begin(), children.end()); + + auto node_it = std::find(children.begin(), children.end(), node_name); + if (node_it == children.begin()) + { + locked = true; + return; + } + + client->waitForDisappear(*(node_it - 1)); + } + } + + void unlock() + { + assert(locked); + client->remove(sequence_path); + } + +private: + std::string lock_path; + std::string sequence_path; + zkutil::ZooKeeperPtr client; + bool locked{false}; +}; + } class StorageKeeperMapSink : public SinkToStorage @@ -117,16 +177,58 @@ public: void onFinish() override { auto & zookeeper = storage.getClient(); - Coordination::Requests requests; - for (const auto & [key, value] : new_values) - { - auto path = storage.fullPathForKey(key); - if (zookeeper->exists(path)) - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - else - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + auto keys_limit = storage.keysLimit(); + + Coordination::Requests requests; + + if (!keys_limit) + { + for (const auto & [key, value] : new_values) + { + auto path = storage.fullPathForKey(key); + + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + } } + else + { + ZooKeeperLock lock(storage.lockPath(), zookeeper); + + auto children = zookeeper->getChildren(storage.rootKeeperPath()); + std::unordered_set children_set(children.begin(), children.end()); + + size_t created_nodes = 0; + for (const auto & [key, value] : new_values) + { + auto path = storage.fullPathForKey(key); + + if (children_set.contains(key)) + { + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + } + else + { + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + ++created_nodes; + } + } + + size_t keys_num_after_insert = children.size() - 1 + created_nodes; + if (keys_limit && keys_num_after_insert > keys_limit) + { + throw Exception( + DB::ErrorCodes::BAD_ARGUMENTS, + "Cannot insert values. {} key would be created setting the total keys number to {} exceeding the limit of {}", + created_nodes, + keys_num_after_insert, + keys_limit); + } + } + zookeeper->multi(requests); } @@ -191,13 +293,13 @@ public: namespace { -zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) -{ - if (hosts == default_host) - return context->getZooKeeper()->startNewSession(); + zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) + { + if (hosts == default_host) + return context->getZooKeeper()->startNewSession(); - return std::make_shared(hosts); -} + return std::make_shared(hosts); + } } @@ -208,7 +310,8 @@ StorageKeeperMap::StorageKeeperMap( std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, - bool create_missing_root_path) + bool create_missing_root_path, + size_t keys_limit_) : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) { setInMemoryMetadata(metadata); @@ -219,6 +322,7 @@ StorageKeeperMap::StorageKeeperMap( throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); auto client = getClient(); + if (keeper_path != "/" && !client->exists(keeper_path)) { if (!create_missing_root_path) @@ -241,12 +345,46 @@ StorageKeeperMap::StorageKeeperMap( throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); auto path = keeper_path.substr(0, cur_pos); - auto status = client->tryCreate(path, "", zkutil::CreateMode::Persistent); - if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(status, path); + client->createIfNotExists(path, ""); } while (cur_pos != std::string_view::npos); } } + + // create metadata nodes + std::filesystem::path root_path{keeper_path}; + + auto metadata_path_fs = root_path / "__ch_metadata"; + metadata_path = metadata_path_fs; + client->createIfNotExists(metadata_path, ""); + + lock_path = metadata_path_fs / "lock"; + client->createIfNotExists(lock_path, ""); + + auto keys_limit_path = metadata_path_fs / "keys_limit"; + auto status = client->tryCreate(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); + if (status == Coordination::Error::ZNODEEXISTS) + { + auto data = client->get(keys_limit_path, nullptr, nullptr); + UInt64 stored_keys_limit = parse(data); + if (stored_keys_limit != keys_limit_) + { + keys_limit = stored_keys_limit; + LOG_WARNING( + &Poco::Logger::get("StorageKeeperMap"), + "Keys limit is already set for {} to {}. Going to use already set value", + keeper_path, + stored_keys_limit); + } + } + else if (status == Coordination::Error::ZOK) + { + keys_limit = keys_limit_; + } + else + { + throw zkutil::KeeperException(status, keys_limit_path); + } + LOG_INFO(&Poco::Logger::get("LOGGER"), "Keys limit set to {}", keys_limit); } @@ -254,7 +392,7 @@ Pipe StorageKeeperMap::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr /*context*/, + ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) @@ -266,7 +404,7 @@ Pipe StorageKeeperMap::read( Block sample_block = storage_snapshot->metadata->getSampleBlock(); auto primary_key_type = sample_block.getByName(primary_key).type; - std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info); + std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context); const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { @@ -329,6 +467,16 @@ std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const return fmt::format("{}/{}", keeper_path, key); } +const std::string & StorageKeeperMap::lockPath() const +{ + return lock_path; +} + +UInt64 StorageKeeperMap::keysLimit() const +{ + return keys_limit; +} + Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const { if (keys.size() != 1) @@ -361,7 +509,14 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (const auto & key : keys) { - values.emplace_back(client->asyncTryGet(fullPathForKey(key))); + const auto full_path = fullPathForKey(key); + if (full_path == metadata_path) + { + values.emplace_back(); + continue; + } + + values.emplace_back(client->asyncTryGet(full_path)); } auto wait_until = std::chrono::system_clock::now() + std::chrono::milliseconds(Coordination::DEFAULT_OPERATION_TIMEOUT_MS); @@ -369,6 +524,9 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (size_t i = 0; i < keys.size(); ++i) { auto & value = values[i]; + if (!value.valid()) + continue; + if (value.wait_until(wait_until) != std::future_status::ready) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fetch values: timeout"); @@ -440,7 +598,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path); + args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 6e2d6298559..4a739311f2a 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -25,7 +25,8 @@ public: std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, - bool create_missing_root_path); + bool create_missing_root_path, + size_t keys_limit); Pipe read( const Names & column_names, @@ -56,9 +57,15 @@ public: const std::string & rootKeeperPath() const; std::string fullPathForKey(std::string_view key) const; + const std::string & lockPath() const; + UInt64 keysLimit() const; + private: std::string keeper_path; std::string primary_key; + std::string metadata_path; + std::string lock_path; + UInt64 keys_limit{0}; mutable zkutil::ZooKeeperPtr zookeeper_client; }; From 617ef00908f4362e9afd7ebcf072b2d27e3350b5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 8 Aug 2022 14:09:37 +0000 Subject: [PATCH 059/582] Small fixes --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Storages/KVStorageUtils.h | 2 ++ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 1 - src/Storages/StorageKeeperMap.cpp | 19 +++++++++---------- src/Storages/StorageKeeperMap.h | 2 +- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ea11a1b23ac..7a00bbf524c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -720,7 +720,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } /// We can have queries like "CREATE TABLE
ENGINE=" if /// supports schema inference (will determine table structure in it's constructor). - else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name) && create.storage->engine->name != "KeeperMap") // NOLINT + else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) // NOLINT throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); /// Even if query has list of columns, canonicalize it (unfold Nested columns). diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index ca6aecd1374..3807e5f084b 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -29,6 +30,7 @@ void fillColumns(const K & key, const V & value, size_t key_pos, const Block & h for (size_t i = 0; i < header.columns(); ++i) { const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); + LOG_INFO(&Poco::Logger::get("LOGGER"), "Reading coluimn {} of type {}", i, columns[i]->getDataType()); serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); } } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 169b6c1e1bc..62c9a0eeae7 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1fb0279b404..081861653ae 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -24,7 +24,7 @@ #include #include -#include "Common/ZooKeeper/ZooKeeper.h" +#include #include #include #include @@ -293,13 +293,13 @@ public: namespace { - zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) - { - if (hosts == default_host) - return context->getZooKeeper()->startNewSession(); +zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) +{ + if (hosts == default_host) + return context->getZooKeeper()->startNewSession(); - return std::make_shared(hosts); - } + return std::make_shared(hosts); +} } @@ -384,7 +384,6 @@ StorageKeeperMap::StorageKeeperMap( { throw zkutil::KeeperException(status, keys_limit_path); } - LOG_INFO(&Poco::Logger::get("LOGGER"), "Keys limit set to {}", keys_limit); } @@ -561,7 +560,7 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 3) + if (engine_args.empty() || engine_args.size() > 4) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-4 arguments:\n" @@ -577,7 +576,7 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.size() > 1) hosts = checkAndGetLiteralArgument(engine_args[1], "hosts"); - [[maybe_unused]] size_t keys_limit = 0; + size_t keys_limit = 0; if (engine_args.size() > 2) keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 4a739311f2a..fdbda1cde34 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -5,7 +5,7 @@ #include #include #include -#include "Common/PODArray_fwd.h" +#include #include #include From 4023d4a37af0f739f88b7eec3d7e691cb990b1e8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Aug 2022 07:24:56 +0000 Subject: [PATCH 060/582] Add support for drop/truncate and tests --- src/Common/ZooKeeper/ZooKeeper.cpp | 12 +-- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Storages/KVStorageUtils.h | 1 - src/Storages/StorageKeeperMap.cpp | 86 ++++++++++++------- src/Storages/StorageKeeperMap.h | 8 +- .../0_stateless/02381_keeper_map.reference | 6 ++ .../queries/0_stateless/02381_keeper_map.sql | 42 +++++++++ 7 files changed, 114 insertions(+), 43 deletions(-) create mode 100644 tests/queries/0_stateless/02381_keeper_map.reference create mode 100644 tests/queries/0_stateless/02381_keeper_map.sql diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 96abf3b543a..74945e3b50c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -737,7 +737,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & } } -bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const String & keep_child_node) +bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const std::string_view keep_child_node) { Strings children; if (tryGetChildren(path, children) != Coordination::Error::ZOK) @@ -754,13 +754,13 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); - /// Will try to avoid recursive getChildren calls if child_path probably has no children. - /// It may be extremely slow when path contain a lot of leaf children. - if (!probably_flat) - tryRemoveChildrenRecursive(child_path); - if (likely(keep_child_node.empty() || keep_child_node != children.back())) { + /// Will try to avoid recursive getChildren calls if child_path probably has no children. + /// It may be extremely slow when path contain a lot of leaf children. + if (!probably_flat) + tryRemoveChildrenRecursive(child_path); + batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c9b5dc69499..416fc78d814 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -238,7 +238,7 @@ public: /// If probably_flat is true, this method will optimistically try to remove children non-recursive /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child. /// Returns true if no kind of fallback happened. - bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, const String & keep_child_node = {}); + bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, std::string_view keep_child_node = {}); /// Remove all children nodes (non recursive). void removeChildren(const std::string & path); diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index 3807e5f084b..e3216164869 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -30,7 +30,6 @@ void fillColumns(const K & key, const V & value, size_t key_pos, const Block & h for (size_t i = 0; i < header.columns(); ++i) { const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); - LOG_INFO(&Poco::Logger::get("LOGGER"), "Reading coluimn {} of type {}", i, columns[i]->getDataType()); serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); } } diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 081861653ae..3096ea0ba72 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -82,10 +82,11 @@ std::string_view getBaseName(const std::string_view path) struct ZooKeeperLock { - explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_) + explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_, bool defer_lock = false) : lock_path(std::move(lock_path_)), client(std::move(client_)) { - lock(); + if (!defer_lock) + lock(); } ~ZooKeeperLock() @@ -120,7 +121,7 @@ struct ZooKeeperLock void unlock() { assert(locked); - client->remove(sequence_path); + client->tryRemove(sequence_path); } private: @@ -307,11 +308,13 @@ StorageKeeperMap::StorageKeeperMap( ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata, + bool attach, std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, bool create_missing_root_path, - size_t keys_limit_) + size_t keys_limit_, + bool remove_existing_data) : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) { setInMemoryMetadata(metadata); @@ -323,6 +326,12 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); + if (attach) + { + // validate all metadata nodes are present + return; + } + if (keeper_path != "/" && !client->exists(keeper_path)) { if (!create_missing_root_path) @@ -350,40 +359,32 @@ StorageKeeperMap::StorageKeeperMap( } } + Coordination::Stat stats; + auto exists = client->exists(keeper_path, &stats); + if (!exists) + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", keeper_path); + + if (stats.numChildren != 0) + { + if (!remove_existing_data) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes", keeper_path); + + LOG_INFO(&Poco::Logger::get("StorageKeepermap"), "Removing children for {} because remove_existing_data was set to true.", keeper_path); + client->removeChildrenRecursive(keeper_path); + } + // create metadata nodes std::filesystem::path root_path{keeper_path}; auto metadata_path_fs = root_path / "__ch_metadata"; metadata_path = metadata_path_fs; - client->createIfNotExists(metadata_path, ""); + client->create(metadata_path, "", zkutil::CreateMode::Persistent); lock_path = metadata_path_fs / "lock"; - client->createIfNotExists(lock_path, ""); + client->create(lock_path, "", zkutil::CreateMode::Persistent); auto keys_limit_path = metadata_path_fs / "keys_limit"; - auto status = client->tryCreate(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); - if (status == Coordination::Error::ZNODEEXISTS) - { - auto data = client->get(keys_limit_path, nullptr, nullptr); - UInt64 stored_keys_limit = parse(data); - if (stored_keys_limit != keys_limit_) - { - keys_limit = stored_keys_limit; - LOG_WARNING( - &Poco::Logger::get("StorageKeeperMap"), - "Keys limit is already set for {} to {}. Going to use already set value", - keeper_path, - stored_keys_limit); - } - } - else if (status == Coordination::Error::ZOK) - { - keys_limit = keys_limit_; - } - else - { - throw zkutil::KeeperException(status, keys_limit_path); - } + client->create(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); } @@ -445,6 +446,20 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot); } +void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) +{ + auto client = getClient(); + + ZooKeeperLock keeper_lock(lockPath(), client); + client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ true, getBaseName(metadata_path)); +} + +void StorageKeeperMap::drop() +{ + auto client = getClient(); + client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ false); +} + zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { if (zookeeper_client->expired()) @@ -563,11 +578,12 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.empty() || engine_args.size() > 4) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-4 arguments:\n" + "Storage KeeperMap requires 1-5 arguments:\n" "keeper_path: path in the Keeper where the values will be stored (required)\n" - "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" - "create_missing_root_path: true if the root path should be created if it's missing (default: 1)", + "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" + "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", + "remove_existing_data: true if children inside 'keeper_path' should be deleted, otherwise throw exception (default: 0)", default_host); auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); @@ -584,6 +600,10 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.size() > 3) create_missing_root_path = checkAndGetLiteralArgument(engine_args[3], "create_missing_root_path"); + bool remove_existing_data = false; + if (engine_args.size() > 4) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[4], "create_missing_root_path"); + StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); @@ -597,7 +617,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index fdbda1cde34..e1ddc304c68 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -17,16 +17,17 @@ namespace DB class StorageKeeperMap final : public IKeyValueStorage { public: - // TODO(antonio2368): add setting to control creating if keeper_path doesn't exist StorageKeeperMap( ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata, + bool attach, std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, bool create_missing_root_path, - size_t keys_limit); + size_t keys_limit, + bool remove_existing_data); Pipe read( const Names & column_names, @@ -39,6 +40,9 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) override; + void drop() override; + std::string getName() const override { return "KeeperMap"; } Names getPrimaryKey() const override { return {primary_key}; } diff --git a/tests/queries/0_stateless/02381_keeper_map.reference b/tests/queries/0_stateless/02381_keeper_map.reference new file mode 100644 index 00000000000..eea8dd975e8 --- /dev/null +++ b/tests/queries/0_stateless/02381_keeper_map.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 1 1 1 1 +1 diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql new file mode 100644 index 00000000000..24048a67cfd --- /dev/null +++ b/tests/queries/0_stateless/02381_keeper_map.sql @@ -0,0 +1,42 @@ +-- Tags: no-ordinary-database, no-fasttest +-- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database +-- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default + +SET database_atomic_wait_for_drop_and_detach_synchronously = 1; + +DROP TABLE IF EXISTS 02381_test; + +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02381_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +DROP TABLE IF EXISTS 02381_test; +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +INSERT INTO 02381_test SELECT '1_1', number FROM numbers(10000); +SELECT COUNT(1) == 1 FROM 02381_test; + +INSERT INTO 02381_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT COUNT(1) == 10000 FROM 02381_test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02381_test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM 02381_test WHERE key IN ('1_1', '99_1', '900_1'); + +DROP TABLE IF EXISTS 02381_test; +DROP TABLE IF EXISTS 02381_test_memory; + +CREATE TABLE 02381_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); +CREATE TABLE 02381_test_memory AS 02381_test Engine = Memory; + +INSERT INTO 02381_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + +INSERT INTO 02381_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + + +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test_memory) B USING a ORDER BY a; + +TRUNCATE TABLE 02381_test; +SELECT 0 == COUNT(1) FROM 02381_test; + +DROP TABLE IF EXISTS 02381_test; +DROP TABLE IF EXISTS 02381_test_memory; From e19ecd95d6ac8b4f5cdf9cc0710c59b6140aced1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Aug 2022 08:52:36 +0000 Subject: [PATCH 061/582] Add basic support for attach --- src/Storages/StorageKeeperMap.cpp | 83 +++++++++++++++++-------------- src/Storages/StorageKeeperMap.h | 6 ++- 2 files changed, 50 insertions(+), 39 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 3096ea0ba72..f90e6d4e366 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -310,81 +310,90 @@ StorageKeeperMap::StorageKeeperMap( const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view keeper_path_, + std::string_view root_path_, const std::string & hosts, bool create_missing_root_path, size_t keys_limit_, bool remove_existing_data) - : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) + : IKeyValueStorage(table_id), root_path(root_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)), log(&Poco::Logger::get("StorageKeeperMap")) { setInMemoryMetadata(metadata); - if (keeper_path.empty()) - throw Exception("keeper_path should not be empty", ErrorCodes::BAD_ARGUMENTS); - if (!keeper_path.starts_with('/')) - throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); + if (root_path.empty()) + throw Exception("root_path should not be empty", ErrorCodes::BAD_ARGUMENTS); + if (!root_path.starts_with('/')) + throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); auto client = getClient(); + std::filesystem::path root_path_fs{root_path}; + auto metadata_path_fs = root_path_fs / "__ch_metadata"; + metadata_path = metadata_path_fs; + lock_path = metadata_path_fs / "lock"; + auto keys_limit_path = metadata_path_fs / "keys_limit"; + if (attach) { // validate all metadata nodes are present + Coordination::Requests requests; + + requests.push_back(zkutil::makeCheckRequest(root_path, -1)); + requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); + requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); + requests.push_back(zkutil::makeCheckRequest(keys_limit_path, -1)); + + client->multi(requests); return; } - if (keeper_path != "/" && !client->exists(keeper_path)) + if (root_path != "/" && !client->exists(root_path)) { if (!create_missing_root_path) { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Path '{}' doesn't exist. Please create it or set 'create_missing_root_path' to true'", - keeper_path_); + root_path_); } else { - LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); + LOG_TRACE(log, "Creating root path {}", root_path); size_t cur_pos = 0; do { size_t search_start = cur_pos + 1; - cur_pos = keeper_path.find('/', search_start); + cur_pos = root_path.find('/', search_start); if (search_start == cur_pos) - throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + throw Exception("root_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); - auto path = keeper_path.substr(0, cur_pos); + auto path = root_path.substr(0, cur_pos); client->createIfNotExists(path, ""); } while (cur_pos != std::string_view::npos); } } Coordination::Stat stats; - auto exists = client->exists(keeper_path, &stats); + auto exists = client->exists(root_path, &stats); if (!exists) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", keeper_path); + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); if (stats.numChildren != 0) { if (!remove_existing_data) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes", keeper_path); + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes. Set remove_existing_data to 1 to clear children nodes", root_path); - LOG_INFO(&Poco::Logger::get("StorageKeepermap"), "Removing children for {} because remove_existing_data was set to true.", keeper_path); - client->removeChildrenRecursive(keeper_path); + LOG_INFO(log, "Removing children for {} because remove_existing_data was set to true.", root_path); + client->removeChildrenRecursive(root_path); } // create metadata nodes - std::filesystem::path root_path{keeper_path}; + Coordination::Requests create_requests; + create_requests.push_back(zkutil::makeCreateRequest(metadata_path, "", zkutil::CreateMode::Persistent)); + create_requests.push_back(zkutil::makeCreateRequest(lock_path, "", zkutil::CreateMode::Persistent)); + create_requests.push_back(zkutil::makeCreateRequest(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent)); - auto metadata_path_fs = root_path / "__ch_metadata"; - metadata_path = metadata_path_fs; - client->create(metadata_path, "", zkutil::CreateMode::Persistent); - - lock_path = metadata_path_fs / "lock"; - client->create(lock_path, "", zkutil::CreateMode::Persistent); - - auto keys_limit_path = metadata_path_fs / "keys_limit"; - client->create(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); + client->multi(create_requests); } @@ -436,7 +445,7 @@ Pipe StorageKeeperMap::read( auto & client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(keeper_path))); + return process_keys(std::make_shared>(client->getChildren(root_path))); return process_keys(std::move(filtered_keys)); } @@ -451,13 +460,13 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr & , Con auto client = getClient(); ZooKeeperLock keeper_lock(lockPath(), client); - client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ true, getBaseName(metadata_path)); + client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ true, getBaseName(metadata_path)); } void StorageKeeperMap::drop() { auto client = getClient(); - client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ false); + client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ false); } zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const @@ -473,12 +482,12 @@ zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const const std::string & StorageKeeperMap::rootKeeperPath() const { - return keeper_path; + return root_path; } std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const { - return fmt::format("{}/{}", keeper_path, key); + return fmt::format("{}/{}", root_path, key); } const std::string & StorageKeeperMap::lockPath() const @@ -579,14 +588,14 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-5 arguments:\n" - "keeper_path: path in the Keeper where the values will be stored (required)\n" + "root_path: path in the Keeper where the values will be stored (required)\n" "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "remove_existing_data: true if children inside 'keeper_path' should be deleted, otherwise throw exception (default: 0)", + "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", default_host); - auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); std::string hosts = "default"; if (engine_args.size() > 1) @@ -602,7 +611,7 @@ StoragePtr create(const StorageFactory::Arguments & args) bool remove_existing_data = false; if (engine_args.size() > 4) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[4], "create_missing_root_path"); + remove_existing_data = checkAndGetLiteralArgument(engine_args[4], "remove_existing_data"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -617,7 +626,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index e1ddc304c68..fd243206721 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -23,7 +23,7 @@ public: const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view keeper_path_, + std::string_view root_path_, const std::string & hosts, bool create_missing_root_path, size_t keys_limit, @@ -65,13 +65,15 @@ public: UInt64 keysLimit() const; private: - std::string keeper_path; + std::string root_path; std::string primary_key; std::string metadata_path; std::string lock_path; UInt64 keys_limit{0}; mutable zkutil::ZooKeeperPtr zookeeper_client; + + Poco::Logger * log; }; } From 18790539c8f162751be779a66aa1320b02a345d1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Aug 2022 08:52:47 +0000 Subject: [PATCH 062/582] Update test --- tests/queries/0_stateless/02381_keeper_map.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql index 24048a67cfd..9a4f0b367f7 100644 --- a/tests/queries/0_stateless/02381_keeper_map.sql +++ b/tests/queries/0_stateless/02381_keeper_map.sql @@ -1,6 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest --- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database --- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default +-- Tags: no-fasttest SET database_atomic_wait_for_drop_and_detach_synchronously = 1; From 4c7c07ee7ab324d86710fc7d5d3cbb647b18e49d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 Aug 2022 13:15:56 +0200 Subject: [PATCH 063/582] Fix fast test --- .../0_stateless/01798_uniq_theta_union_intersect_not.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql b/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql index cc414b06403..ccaf6ca6c6c 100644 --- a/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql +++ b/tests/queries/0_stateless/01798_uniq_theta_union_intersect_not.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SELECT 'uniqTheta union test'; select finalizeAggregation(uniqThetaUnion(a, b)), finalizeAggregation(a), finalizeAggregation(b) from (select arrayReduce('uniqThetaState',[]) as a, arrayReduce('uniqThetaState',[]) as b ); From 456291f33519a0a545f7fbcfb1e6e2a1f7292fa9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 Aug 2022 13:08:56 +0200 Subject: [PATCH 064/582] Fix style --- src/Functions/UniqTheta/FunctionsUniqTheta.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.h b/src/Functions/UniqTheta/FunctionsUniqTheta.h index bd9ad11b9b3..78eb1818647 100644 --- a/src/Functions/UniqTheta/FunctionsUniqTheta.h +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.h @@ -30,7 +30,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int LOGICAL_ERROR; } struct UniqThetaIntersectImpl From 1b8ca90fccba6ea92711c2e2f46180dc875c4f9a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 11 Aug 2022 11:32:32 +0000 Subject: [PATCH 065/582] Add schema inference to clickhouse-obfuscator --- programs/obfuscator/Obfuscator.cpp | 77 ++++++++++++++++++++++-------- 1 file changed, 56 insertions(+), 21 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 6891d2113a5..2607d9d0902 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -1239,7 +1240,6 @@ try if (options.count("help") || !options.count("seed") - || !options.count("structure") || !options.count("input-format") || !options.count("output-format")) { @@ -1259,7 +1259,11 @@ try UInt64 seed = sipHash64(options["seed"].as()); - std::string structure = options["structure"].as(); + std::string structure; + + if (options.count("structure")) + structure = options["structure"].as(); + std::string input_format = options["input-format"].as(); std::string output_format = options["output-format"].as(); @@ -1287,32 +1291,63 @@ try markov_model_params.determinator_sliding_window_size = options["determinator-sliding-window-size"].as(); /// Create the header block - std::vector structure_vals; - boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on); - - if (structure_vals.size() % 2 != 0) - throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR); - - Block header; - const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); - - for (size_t i = 0, size = structure_vals.size(); i < size; i += 2) - { - ColumnWithTypeAndName column; - column.name = structure_vals[i]; - column.type = data_type_factory.get(structure_vals[i + 1]); - column.column = column.type->createColumn(); - header.insert(std::move(column)); - } - SharedContextHolder shared_context = Context::createShared(); auto context = Context::createGlobal(shared_context.get()); context->makeGlobalContext(); + Block header; + + if (structure.empty()) + { + std::unique_ptr read_buffer_from_fd; + std::unique_ptr peekable_read_buffer_from_fd; + + ReadBufferIterator read_buffer_iterator = [&]() + { + read_buffer_from_fd = std::make_unique(STDIN_FILENO); + auto read_buf = std::make_unique(*read_buffer_from_fd); + read_buf->setCheckpoint(); + return read_buf; + }; + + auto context_const = WithContext(context).getContext(); + + auto schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const, peekable_read_buffer_from_fd); + auto schema_columns_info = schema_columns.getOrdinary(); + + for (auto & info : schema_columns_info) + { + ColumnWithTypeAndName column; + column.name = info.name; + column.type = info.type; + column.column = column.type->createColumn(); + header.insert(std::move(column)); + } + } + else + { + std::vector structure_vals; + boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on); + + if (structure_vals.size() % 2 != 0) + throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR); + + const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); + + for (size_t i = 0, size = structure_vals.size(); i < size; i += 2) + { + ColumnWithTypeAndName column; + column.name = structure_vals[i]; + column.type = data_type_factory.get(structure_vals[i + 1]); + column.column = column.type->createColumn(); + header.insert(std::move(column)); + } + } + ReadBufferFromFileDescriptor file_in(STDIN_FILENO); WriteBufferFromFileDescriptor file_out(STDOUT_FILENO); - if (load_from_file.empty()) + if (load_from_file.empty() || structure.empty()) { /// stdin must be seekable auto res = lseek(file_in.getFD(), 0, SEEK_SET); From c108f4bf0ad2b97a2d4716e2cb96cf544bd45a8c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 11 Aug 2022 11:56:30 +0000 Subject: [PATCH 066/582] Add test --- .../00175_obfuscator_schema_inference.sh | 24 +++++++++++++++++++ ...0176_obfuscator_schema_inference.reference | 4 ++++ 2 files changed, 28 insertions(+) create mode 100755 tests/queries/1_stateful/00175_obfuscator_schema_inference.sh create mode 100644 tests/queries/1_stateful/00176_obfuscator_schema_inference.reference diff --git a/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh b/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh new file mode 100755 index 00000000000..d8ee72aa1c0 --- /dev/null +++ b/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --max_threads 1 --query="SELECT URL, Title, SearchPhrase FROM test.hits LIMIT 1000" > "${CLICKHOUSE_TMP}"/data.tsv + +# Test obfuscator without saving the model +$CLICKHOUSE_OBFUSCATOR --input-format TSV --output-format TSV --seed hello --limit 2500 < "${CLICKHOUSE_TMP}"/data.tsv > "${CLICKHOUSE_TMP}"/data2500.tsv 2>/dev/null + +# Test obfuscator with saving the model +$CLICKHOUSE_OBFUSCATOR --input-format TSV --output-format TSV --seed hello --limit 0 --save "${CLICKHOUSE_TMP}"/model.bin < "${CLICKHOUSE_TMP}"/data.tsv 2>/dev/null +wc -c < "${CLICKHOUSE_TMP}"/model.bin +$CLICKHOUSE_OBFUSCATOR --input-format TSV --output-format TSV --seed hello --limit 2500 --load "${CLICKHOUSE_TMP}"/model.bin < "${CLICKHOUSE_TMP}"/data.tsv > "${CLICKHOUSE_TMP}"/data2500_load_from_model.tsv 2>/dev/null +rm "${CLICKHOUSE_TMP}"/model.bin + +$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data.tsv +$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data2500.tsv +$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data2500_load_from_model.tsv + +rm "${CLICKHOUSE_TMP}"/data.tsv +rm "${CLICKHOUSE_TMP}"/data2500.tsv +rm "${CLICKHOUSE_TMP}"/data2500_load_from_model.tsv diff --git a/tests/queries/1_stateful/00176_obfuscator_schema_inference.reference b/tests/queries/1_stateful/00176_obfuscator_schema_inference.reference new file mode 100644 index 00000000000..a704091b1a9 --- /dev/null +++ b/tests/queries/1_stateful/00176_obfuscator_schema_inference.reference @@ -0,0 +1,4 @@ +403499 +1000 320 171 23 +2500 569 354 13 +2500 569 354 13 From 644be9dc4231e72552f9912e68270e474e4fb556 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 11 Aug 2022 13:27:09 +0000 Subject: [PATCH 067/582] Remove unneded ReadBuffers --- programs/obfuscator/Obfuscator.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 2607d9d0902..15a99c28890 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1299,20 +1299,14 @@ try if (structure.empty()) { - std::unique_ptr read_buffer_from_fd; - std::unique_ptr peekable_read_buffer_from_fd; - ReadBufferIterator read_buffer_iterator = [&]() { - read_buffer_from_fd = std::make_unique(STDIN_FILENO); - auto read_buf = std::make_unique(*read_buffer_from_fd); - read_buf->setCheckpoint(); - return read_buf; + return std::make_unique(STDIN_FILENO); }; auto context_const = WithContext(context).getContext(); - auto schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const, peekable_read_buffer_from_fd); + auto schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const); auto schema_columns_info = schema_columns.getOrdinary(); for (auto & info : schema_columns_info) From 9bc9555a5839baa9b1440baa2fddc4388e731c72 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 12 Aug 2022 00:20:05 +0000 Subject: [PATCH 068/582] Fix test --- ...ence.reference => 00175_obfuscator_schema_inference.reference} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/1_stateful/{00176_obfuscator_schema_inference.reference => 00175_obfuscator_schema_inference.reference} (100%) diff --git a/tests/queries/1_stateful/00176_obfuscator_schema_inference.reference b/tests/queries/1_stateful/00175_obfuscator_schema_inference.reference similarity index 100% rename from tests/queries/1_stateful/00176_obfuscator_schema_inference.reference rename to tests/queries/1_stateful/00175_obfuscator_schema_inference.reference From 243bd492facfba55aeece24779cbe253190e80c9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Aug 2022 20:55:11 +0200 Subject: [PATCH 069/582] Trying to fix it --- docker/test/stress/run.sh | 54 +++++++++++++++++++++++++++++---------- 1 file changed, 40 insertions(+), 14 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 144e4f0a827..dc5d626420b 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -27,9 +27,6 @@ export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000 -export EXPORT_S3_STORAGE_POLICIES=1 -export USE_S3_STORAGE_FOR_MERGE_TREE=1 - function install_packages() { @@ -43,6 +40,7 @@ function configure() { # install test configs export USE_DATABASE_ORDINARY=1 + export EXPORT_S3_STORAGE_POLICIES=1 /usr/share/clickhouse-test/config/install.sh # we mount tests folder from repo to /usr/share @@ -185,24 +183,52 @@ configure start -# shellcheck disable=SC2086 # No quotes because I want to split it into words. -#/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS ||: +shellcheck disable=SC2086 # No quotes because I want to split it into words. +/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse -#clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" ||: -#clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" ||: +clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log start -#clickhouse-client --query "SHOW TABLES FROM datasets" ||: -#clickhouse-client --query "SHOW TABLES FROM test" ||: -#clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" ||: -#clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" ||: -#clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" ||: -#clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" ||: -#clickhouse-client --query "SHOW TABLES FROM test" ||: +clickhouse-client --query "SHOW TABLES FROM datasets" +clickhouse-client --query "SHOW TABLES FROM test" + +clickhouse-client --query "CREATE TABLE test.hits_s3 as datasets.hits_v1" +clickhouse-client --query "ALTER TABLE test.hits_s3 MODIFY SETTING storage_policy='s3_cache'" + +clickhouse-client --query "CREATE TABLE test.hits as datasets.hits_v1" +clickhouse-client --query "ALTER TABLE test.hits MODIFY SETTING storage_policy='s3_cache'" + +clickhouse-client --query "CREATE TABLE test.visits as datasets.visits_v1" +clickhouse-client --query "ALTER TABLE test.visits MODIFY SETTING storage_policy='s3_cache'" + +clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1" + +clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" +clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" + +clickhouse-client --query "DROP TABLE datasets.visits_v1" +clickhouse-client --query "DROP TABLE datasets.hits_v1" + +clickhouse-client --query "SHOW TABLES FROM test" + +stop + +# Let's enable S3 storage by default +export USE_S3_STORAGE_FOR_MERGE_TREE=1 +configure + +# But we still need default disk because some tables loaded only into it +sudo cat /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml | sed "s|s3|s3default|" > /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml.tmp +mv /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml.tmp /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml +sudo chown clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml +sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml + +start ./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \ && echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \ From 1ec6627a70df4255a475d32de60e057d3e62aff3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Aug 2022 18:28:17 +0200 Subject: [PATCH 070/582] Fix tables creation --- docker/test/stress/run.sh | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index dc5d626420b..9546cbb8d54 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -197,14 +197,11 @@ start clickhouse-client --query "SHOW TABLES FROM datasets" clickhouse-client --query "SHOW TABLES FROM test" -clickhouse-client --query "CREATE TABLE test.hits_s3 as datasets.hits_v1" -clickhouse-client --query "ALTER TABLE test.hits_s3 MODIFY SETTING storage_policy='s3_cache'" +clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" -clickhouse-client --query "CREATE TABLE test.hits as datasets.hits_v1" -clickhouse-client --query "ALTER TABLE test.hits MODIFY SETTING storage_policy='s3_cache'" +clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" -clickhouse-client --query "CREATE TABLE test.visits as datasets.visits_v1" -clickhouse-client --query "ALTER TABLE test.visits MODIFY SETTING storage_policy='s3_cache'" +clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1" From bca28ba9f8cb25cad2947da535a8fce006185383 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 8 Jul 2022 21:24:33 +0200 Subject: [PATCH 071/582] split prepareBlockAndFill --- src/Interpreters/Aggregator.cpp | 51 +++++++++++++++++++++++---------- src/Interpreters/Aggregator.h | 19 ++++++++++++ 2 files changed, 55 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 6e101005599..e59605890d3 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1951,13 +1951,7 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal( }); } - -template -Block Aggregator::prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - size_t rows, - Filler && filler) const +Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const { MutableColumns key_columns(params.keys_size); MutableColumns aggregate_columns(params.aggregates_size); @@ -1982,7 +1976,7 @@ Block Aggregator::prepareBlockAndFill( /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); - for (auto & pool : data_variants.aggregates_pools) + for (auto & pool : aggregates_pools) column_aggregate_func.addArena(pool); aggregate_columns_data[i] = &column_aggregate_func.getData(); @@ -1997,22 +1991,49 @@ Block Aggregator::prepareBlockAndFill( { /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) - for (auto & pool : data_variants.aggregates_pools) + for (auto & pool : aggregates_pools) column_aggregate_func->addArena(pool); /// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator. - final_aggregate_columns[i]->forEachSubcolumn([&data_variants](auto & subcolumn) - { - if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) - for (auto & pool : data_variants.aggregates_pools) - column_aggregate_func->addArena(pool); - }); + final_aggregate_columns[i]->forEachSubcolumn( + [&aggregates_pools](auto & subcolumn) + { + if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + }); } } } + return { + .key_columns = std::move(key_columns), + .aggregate_columns = std::move(aggregate_columns), + .final_aggregate_columns = std::move(final_aggregate_columns), + .aggregate_columns_data = std::move(aggregate_columns_data), + }; +} + +template +Block Aggregator::prepareBlockAndFill( + AggregatedDataVariants & data_variants, + bool final, + size_t rows, + Filler && filler) const +{ + auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); + return finalizeBlock(std::move(out_cols), final, rows); +} + +Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const +{ + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + Block res_header = getHeader(final); Block res = res_header.cloneEmpty(); for (size_t i = 0; i < params.keys_size; ++i) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3e8b25c1a8c..6e37c5a63d4 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1306,6 +1306,25 @@ private: std::vector key_columns, AggregateColumnsData & aggregate_columns) const; + struct OutputBlockColumns + { + /*OutputBlockColumns(size_t keys_size, size_t aggregates_size) + : key_columns(keys_size) + , aggregate_columns(aggregates_size) + , final_aggregate_columns(aggregates_size) + , aggregate_columns_data(aggregates_size) + { + }*/ + + MutableColumns key_columns; + MutableColumns aggregate_columns; + MutableColumns final_aggregate_columns; + AggregateColumnsData aggregate_columns_data; + }; + + OutputBlockColumns prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const; + Block finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const; + template Block prepareBlockAndFill( AggregatedDataVariants & data_variants, From 4e974661d6033c5fd0a55b5741add4f39458021e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 9 Jul 2022 01:22:12 +0200 Subject: [PATCH 072/582] refactor convertToBlockImpl --- src/Interpreters/Aggregator.cpp | 158 ++++++++++++++++---------------- src/Interpreters/Aggregator.h | 22 +---- 2 files changed, 83 insertions(+), 97 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e59605890d3..4420bdb700c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1578,16 +1578,9 @@ Block Aggregator::convertOneBucketToBlock( bool final, size_t bucket) const { - Block block = prepareBlockAndFill(data_variants, final, method.data.impls[bucket].size(), - [bucket, &method, arena, this] ( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_) - { - convertToBlockImpl(method, method.data.impls[bucket], - key_columns, aggregate_columns, final_aggregate_columns, arena, final_); - }); + Block block = convertToBlockImpl( + method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()) + .front(); block.info.bucket_num = bucket; return block; @@ -1694,25 +1687,16 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const template -void Aggregator::convertToBlockImpl( - Method & method, - Table & data, - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool final) const +BlocksList +Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const { if (data.empty()) - return; + { + auto && out_cols = prepareOutputBlockColumns(aggregates_pools, final, rows); + return {finalizeBlock(std::move(out_cols), final, rows)}; + } - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); + BlocksList res; if (final) { @@ -1720,20 +1704,23 @@ void Aggregator::convertToBlockImpl( if (compiled_aggregate_functions_holder) { static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; - convertToBlockImplFinal(method, data, std::move(raw_key_columns), final_aggregate_columns, arena); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } else #endif { - convertToBlockImplFinal(method, data, std::move(raw_key_columns), final_aggregate_columns, arena); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } } else { - convertToBlockImplNotFinal(method, data, std::move(raw_key_columns), aggregate_columns); + res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); } + /// In order to release memory early. data.clearAndShrink(); + + return res; } @@ -1803,13 +1790,20 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template -void NO_INLINE Aggregator::convertToBlockImplFinal( - Method & method, - Table & data, - std::vector key_columns, - MutableColumns & final_aggregate_columns, - Arena * arena) const +BlocksList NO_INLINE +Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const { + auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, rows); + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + if constexpr (Method::low_cardinality_optimization) { if (data.hasNullKeyData()) @@ -1819,20 +1813,21 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( } } - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); + auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; PaddedPODArray places; places.reserve(data.size()); - data.forEachValue([&](const auto & key, auto & mapped) - { - method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); - places.emplace_back(mapped); + data.forEachValue( + [&](const auto & key, auto & mapped) + { + method.insertKeyIntoColumns(key, raw_key_columns, key_sizes_ref); + places.emplace_back(mapped); - /// Mark the cell as destroyed so it will not be destroyed in destructor. - mapped = nullptr; - }); + /// Mark the cell as destroyed so it will not be destroyed in destructor. + mapped = nullptr; + }); std::exception_ptr exception; size_t aggregate_functions_destroy_index = 0; @@ -1914,41 +1909,53 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( if (exception) std::rethrow_exception(exception); + + return {finalizeBlock(std::move(out_cols), /* final */ true, rows)}; } template -void NO_INLINE Aggregator::convertToBlockImplNotFinal( - Method & method, - Table & data, - std::vector key_columns, - AggregateColumnsData & aggregate_columns) const +BlocksList NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const { + auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + if constexpr (Method::low_cardinality_optimization) { if (data.hasNullKeyData()) { - key_columns[0]->insertDefault(); + raw_key_columns[0]->insertDefault(); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); + aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); data.getNullKeyData() = nullptr; } } - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); + auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - data.forEachValue([&](const auto & key, auto & mapped) - { - method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); + data.forEachValue( + [&](const auto & key, auto & mapped) + { + method.insertKeyIntoColumns(key, raw_key_columns, key_sizes_ref); - /// reserved, so push_back does not throw exceptions - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(mapped + offsets_of_aggregate_states[i]); + /// reserved, so push_back does not throw exceptions + for (size_t i = 0; i < params.aggregates_size; ++i) + out_cols.aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]); - mapped = nullptr; - }); + mapped = nullptr; + }); + + return {finalizeBlock(std::move(out_cols), /* final */ false, rows)}; } Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const @@ -2162,27 +2169,22 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { - size_t rows = data_variants.sizeWithoutOverflowRow(); + // clang-format off + const size_t rows = data_variants.sizeWithoutOverflowRow(); +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + return convertToBlockImpl( \ + *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows).front(); \ + } - auto filler = [&data_variants, this]( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_) + if (false) { - #define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \ - key_columns, aggregate_columns, final_aggregate_columns, data_variants.aggregates_pool, final_); - - if (false) {} // NOLINT - APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) - #undef M - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - }; - - return prepareBlockAndFill(data_variants, final, rows, filler); + } // NOLINT + APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) +#undef M + else + throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 6e37c5a63d4..9dd019da883 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1276,14 +1276,7 @@ private: ManyAggregatedDataVariants & non_empty_data) const; template - void convertToBlockImpl( - Method & method, - Table & data, - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool final) const; + BlocksList convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const; template void insertAggregatesIntoColumns( @@ -1292,19 +1285,10 @@ private: Arena * arena) const; template - void convertToBlockImplFinal( - Method & method, - Table & data, - std::vector key_columns, - MutableColumns & final_aggregate_columns, - Arena * arena) const; + BlocksList convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; template - void convertToBlockImplNotFinal( - Method & method, - Table & data, - std::vector key_columns, - AggregateColumnsData & aggregate_columns) const; + BlocksList convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; struct OutputBlockColumns { From e5e0a24ab3d5c843ba72920c9d1920d16ec49f76 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 11 Jul 2022 23:50:58 +0200 Subject: [PATCH 073/582] return chunks from prepareBlockAndFillWithoutKey --- src/Interpreters/Aggregator.cpp | 16 ++++---- src/Interpreters/Aggregator.h | 2 +- .../AggregatingInOrderTransform.cpp | 4 +- .../Transforms/AggregatingTransform.cpp | 39 ++++++++----------- 4 files changed, 27 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 4420bdb700c..063348f23a9 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2167,15 +2167,15 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va return block; } -Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const +BlocksList Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { // clang-format off const size_t rows = data_variants.sizeWithoutOverflowRow(); -#define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - { \ - return convertToBlockImpl( \ - *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows).front(); \ +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + return convertToBlockImpl( \ + *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } if (false) @@ -2306,7 +2306,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b if (data_variants.type != AggregatedDataVariants::Type::without_key) { if (!data_variants.isTwoLevel()) - blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final)); + blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final).front()); else blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); } @@ -3075,7 +3075,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) block = prepareBlockAndFillWithoutKey(result, final, is_overflows); else - block = prepareBlockAndFillSingleLevel(result, final); + block = prepareBlockAndFillSingleLevel(result, final).front(); /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. if (!final) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 9dd019da883..74cbcee433a 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1332,7 +1332,7 @@ private: std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; - Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; + BlocksList prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const; template diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index ce50ae5eeee..6b8707113ac 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -182,7 +182,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes) { if (group_by_key) - group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false); + group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false).front(); cur_block_bytes += current_memory_usage; finalizeCurrentChunk(std::move(chunk), key_end); return; @@ -293,7 +293,7 @@ void AggregatingInOrderTransform::generate() if (cur_block_size && is_consume_finished) { if (group_by_key) - group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false); + group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false).front(); else params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); variants.invalidate(); diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 7f5896f5e97..a2be680c6c2 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -203,7 +203,7 @@ public: { auto & output = outputs.front(); - if (finished && !has_input) + if (finished && single_level_chunks.empty()) { output.finish(); return Status::Finished; @@ -230,7 +230,7 @@ public: if (!processors.empty()) return Status::ExpandPipeline; - if (has_input) + if (!single_level_chunks.empty()) return preparePushToOutput(); /// Single level case. @@ -244,9 +244,12 @@ public: private: IProcessor::Status preparePushToOutput() { + if (single_level_chunks.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Some ready chunks expected"); + auto & output = outputs.front(); - output.push(std::move(current_chunk)); - has_input = false; + output.push(std::move(single_level_chunks.back())); + single_level_chunks.pop_back(); if (finished) { @@ -268,17 +271,17 @@ private: { auto chunk = input.pull(); auto bucket = getInfoFromChunk(chunk)->bucket_num; - chunks[bucket] = std::move(chunk); + two_level_chunks[bucket] = std::move(chunk); } } if (!shared_data->is_bucket_processed[current_bucket_num]) return Status::NeedData; - if (!chunks[current_bucket_num]) + if (!two_level_chunks[current_bucket_num]) return Status::NeedData; - output.push(std::move(chunks[current_bucket_num])); + output.push(std::move(two_level_chunks[current_bucket_num])); ++current_bucket_num; if (current_bucket_num == NUM_BUCKETS) @@ -298,27 +301,16 @@ private: size_t num_threads; bool is_initialized = false; - bool has_input = false; bool finished = false; - Chunk current_chunk; + Chunks single_level_chunks; UInt32 current_bucket_num = 0; static constexpr Int32 NUM_BUCKETS = 256; - std::array chunks; + std::array two_level_chunks; Processors processors; - void setCurrentChunk(Chunk chunk) - { - if (has_input) - throw Exception("Current chunk was already set in " - "ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR); - - has_input = true; - current_chunk = std::move(chunk); - } - void initialize() { is_initialized = true; @@ -339,7 +331,7 @@ private: auto block = params->aggregator.prepareBlockAndFillWithoutKey( *first, params->final, first->type != AggregatedDataVariants::Type::without_key); - setCurrentChunk(convertToChunk(block)); + single_level_chunks.emplace_back(convertToChunk(block)); } } @@ -364,9 +356,10 @@ private: else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - auto block = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final); + auto blocks = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final); + for (auto & block : blocks) + single_level_chunks.emplace_back(convertToChunk(block)); - setCurrentChunk(convertToChunk(block)); finished = true; } From db0110fd7a1a62312ac7b93f67e4651573905def Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 12 Jul 2022 17:06:20 +0200 Subject: [PATCH 074/582] more accurate crutch --- src/Interpreters/Aggregator.cpp | 36 ++++++++++--------- src/Interpreters/Aggregator.h | 27 +++++++++----- .../AggregatingInOrderTransform.cpp | 6 ++-- .../Transforms/AggregatingTransform.cpp | 2 +- 4 files changed, 42 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 063348f23a9..56f69c1f940 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1578,9 +1578,8 @@ Block Aggregator::convertOneBucketToBlock( bool final, size_t bucket) const { - Block block = convertToBlockImpl( - method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()) - .front(); + Block block = convertToBlockImpl( + method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()); block.info.bucket_num = bucket; return block; @@ -1686,8 +1685,8 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const } -template -BlocksList +template +Aggregator::ConvertToBlockRes Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const { if (data.empty()) @@ -1696,7 +1695,7 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are return {finalizeBlock(std::move(out_cols), final, rows)}; } - BlocksList res; + ConvertToBlockRes res; if (final) { @@ -1704,17 +1703,17 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are if (compiled_aggregate_functions_holder) { static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; - res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } else #endif { - res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } } else { - res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); + res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); } /// In order to release memory early. @@ -1789,8 +1788,8 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu } -template -BlocksList NO_INLINE +template +Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const { auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, rows); @@ -1913,8 +1912,9 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena return {finalizeBlock(std::move(out_cols), /* final */ true, rows)}; } -template -BlocksList NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const +template +Aggregator::ConvertToBlockRes NO_INLINE +Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const { auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; @@ -2167,14 +2167,16 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va return block; } -BlocksList Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const +template +Aggregator::ConvertToBlockRes +Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { // clang-format off const size_t rows = data_variants.sizeWithoutOverflowRow(); #define M(NAME) \ else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ { \ - return convertToBlockImpl( \ + return convertToBlockImpl( \ *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } @@ -2306,7 +2308,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b if (data_variants.type != AggregatedDataVariants::Type::without_key) { if (!data_variants.isTwoLevel()) - blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final).front()); + blocks.splice(blocks.end(), prepareBlockAndFillSingleLevel(data_variants, final)); else blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); } @@ -3075,7 +3077,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) block = prepareBlockAndFillWithoutKey(result, final, is_overflows); else - block = prepareBlockAndFillSingleLevel(result, final).front(); + block = prepareBlockAndFillSingleLevel(result, final); /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. if (!final) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 74cbcee433a..4f06ee67624 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1,8 +1,9 @@ #pragma once -#include -#include #include +#include +#include +#include #include @@ -1275,8 +1276,12 @@ private: void mergeSingleLevelDataImpl( ManyAggregatedDataVariants & non_empty_data) const; - template - BlocksList convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const; + template + using ConvertToBlockRes = std::conditional_t; + + template + ConvertToBlockRes + convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const; template void insertAggregatesIntoColumns( @@ -1284,11 +1289,13 @@ private: MutableColumns & final_aggregate_columns, Arena * arena) const; - template - BlocksList convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; + template + ConvertToBlockRes + convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; - template - BlocksList convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; + template + ConvertToBlockRes + convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; struct OutputBlockColumns { @@ -1332,9 +1339,11 @@ private: std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; - BlocksList prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const; + template + ConvertToBlockRes prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; + template BlocksList prepareBlocksAndFillTwoLevelImpl( AggregatedDataVariants & data_variants, diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 6b8707113ac..9137a50aba5 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -182,7 +182,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes) { if (group_by_key) - group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false).front(); + group_by_block + = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false); cur_block_bytes += current_memory_usage; finalizeCurrentChunk(std::move(chunk), key_end); return; @@ -293,7 +294,8 @@ void AggregatingInOrderTransform::generate() if (cur_block_size && is_consume_finished) { if (group_by_key) - group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false).front(); + group_by_block + = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false); else params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); variants.invalidate(); diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index a2be680c6c2..fb1dd45544d 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -356,7 +356,7 @@ private: else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - auto blocks = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final); + auto blocks = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final); for (auto & block : blocks) single_level_chunks.emplace_back(convertToChunk(block)); From f650b23ee37febab19a53a6208e9485ded902ac1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 12 Jul 2022 22:33:42 +0200 Subject: [PATCH 075/582] generate many blocks --- src/Interpreters/Aggregator.cpp | 247 ++++++++++-------- src/Interpreters/Aggregator.h | 9 +- .../Transforms/AggregatingTransform.cpp | 2 +- 3 files changed, 141 insertions(+), 117 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 56f69c1f940..367da5d1352 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -1789,76 +1790,76 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template -Aggregator::ConvertToBlockRes NO_INLINE -Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const +Aggregator::ConvertToBlockRes + NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const { - auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, rows); - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); - - if constexpr (Method::low_cardinality_optimization) + auto insert_results_into_cols = [this, arena](PaddedPODArray & places, auto & out_cols) { - if (data.hasNullKeyData()) + std::exception_ptr exception; + size_t aggregate_functions_destroy_index = 0; + + try { - key_columns[0]->insertDefault(); - insertAggregatesIntoColumns(data.getNullKeyData(), final_aggregate_columns, arena); - } - } - - auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); - const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - - PaddedPODArray places; - places.reserve(data.size()); - - data.forEachValue( - [&](const auto & key, auto & mapped) - { - method.insertKeyIntoColumns(key, raw_key_columns, key_sizes_ref); - places.emplace_back(mapped); - - /// Mark the cell as destroyed so it will not be destroyed in destructor. - mapped = nullptr; - }); - - std::exception_ptr exception; - size_t aggregate_functions_destroy_index = 0; - - try - { #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - /** For JIT compiled functions we need to resize columns before pass them into compiled code. + if constexpr (use_compiled_functions) + { + /** For JIT compiled functions we need to resize columns before pass them into compiled code. * insert_aggregates_into_columns_function function does not throw exception. */ - std::vector columns_data; + std::vector columns_data; - auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; + auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!is_aggregate_function_compiled[i]) - continue; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!is_aggregate_function_compiled[i]) + continue; - auto & final_aggregate_column = final_aggregate_columns[i]; - final_aggregate_column = final_aggregate_column->cloneResized(places.size()); - columns_data.emplace_back(getColumnData(final_aggregate_column.get())); + auto & final_aggregate_column = out_cols.final_aggregate_columns[i]; + final_aggregate_column = final_aggregate_column->cloneResized(places.size()); + columns_data.emplace_back(getColumnData(final_aggregate_column.get())); + } + + auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; + insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); } - - auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; - insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); - } #endif - for (; aggregate_functions_destroy_index < params.aggregates_size;) + for (; aggregate_functions_destroy_index < params.aggregates_size;) + { + if constexpr (use_compiled_functions) + { + if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) + { + ++aggregate_functions_destroy_index; + continue; + } + } + + auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index]; + size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; + + /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch + * throws exception, it also must destroy all necessary states. + * Then code need to continue to destroy other aggregate function states with next function index. + */ + size_t destroy_index = aggregate_functions_destroy_index; + ++aggregate_functions_destroy_index; + + /// For State AggregateFunction ownership of aggregate place is passed to result column after insert + bool is_state = aggregate_functions[destroy_index]->isState(); + bool destroy_place_after_insert = !is_state; + + aggregate_functions[destroy_index]->insertResultIntoBatch( + 0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert); + } + } + catch (...) + { + exception = std::current_exception(); + } + + for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) { if constexpr (use_compiled_functions) { @@ -1869,47 +1870,76 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena } } - auto & final_aggregate_column = final_aggregate_columns[aggregate_functions_destroy_index]; size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; - - /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch - * throws exception, it also must destroy all necessary states. - * Then code need to continue to destroy other aggregate function states with next function index. - */ - size_t destroy_index = aggregate_functions_destroy_index; - ++aggregate_functions_destroy_index; - - /// For State AggregateFunction ownership of aggregate place is passed to result column after insert - bool is_state = aggregate_functions[destroy_index]->isState(); - bool destroy_place_after_insert = !is_state; - - aggregate_functions[destroy_index]->insertResultIntoBatch(0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert); + aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); } - } - catch (...) - { - exception = std::current_exception(); - } - for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) + if (exception) + std::rethrow_exception(exception); + + return finalizeBlock(std::move(out_cols), /* final */ true, places.size()); + }; + + const size_t max_block_size = DEFAULT_BLOCK_SIZE; + ConvertToBlockRes res; + + std::optional out_cols; + std::optional shuffled_key_sizes; + const Sizes * key_sizes_ptr = nullptr; + PaddedPODArray places; + + auto init_out_cols = [&]() { - if constexpr (use_compiled_functions) + out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, max_block_size); + + if constexpr (Method::low_cardinality_optimization) { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) + if (data.hasNullKeyData()) { - ++aggregate_functions_destroy_index; - continue; + out_cols->key_columns[0]->insertDefault(); + insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena); } } - size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; - aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); + shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); + key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; + + places.reserve(max_block_size); + }; + + data.forEachValue( + [&](const auto & key, auto & mapped) + { + if (!out_cols.has_value()) + init_out_cols(); + + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); + places.emplace_back(mapped); + + /// Mark the cell as destroyed so it will not be destroyed in destructor. + mapped = nullptr; + + if constexpr (!return_single_block) + { + if (places.size() >= max_block_size) + { + res.emplace_back(insert_results_into_cols(places, out_cols.value())); + places.clear(); + out_cols.reset(); + } + } + }); + + if constexpr (return_single_block) + { + return insert_results_into_cols(places, out_cols.value()); + } + else + { + if (!places.empty()) + res.emplace_back(insert_results_into_cols(places, out_cols.value())); + return res; } - - if (exception) - std::rethrow_exception(exception); - - return {finalizeBlock(std::move(out_cols), /* final */ true, rows)}; } template @@ -1917,15 +1947,7 @@ Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const { auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; if constexpr (Method::low_cardinality_optimization) { @@ -1946,7 +1968,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a data.forEachValue( [&](const auto & key, auto & mapped) { - method.insertKeyIntoColumns(key, raw_key_columns, key_sizes_ref); + method.insertKeyIntoColumns(key, out_cols.raw_key_columns, key_sizes_ref); /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) @@ -2013,8 +2035,17 @@ Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & ag } } + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + return { .key_columns = std::move(key_columns), + .raw_key_columns = std::move(raw_key_columns), .aggregate_columns = std::move(aggregate_columns), .final_aggregate_columns = std::move(final_aggregate_columns), .aggregate_columns_data = std::move(aggregate_columns_data), @@ -2029,7 +2060,7 @@ Block Aggregator::prepareBlockAndFill( Filler && filler) const { auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); @@ -2038,7 +2069,7 @@ Block Aggregator::prepareBlockAndFill( Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const { - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; Block res_header = getHeader(final); Block res = res_header.cloneEmpty(); @@ -2171,12 +2202,11 @@ template Aggregator::ConvertToBlockRes Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { - // clang-format off const size_t rows = data_variants.sizeWithoutOverflowRow(); -#define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - { \ - return convertToBlockImpl( \ +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + return convertToBlockImpl( \ *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } @@ -2185,8 +2215,7 @@ Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variant } // NOLINT APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) #undef M - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } @@ -3278,4 +3307,6 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons } +template Aggregator::ConvertToBlockRes +Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 4f06ee67624..a3d7097a7cc 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1299,15 +1299,8 @@ private: struct OutputBlockColumns { - /*OutputBlockColumns(size_t keys_size, size_t aggregates_size) - : key_columns(keys_size) - , aggregate_columns(aggregates_size) - , final_aggregate_columns(aggregates_size) - , aggregate_columns_data(aggregates_size) - { - }*/ - MutableColumns key_columns; + std::vector raw_key_columns; MutableColumns aggregate_columns; MutableColumns final_aggregate_columns; AggregateColumnsData aggregate_columns_data; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index fb1dd45544d..4e55081ca48 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -251,7 +251,7 @@ private: output.push(std::move(single_level_chunks.back())); single_level_chunks.pop_back(); - if (finished) + if (finished && single_level_chunks.empty()) { output.finish(); return Status::Finished; From 63bc894a42ba2e3660dd7a112c7a201b51e28e75 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 13 Jul 2022 12:36:04 +0200 Subject: [PATCH 076/582] more parallelism --- src/Interpreters/Aggregator.cpp | 5 +++-- src/Processors/QueryPlan/AggregatingStep.cpp | 9 ++++----- src/QueryPipeline/QueryPipelineBuilder.h | 2 ++ tests/performance/queries_over_aggregation.xml | 4 ++++ .../02343_aggregation_pipeline.reference | 17 +++++++++++++++++ .../0_stateless/02343_aggregation_pipeline.sql | 2 ++ 6 files changed, 32 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 367da5d1352..5dc0a6e319a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -1579,7 +1578,9 @@ Block Aggregator::convertOneBucketToBlock( bool final, size_t bucket) const { - Block block = convertToBlockImpl( + // Used in ConvertingAggregatedToChunksSource -> ConvertingAggregatedToChunksTransform (expects single chunk for each bucket_id). + constexpr bool return_single_block = true; + Block block = convertToBlockImpl( method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()); block.info.bucket_num = bucket; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index f0374d2419b..3680b28f449 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -376,16 +376,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B }); /// We add the explicit resize here, but not in case of aggregating in order, since AIO don't use two-level hash tables and thus returns only buckets with bucket_number = -1. - pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : pipeline.getNumStreams(), true /* force */); + pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : params.max_threads, true /* force */); aggregating = collector.detachProcessors(0); } else { - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, transform_params); - }); + pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared(header, transform_params); }); + + pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : params.max_threads, false /* force */); aggregating = collector.detachProcessors(0); } diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 100a2e07341..850c4a66615 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -161,6 +161,8 @@ public: return std::max(1, num_threads); } + size_t getMaxThreads() const { return max_threads; } + /// Set upper limit for the recommend number of threads void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } diff --git a/tests/performance/queries_over_aggregation.xml b/tests/performance/queries_over_aggregation.xml index 2a92ea26819..ceaed61c5bb 100644 --- a/tests/performance/queries_over_aggregation.xml +++ b/tests/performance/queries_over_aggregation.xml @@ -1,4 +1,8 @@ + select sipHash64(number) from numbers(1e7) group by number format Null + select * from (select * from numbers(1e7) group by number) group by number format Null + select * from (select * from numbers(1e7) group by number) order by number format Null + select * from (select * from numbers_mt(1e7) group by number) group by number format Null select * from (select * from numbers_mt(1e7) group by number) order by number format Null select * from (select * from numbers_mt(1e7) group by number) group by number format Null settings max_bytes_before_external_group_by = 1 diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.reference b/tests/queries/0_stateless/02343_aggregation_pipeline.reference index 67bd9c414ba..ec9a394d05d 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.reference +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.reference @@ -1,5 +1,22 @@ -- { echoOn } +explain pipeline select * from (select * from numbers(1e8) group by number) group by number; +(Expression) +ExpressionTransform × 16 + (Aggregating) + Resize 16 → 16 + AggregatingTransform × 16 + StrictResize 16 → 16 + (Expression) + ExpressionTransform × 16 + (Aggregating) + Resize 1 → 16 + AggregatingTransform + (Expression) + ExpressionTransform + (ReadFromStorage) + Limit + Numbers 0 → 1 explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number; (Expression) ExpressionTransform × 16 diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.sql b/tests/queries/0_stateless/02343_aggregation_pipeline.sql index d259889b042..f25cef15733 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.sql +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.sql @@ -4,6 +4,8 @@ set optimize_aggregation_in_order = 0; -- { echoOn } +explain pipeline select * from (select * from numbers(1e8) group by number) group by number; + explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number; explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number; From ae5baf290619c3c99e0a8ea9c560eb77c366f5da Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 13 Jul 2022 21:50:40 +0200 Subject: [PATCH 077/582] fix tests --- tests/queries/0_stateless/01091_num_threads.sql | 2 +- .../01524_do_not_merge_across_partitions_select_final.reference | 2 +- .../01524_do_not_merge_across_partitions_select_final.sql | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01091_num_threads.sql b/tests/queries/0_stateless/01091_num_threads.sql index faeceb0e6d6..0d2a66a8c2e 100644 --- a/tests/queries/0_stateless/01091_num_threads.sql +++ b/tests/queries/0_stateless/01091_num_threads.sql @@ -28,7 +28,7 @@ WITH ORDER BY event_time DESC LIMIT 1 ) AS id -SELECT uniqExact(thread_id) +SELECT uniqExact(thread_id) > 2 FROM system.query_thread_log WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id); diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index a3f2106cd5f..540137d4887 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -6,4 +6,4 @@ 2020-01-01 00:00:00 2 1 499999 -5 +18 diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index a5423d1a3ff..23678c1abd9 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS select_final; SET do_not_merge_across_partitions_select_final = 1; -SET max_threads = 0; +SET max_threads = 16; CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t); From 433657e978c292e39bbd329f2a0bf18cf16fc84e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 01:46:18 +0200 Subject: [PATCH 078/582] rm prepareBlockAndFill --- src/Interpreters/Aggregator.cpp | 74 +++++++++++++-------------------- src/Interpreters/Aggregator.h | 7 ---- 2 files changed, 30 insertions(+), 51 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 5dc0a6e319a..5a75cc2bacd 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2053,21 +2053,6 @@ Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & ag }; } -template -Block Aggregator::prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - size_t rows, - Filler && filler) const -{ - auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); - - return finalizeBlock(std::move(out_cols), final, rows); -} - Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const { auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; @@ -2156,39 +2141,34 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const { size_t rows = 1; + auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - auto filler = [&data_variants, this]( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_) + if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row) { - if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row) + AggregatedDataWithoutKey & data = data_variants.without_key; + + if (!data) + throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR); + + if (!final) { - AggregatedDataWithoutKey & data = data_variants.without_key; - - if (!data) - throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR); - - if (!final_) - { - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]); - data = nullptr; - } - else - { - /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'. - insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool); - } - - if (params.overflow_row) - for (size_t i = 0; i < params.keys_size; ++i) - key_columns[i]->insertDefault(); + for (size_t i = 0; i < params.aggregates_size; ++i) + aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]); + data = nullptr; + } + else + { + /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'. + insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool); } - }; - Block block = prepareBlockAndFill(data_variants, final, rows, filler); + if (params.overflow_row) + for (size_t i = 0; i < params.keys_size; ++i) + key_columns[i]->insertDefault(); + } + + Block block = finalizeBlock(std::move(out_cols), final, rows); if (is_overflows) block.info.is_overflows = true; @@ -3105,9 +3085,15 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) Block block; if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) + { block = prepareBlockAndFillWithoutKey(result, final, is_overflows); + } else - block = prepareBlockAndFillSingleLevel(result, final); + { + // Used during memory efficient merging in SortingAggregatedTransform (expects single chunk for each bucket_id). + constexpr bool return_single_block = true; + block = prepareBlockAndFillSingleLevel(result, final); + } /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. if (!final) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index a3d7097a7cc..7021a757df1 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1309,13 +1309,6 @@ private: OutputBlockColumns prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const; Block finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const; - template - Block prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - size_t rows, - Filler && filler) const; - template Block convertOneBucketToBlock( AggregatedDataVariants & data_variants, From 56c09bf8a90ffe03234ccc73494bc6d2e2badd35 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 01:46:41 +0200 Subject: [PATCH 079/582] generate many blocks in convertToBlockImplNotFinal --- src/Interpreters/Aggregator.cpp | 66 +++++++++++++++++++++++++-------- 1 file changed, 51 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 5a75cc2bacd..96119a06fa6 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1945,40 +1945,76 @@ Aggregator::ConvertToBlockRes template Aggregator::ConvertToBlockRes NO_INLINE -Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const +Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { - auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + const size_t max_block_size = DEFAULT_BLOCK_SIZE; + ConvertToBlockRes res; - if constexpr (Method::low_cardinality_optimization) + std::optional out_cols; + std::optional shuffled_key_sizes; + const Sizes * key_sizes_ptr = nullptr; + + auto init_out_cols = [&]() { - if (data.hasNullKeyData()) + out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, max_block_size); + + if constexpr (Method::low_cardinality_optimization) { - raw_key_columns[0]->insertDefault(); + if (data.hasNullKeyData()) + { + out_cols->raw_key_columns[0]->insertDefault(); - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); + for (size_t i = 0; i < params.aggregates_size; ++i) + out_cols->aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); - data.getNullKeyData() = nullptr; + data.getNullKeyData() = nullptr; + } } - } - auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); - const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; + shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); + key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; + }; + + size_t rows_in_current_block = 0; data.forEachValue( [&](const auto & key, auto & mapped) { - method.insertKeyIntoColumns(key, out_cols.raw_key_columns, key_sizes_ref); + if (!out_cols.has_value()) + init_out_cols(); + + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) - out_cols.aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]); + out_cols->aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]); mapped = nullptr; + + ++rows_in_current_block; + + if constexpr (!return_single_block) + { + if (rows_in_current_block >= max_block_size) + { + res.emplace_back(finalizeBlock(std::move(out_cols.value()), /* final */ false, rows_in_current_block)); + out_cols.reset(); + rows_in_current_block = 0; + } + } }); - return {finalizeBlock(std::move(out_cols), /* final */ false, rows)}; + if constexpr (return_single_block) + { + return finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block); + } + else + { + if (rows_in_current_block) + res.emplace_back(finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block)); + return res; + } + return res; } Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const From 66b3268c659c3d3594821e7b4ba74f0a20877a30 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 20:13:33 +0200 Subject: [PATCH 080/582] fix --- src/Interpreters/Aggregator.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 96119a06fa6..1f47684575b 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1908,6 +1908,9 @@ Aggregator::ConvertToBlockRes places.reserve(max_block_size); }; + // should be invoked at least once, because null data might be the only content of the `data` + init_out_cols(); + data.forEachValue( [&](const auto & key, auto & mapped) { @@ -1975,6 +1978,9 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; }; + // should be invoked at least once, because null data might be the only content of the `data` + init_out_cols(); + size_t rows_in_current_block = 0; data.forEachValue( From 4250a142a7300d82fdca48eaaff0161cc853bc7a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 20:51:49 +0200 Subject: [PATCH 081/582] fix test --- ...egate_function_group_bitmap_data.reference | 68 +++++++++---------- ...1_aggregate_function_group_bitmap_data.sql | 3 +- 2 files changed, 36 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference index 161f4a6372f..8c3288df670 100644 --- a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference +++ b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference @@ -1,27 +1,12 @@ 1 50 50 1 0 49 1 50 50 1 0 49 1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 50 1 0 49 -1 50 50 1 0 49 1 50 50 1 0 49 1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 51 0 1 51 1 50 50 1 0 49 1 50 50 1 0 49 1 50 50 1 0 49 -1 50 51 0 1 51 1 50 50 1 0 49 -1 50 51 0 1 51 1 50 50 1 0 49 1 50 50 1 0 49 1 50 50 1 0 49 @@ -29,32 +14,47 @@ 1 50 50 1 0 49 1 50 50 1 0 49 1 50 50 1 0 49 -1 50 51 0 1 51 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 +1 50 50 1 0 49 1 50 50 1 0 49 1 50 51 0 1 51 -1 50 50 1 0 49 1 50 51 0 1 51 1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 50 1 0 49 -1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 50 1 0 49 1 50 51 0 1 51 1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 50 1 0 49 -1 50 50 1 0 49 -1 50 50 1 0 49 1 50 51 0 1 51 1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 50 1 0 49 1 50 51 0 1 51 -1 50 50 1 0 49 -1 50 50 1 0 49 -1 50 50 1 0 49 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 +1 50 51 0 1 51 1 50 51 0 1 51 diff --git a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql index d70665655ca..a04f40058fd 100644 --- a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql +++ b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql @@ -52,6 +52,7 @@ ALL LEFT JOIN FROM group_bitmap_data_test WHERE pickup_date = '2019-01-01' GROUP BY city_id -) AS js2 USING (city_id); +) AS js2 USING (city_id) +ORDER BY today_users, before_users, ll_users, old_users, new_users, diff_users; DROP TABLE IF EXISTS group_bitmap_data_test; From 2b76abdacd539456280e8b85204736904bf50941 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 18 Jul 2022 15:19:26 +0200 Subject: [PATCH 082/582] fix tidy --- src/Interpreters/Aggregator.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 1f47684575b..9f075602ee0 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2233,9 +2233,7 @@ Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variant *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } - if (false) - { - } // NOLINT + if (false) {} // NOLINT APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) #undef M else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); From 370c25cd2a387df1bb3b3bf67c3e1b07c1500501 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 29 Jul 2022 16:05:18 +0200 Subject: [PATCH 083/582] fix comment --- src/Interpreters/Aggregator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 9f075602ee0..984b4055ada 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -3130,7 +3130,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) } else { - // Used during memory efficient merging in SortingAggregatedTransform (expects single chunk for each bucket_id). + // Used during memory efficient merging (SortingAggregatedTransform expects single chunk for each bucket_id). constexpr bool return_single_block = true; block = prepareBlockAndFillSingleLevel(result, final); } From 6bdbaccc37379a73a975426b9cc6959f0ea1c0f7 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 29 Jul 2022 16:10:53 +0200 Subject: [PATCH 084/582] use max_block_size from settings --- src/Interpreters/Aggregator.cpp | 6 +++--- src/Interpreters/Aggregator.h | 8 ++++++-- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- src/Processors/QueryPlan/AggregatingStep.cpp | 1 + src/Processors/TTL/TTLAggregationAlgorithm.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + tests/queries/0_stateless/02343_aggregation_pipeline.sql | 1 + .../02355_control_block_size_in_aggregator.reference | 1 + .../02355_control_block_size_in_aggregator.sql | 9 +++++++++ 9 files changed, 26 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference create mode 100644 tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 984b4055ada..49cf42ac3b4 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1792,7 +1792,7 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template Aggregator::ConvertToBlockRes - NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const { auto insert_results_into_cols = [this, arena](PaddedPODArray & places, auto & out_cols) { @@ -1881,7 +1881,7 @@ Aggregator::ConvertToBlockRes return finalizeBlock(std::move(out_cols), /* final */ true, places.size()); }; - const size_t max_block_size = DEFAULT_BLOCK_SIZE; + const size_t max_block_size = params.max_block_size; ConvertToBlockRes res; std::optional out_cols; @@ -1950,7 +1950,7 @@ template Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { - const size_t max_block_size = DEFAULT_BLOCK_SIZE; + const size_t max_block_size = params.max_block_size; ConvertToBlockRes res; std::optional out_cols; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 7021a757df1..3e9dbb7780a 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -933,6 +933,8 @@ public: bool compile_aggregate_expressions; size_t min_count_to_compile_aggregate_expression; + size_t max_block_size; + bool only_merge; struct StatsCollectingParams @@ -969,6 +971,7 @@ public: size_t min_free_disk_space_, bool compile_aggregate_expressions_, size_t min_count_to_compile_aggregate_expression_, + size_t max_block_size_, bool only_merge_ = false, // true for projections const StatsCollectingParams & stats_collecting_params_ = {}) : keys(keys_) @@ -987,15 +990,16 @@ public: , 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_) + , max_block_size(max_block_size_) , only_merge(only_merge_) , stats_collecting_params(stats_collecting_params_) { } /// Only parameters that matter during merge. - Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_) + Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_, size_t max_block_size_) : Params( - keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, true, {}) + keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, max_block_size_, true, {}) { } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 205ec049975..fd676859c16 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1719,7 +1719,7 @@ static void executeMergeAggregatedImpl( * but it can work more slowly. */ - Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads); + Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size); auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), @@ -2315,6 +2315,7 @@ static Aggregator::Params getAggregatorParams( settings.min_free_disk_space_for_temporary_data, settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression, + settings.max_block_size, /* only_merge */ false, stats_collecting_params }; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 3680b28f449..5a5326091e6 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -182,6 +182,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B transform_params->params.min_free_disk_space, transform_params->params.compile_aggregate_expressions, transform_params->params.min_count_to_compile_aggregate_expression, + transform_params->params.max_block_size, /* only_merge */ false, transform_params->params.stats_collecting_params}; auto transform_params_for_set = std::make_shared(src_header, std::move(params_for_set), final); diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index 0d160b8d32d..6a813a770cf 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -38,7 +38,8 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm( 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, + settings.max_block_size); aggregator = std::make_unique(header, params); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c5f546a9c36..b3d85e3c00c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -313,6 +313,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( settings.min_free_disk_space_for_temporary_data, settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression, + settings.max_block_size, only_merge); return std::make_pair(params, only_merge); diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.sql b/tests/queries/0_stateless/02343_aggregation_pipeline.sql index f25cef15733..85e9fd1be1e 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.sql +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.sql @@ -1,6 +1,7 @@ set max_threads = 16; set prefer_localhost_replica = 1; set optimize_aggregation_in_order = 0; +set max_block_size = 65505; -- { echoOn } diff --git a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql new file mode 100644 index 00000000000..b4754c6d6fe --- /dev/null +++ b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql @@ -0,0 +1,9 @@ +SET max_block_size = 4213; + +SELECT DISTINCT (blockSize() <= 4213) +FROM +( + SELECT number + FROM numbers(100000) + GROUP BY number +); From 248011d7d9a55d6f58cbc53d6339a3e6fc6f169a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 30 Jul 2022 00:05:14 +0200 Subject: [PATCH 085/582] move to utils --- src/Interpreters/AggregationUtils.h | 119 ++++++++++++++++++++++++++ src/Interpreters/Aggregator.cpp | 127 ++++------------------------ src/Interpreters/Aggregator.h | 12 --- 3 files changed, 136 insertions(+), 122 deletions(-) create mode 100644 src/Interpreters/AggregationUtils.h diff --git a/src/Interpreters/AggregationUtils.h b/src/Interpreters/AggregationUtils.h new file mode 100644 index 00000000000..572757f65e9 --- /dev/null +++ b/src/Interpreters/AggregationUtils.h @@ -0,0 +1,119 @@ +#include + +namespace DB +{ + +struct OutputBlockColumns +{ + MutableColumns key_columns; + std::vector raw_key_columns; + MutableColumns aggregate_columns; + MutableColumns final_aggregate_columns; + Aggregator::AggregateColumnsData aggregate_columns_data; +}; + + +inline OutputBlockColumns prepareOutputBlockColumns( + const Aggregator::Params & params, + const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions, + const Block & res_header, + Arenas & aggregates_pools, + bool final, + size_t rows) +{ + MutableColumns key_columns(params.keys_size); + MutableColumns aggregate_columns(params.aggregates_size); + MutableColumns final_aggregate_columns(params.aggregates_size); + Aggregator::AggregateColumnsData aggregate_columns_data(params.aggregates_size); + + for (size_t i = 0; i < params.keys_size; ++i) + { + key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); + key_columns[i]->reserve(rows); + } + + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!final) + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = res_header.getByName(aggregate_column_name).type->createColumn(); + + /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. + ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); + + for (auto & pool : aggregates_pools) + column_aggregate_func.addArena(pool); + + aggregate_columns_data[i] = &column_aggregate_func.getData(); + aggregate_columns_data[i]->reserve(rows); + } + else + { + final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); + final_aggregate_columns[i]->reserve(rows); + + if (aggregate_functions[i]->isState()) + { + /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. + if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + + /// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator. + final_aggregate_columns[i]->forEachSubcolumn( + [&aggregates_pools](auto & subcolumn) + { + if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + }); + } + } + } + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + + return { + .key_columns = std::move(key_columns), + .raw_key_columns = std::move(raw_key_columns), + .aggregate_columns = std::move(aggregate_columns), + .final_aggregate_columns = std::move(final_aggregate_columns), + .aggregate_columns_data = std::move(aggregate_columns_data), + }; +} + +inline Block +finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows) +{ + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + Block res = res_header.cloneEmpty(); + + for (size_t i = 0; i < params.keys_size; ++i) + res.getByPosition(i).column = std::move(key_columns[i]); + + for (size_t i = 0; i < params.aggregates_size; ++i) + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + if (final) + res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); + else + res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); + } + + /// Change the size of the columns-constants in the block. + size_t columns = res_header.columns(); + for (size_t i = 0; i < columns; ++i) + if (isColumnConst(*res.getByPosition(i).column)) + res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows); + + return res; +} +} diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 49cf42ac3b4..aa04e1aec6e 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -32,6 +32,8 @@ #include +#include + namespace ProfileEvents { extern const Event ExternalAggregationWritePart; @@ -1693,8 +1695,8 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are { if (data.empty()) { - auto && out_cols = prepareOutputBlockColumns(aggregates_pools, final, rows); - return {finalizeBlock(std::move(out_cols), final, rows)}; + auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows); + return {finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows)}; } ConvertToBlockRes res; @@ -1878,7 +1880,7 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are if (exception) std::rethrow_exception(exception); - return finalizeBlock(std::move(out_cols), /* final */ true, places.size()); + return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); }; const size_t max_block_size = params.max_block_size; @@ -1891,7 +1893,8 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, max_block_size); + out_cols = prepareOutputBlockColumns( + params, aggregate_functions, getHeader(/* final */ true), aggregates_pools, /* final */ true, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -1959,7 +1962,8 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, max_block_size); + out_cols = prepareOutputBlockColumns( + params, aggregate_functions, getHeader(/* final */ false), aggregates_pools, /* final */ false, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -2003,7 +2007,8 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a { if (rows_in_current_block >= max_block_size) { - res.emplace_back(finalizeBlock(std::move(out_cols.value()), /* final */ false, rows_in_current_block)); + res.emplace_back(finalizeBlock( + params, getHeader(/* final */ false), std::move(out_cols.value()), /* final */ false, rows_in_current_block)); out_cols.reset(); rows_in_current_block = 0; } @@ -2012,117 +2017,18 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a if constexpr (return_single_block) { - return finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block); + return finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_block); } else { if (rows_in_current_block) - res.emplace_back(finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block)); + res.emplace_back( + finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_block)); return res; } return res; } -Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const -{ - MutableColumns key_columns(params.keys_size); - MutableColumns aggregate_columns(params.aggregates_size); - MutableColumns final_aggregate_columns(params.aggregates_size); - AggregateColumnsData aggregate_columns_data(params.aggregates_size); - - Block res_header = getHeader(final); - - for (size_t i = 0; i < params.keys_size; ++i) - { - key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); - key_columns[i]->reserve(rows); - } - - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!final) - { - const auto & aggregate_column_name = params.aggregates[i].column_name; - aggregate_columns[i] = res_header.getByName(aggregate_column_name).type->createColumn(); - - /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. - ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); - - for (auto & pool : aggregates_pools) - column_aggregate_func.addArena(pool); - - aggregate_columns_data[i] = &column_aggregate_func.getData(); - aggregate_columns_data[i]->reserve(rows); - } - else - { - final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); - final_aggregate_columns[i]->reserve(rows); - - if (aggregate_functions[i]->isState()) - { - /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. - if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) - for (auto & pool : aggregates_pools) - column_aggregate_func->addArena(pool); - - /// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator. - final_aggregate_columns[i]->forEachSubcolumn( - [&aggregates_pools](auto & subcolumn) - { - if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) - for (auto & pool : aggregates_pools) - column_aggregate_func->addArena(pool); - }); - } - } - } - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); - - return { - .key_columns = std::move(key_columns), - .raw_key_columns = std::move(raw_key_columns), - .aggregate_columns = std::move(aggregate_columns), - .final_aggregate_columns = std::move(final_aggregate_columns), - .aggregate_columns_data = std::move(aggregate_columns_data), - }; -} - -Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const -{ - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - Block res_header = getHeader(final); - Block res = res_header.cloneEmpty(); - - for (size_t i = 0; i < params.keys_size; ++i) - res.getByPosition(i).column = std::move(key_columns[i]); - - for (size_t i = 0; i < params.aggregates_size; ++i) - { - const auto & aggregate_column_name = params.aggregates[i].column_name; - if (final) - res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); - else - res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); - } - - /// Change the size of the columns-constants in the block. - size_t columns = res_header.columns(); - for (size_t i = 0; i < columns; ++i) - if (isColumnConst(*res.getByPosition(i).column)) - res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows); - - return res; -} - void Aggregator::addSingleKeyToAggregateColumns( AggregatedDataVariants & data_variants, MutableColumns & aggregate_columns) const @@ -2183,7 +2089,8 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const { size_t rows = 1; - auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); + auto && out_cols + = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), data_variants.aggregates_pools, final, rows); auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row) @@ -2210,7 +2117,7 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va key_columns[i]->insertDefault(); } - Block block = finalizeBlock(std::move(out_cols), final, rows); + Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows); if (is_overflows) block.info.is_overflows = true; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3e9dbb7780a..2fd76a438a6 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1301,18 +1301,6 @@ private: ConvertToBlockRes convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; - struct OutputBlockColumns - { - MutableColumns key_columns; - std::vector raw_key_columns; - MutableColumns aggregate_columns; - MutableColumns final_aggregate_columns; - AggregateColumnsData aggregate_columns_data; - }; - - OutputBlockColumns prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const; - Block finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const; - template Block convertOneBucketToBlock( AggregatedDataVariants & data_variants, From a6c4f9218a7f41b341b488482a240ea46391bfc3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 30 Jul 2022 01:03:01 +0200 Subject: [PATCH 086/582] clean up --- src/Interpreters/AggregationUtils.cpp | 113 ++++++++++++++++ src/Interpreters/AggregationUtils.h | 100 +------------- src/Interpreters/Aggregator.cpp | 165 +++++++++++------------ src/Interpreters/Aggregator.h | 4 + src/QueryPipeline/QueryPipelineBuilder.h | 2 - 5 files changed, 201 insertions(+), 183 deletions(-) create mode 100644 src/Interpreters/AggregationUtils.cpp diff --git a/src/Interpreters/AggregationUtils.cpp b/src/Interpreters/AggregationUtils.cpp new file mode 100644 index 00000000000..43062546450 --- /dev/null +++ b/src/Interpreters/AggregationUtils.cpp @@ -0,0 +1,113 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +OutputBlockColumns prepareOutputBlockColumns( + const Aggregator::Params & params, + const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions, + const Block & res_header, + Arenas & aggregates_pools, + bool final, + size_t rows) +{ + MutableColumns key_columns(params.keys_size); + MutableColumns aggregate_columns(params.aggregates_size); + MutableColumns final_aggregate_columns(params.aggregates_size); + Aggregator::AggregateColumnsData aggregate_columns_data(params.aggregates_size); + + for (size_t i = 0; i < params.keys_size; ++i) + { + key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); + key_columns[i]->reserve(rows); + } + + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!final) + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = res_header.getByName(aggregate_column_name).type->createColumn(); + + /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. + ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); + + for (auto & pool : aggregates_pools) + column_aggregate_func.addArena(pool); + + aggregate_columns_data[i] = &column_aggregate_func.getData(); + aggregate_columns_data[i]->reserve(rows); + } + else + { + final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); + final_aggregate_columns[i]->reserve(rows); + + if (aggregate_functions[i]->isState()) + { + /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. + if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + + /// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator. + final_aggregate_columns[i]->forEachSubcolumn( + [&aggregates_pools](auto & subcolumn) + { + if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + }); + } + } + } + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + + return { + .key_columns = std::move(key_columns), + .raw_key_columns = std::move(raw_key_columns), + .aggregate_columns = std::move(aggregate_columns), + .final_aggregate_columns = std::move(final_aggregate_columns), + .aggregate_columns_data = std::move(aggregate_columns_data), + }; +} + +Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows) +{ + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + Block res = res_header.cloneEmpty(); + + for (size_t i = 0; i < params.keys_size; ++i) + res.getByPosition(i).column = std::move(key_columns[i]); + + for (size_t i = 0; i < params.aggregates_size; ++i) + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + if (final) + res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); + else + res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); + } + + /// Change the size of the columns-constants in the block. + size_t columns = res_header.columns(); + for (size_t i = 0; i < columns; ++i) + if (isColumnConst(*res.getByPosition(i).column)) + res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows); + + return res; +} +} diff --git a/src/Interpreters/AggregationUtils.h b/src/Interpreters/AggregationUtils.h index 572757f65e9..d7769b0309d 100644 --- a/src/Interpreters/AggregationUtils.h +++ b/src/Interpreters/AggregationUtils.h @@ -13,107 +13,13 @@ struct OutputBlockColumns }; -inline OutputBlockColumns prepareOutputBlockColumns( +OutputBlockColumns prepareOutputBlockColumns( const Aggregator::Params & params, const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions, const Block & res_header, Arenas & aggregates_pools, bool final, - size_t rows) -{ - MutableColumns key_columns(params.keys_size); - MutableColumns aggregate_columns(params.aggregates_size); - MutableColumns final_aggregate_columns(params.aggregates_size); - Aggregator::AggregateColumnsData aggregate_columns_data(params.aggregates_size); + size_t rows); - for (size_t i = 0; i < params.keys_size; ++i) - { - key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); - key_columns[i]->reserve(rows); - } - - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!final) - { - const auto & aggregate_column_name = params.aggregates[i].column_name; - aggregate_columns[i] = res_header.getByName(aggregate_column_name).type->createColumn(); - - /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. - ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); - - for (auto & pool : aggregates_pools) - column_aggregate_func.addArena(pool); - - aggregate_columns_data[i] = &column_aggregate_func.getData(); - aggregate_columns_data[i]->reserve(rows); - } - else - { - final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); - final_aggregate_columns[i]->reserve(rows); - - if (aggregate_functions[i]->isState()) - { - /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. - if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) - for (auto & pool : aggregates_pools) - column_aggregate_func->addArena(pool); - - /// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator. - final_aggregate_columns[i]->forEachSubcolumn( - [&aggregates_pools](auto & subcolumn) - { - if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) - for (auto & pool : aggregates_pools) - column_aggregate_func->addArena(pool); - }); - } - } - } - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); - - return { - .key_columns = std::move(key_columns), - .raw_key_columns = std::move(raw_key_columns), - .aggregate_columns = std::move(aggregate_columns), - .final_aggregate_columns = std::move(final_aggregate_columns), - .aggregate_columns_data = std::move(aggregate_columns_data), - }; -} - -inline Block -finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows) -{ - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - Block res = res_header.cloneEmpty(); - - for (size_t i = 0; i < params.keys_size; ++i) - res.getByPosition(i).column = std::move(key_columns[i]); - - for (size_t i = 0; i < params.aggregates_size; ++i) - { - const auto & aggregate_column_name = params.aggregates[i].column_name; - if (final) - res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); - else - res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); - } - - /// Change the size of the columns-constants in the block. - size_t columns = res_header.columns(); - for (size_t i = 0; i < columns; ++i) - if (isColumnConst(*res.getByPosition(i).column)) - res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows); - - return res; -} +Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows); } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index aa04e1aec6e..84708fc72ea 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1792,77 +1792,40 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu } -template -Aggregator::ConvertToBlockRes -NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +template +Block Aggregator::insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena) const { - auto insert_results_into_cols = [this, arena](PaddedPODArray & places, auto & out_cols) - { - std::exception_ptr exception; - size_t aggregate_functions_destroy_index = 0; + std::exception_ptr exception; + size_t aggregate_functions_destroy_index = 0; - try - { + try + { #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - /** For JIT compiled functions we need to resize columns before pass them into compiled code. + if constexpr (use_compiled_functions) + { + /** For JIT compiled functions we need to resize columns before pass them into compiled code. * insert_aggregates_into_columns_function function does not throw exception. */ - std::vector columns_data; + std::vector columns_data; - auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; + auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!is_aggregate_function_compiled[i]) - continue; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!is_aggregate_function_compiled[i]) + continue; - auto & final_aggregate_column = out_cols.final_aggregate_columns[i]; - final_aggregate_column = final_aggregate_column->cloneResized(places.size()); - columns_data.emplace_back(getColumnData(final_aggregate_column.get())); - } - - auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; - insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); + auto & final_aggregate_column = out_cols.final_aggregate_columns[i]; + final_aggregate_column = final_aggregate_column->cloneResized(places.size()); + columns_data.emplace_back(getColumnData(final_aggregate_column.get())); } + + auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; + insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); + } #endif - for (; aggregate_functions_destroy_index < params.aggregates_size;) - { - if constexpr (use_compiled_functions) - { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) - { - ++aggregate_functions_destroy_index; - continue; - } - } - - auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index]; - size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; - - /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch - * throws exception, it also must destroy all necessary states. - * Then code need to continue to destroy other aggregate function states with next function index. - */ - size_t destroy_index = aggregate_functions_destroy_index; - ++aggregate_functions_destroy_index; - - /// For State AggregateFunction ownership of aggregate place is passed to result column after insert - bool is_state = aggregate_functions[destroy_index]->isState(); - bool destroy_place_after_insert = !is_state; - - aggregate_functions[destroy_index]->insertResultIntoBatch( - 0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert); - } - } - catch (...) - { - exception = std::current_exception(); - } - - for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) + for (; aggregate_functions_destroy_index < params.aggregates_size;) { if constexpr (use_compiled_functions) { @@ -1873,28 +1836,65 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are } } + auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index]; size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; - aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); + + /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch + * throws exception, it also must destroy all necessary states. + * Then code need to continue to destroy other aggregate function states with next function index. + */ + size_t destroy_index = aggregate_functions_destroy_index; + ++aggregate_functions_destroy_index; + + /// For State AggregateFunction ownership of aggregate place is passed to result column after insert + bool is_state = aggregate_functions[destroy_index]->isState(); + bool destroy_place_after_insert = !is_state; + + aggregate_functions[destroy_index]->insertResultIntoBatch( + 0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert); + } + } + catch (...) + { + exception = std::current_exception(); + } + + for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) + { + if constexpr (use_compiled_functions) + { + if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) + { + ++aggregate_functions_destroy_index; + continue; + } } - if (exception) - std::rethrow_exception(exception); + size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; + aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); + } - return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); - }; + if (exception) + std::rethrow_exception(exception); + return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); +} + +template +Aggregator::ConvertToBlockRes NO_INLINE +Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +{ const size_t max_block_size = params.max_block_size; + const bool final = true; ConvertToBlockRes res; std::optional out_cols; std::optional shuffled_key_sizes; - const Sizes * key_sizes_ptr = nullptr; PaddedPODArray places; auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns( - params, aggregate_functions, getHeader(/* final */ true), aggregates_pools, /* final */ true, max_block_size); + out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -1906,7 +1906,6 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are } shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); - key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; places.reserve(max_block_size); }; @@ -1920,7 +1919,8 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are if (!out_cols.has_value()) init_out_cols(); - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); + const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref); places.emplace_back(mapped); /// Mark the cell as destroyed so it will not be destroyed in destructor. @@ -1930,7 +1930,7 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are { if (places.size() >= max_block_size) { - res.emplace_back(insert_results_into_cols(places, out_cols.value())); + res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena)); places.clear(); out_cols.reset(); } @@ -1939,12 +1939,12 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are if constexpr (return_single_block) { - return insert_results_into_cols(places, out_cols.value()); + return insertResultsIntoColumns(places, std::move(out_cols.value()), arena); } else { - if (!places.empty()) - res.emplace_back(insert_results_into_cols(places, out_cols.value())); + if (out_cols.has_value()) + res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena)); return res; } } @@ -1954,16 +1954,15 @@ Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { const size_t max_block_size = params.max_block_size; + const bool final = false; ConvertToBlockRes res; std::optional out_cols; std::optional shuffled_key_sizes; - const Sizes * key_sizes_ptr = nullptr; auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns( - params, aggregate_functions, getHeader(/* final */ false), aggregates_pools, /* final */ false, max_block_size); + out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -1979,7 +1978,6 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a } shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); - key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; }; // should be invoked at least once, because null data might be the only content of the `data` @@ -1993,7 +1991,8 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a if (!out_cols.has_value()) init_out_cols(); - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); + const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref); /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) @@ -2007,8 +2006,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a { if (rows_in_current_block >= max_block_size) { - res.emplace_back(finalizeBlock( - params, getHeader(/* final */ false), std::move(out_cols.value()), /* final */ false, rows_in_current_block)); + res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols.value()), final, rows_in_current_block)); out_cols.reset(); rows_in_current_block = 0; } @@ -2017,13 +2015,12 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a if constexpr (return_single_block) { - return finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_block); + return finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block); } else { if (rows_in_current_block) - res.emplace_back( - finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_block)); + res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block)); return res; } return res; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 2fd76a438a6..9c762318b2b 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -872,6 +872,7 @@ using ManyAggregatedDataVariantsPtr = std::shared_ptr + Block insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena) const; + template ConvertToBlockRes convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 850c4a66615..100a2e07341 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -161,8 +161,6 @@ public: return std::max(1, num_threads); } - size_t getMaxThreads() const { return max_threads; } - /// Set upper limit for the recommend number of threads void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } From 14d45f9d86c198006f1d162bf918e8df6fb54bcd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 30 Jul 2022 19:49:43 +0200 Subject: [PATCH 087/582] fix style --- src/Interpreters/AggregationUtils.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/AggregationUtils.h b/src/Interpreters/AggregationUtils.h index d7769b0309d..cc37cec0a69 100644 --- a/src/Interpreters/AggregationUtils.h +++ b/src/Interpreters/AggregationUtils.h @@ -1,3 +1,5 @@ +#pragma once + #include namespace DB From 9af04ddf2186bab1137f3b2b69d127509da74955 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 1 Aug 2022 19:18:19 +0200 Subject: [PATCH 088/582] fix test --- .../01064_incremental_streaming_from_2_src_with_feedback.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index 0bc5fcd1db8..a63aa768051 100644 --- a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -23,6 +23,8 @@ AS FROM numbers(50000) GROUP BY id; +OPTIMIZE TABLE target_table FINAL; + -- source table #1 CREATE TABLE logins ( From 0bad9425d26d08fd2087940374ac5cee5b5f14f4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Aug 2022 19:27:40 +0200 Subject: [PATCH 089/582] Some improvements in tests --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 4 ++++ src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 2 ++ .../01079_parallel_alter_add_drop_column_zookeeper.sh | 4 ++++ 3 files changed, 10 insertions(+) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e017e19c06c..90f08b7ebd1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -267,6 +267,8 @@ void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exis auto outcome = client_ptr->DeleteObject(request); throwIfUnexpectedError(outcome, if_exists); + + LOG_TRACE(log, "Object with path {} was removed from S3", object.absolute_path); } void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_exists) @@ -310,6 +312,8 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e auto outcome = client_ptr->DeleteObjects(request); throwIfUnexpectedError(outcome, if_exists); + + LOG_TRACE(log, "Objects with paths [{}] were removed from S3", keys); } } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 8f20671d841..2a90c0132a1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -179,6 +179,8 @@ private: const String version_id; FileCachePtr cache; + + Poco::Logger * log = &Poco::Logger::get("S3ObjectStorage"); }; } diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index 06d6ef6a94b..26c2bf133ac 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -111,5 +111,9 @@ for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i'" $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i' and (type = 'ALTER_METADATA' or type = 'MUTATE_PART')" + + $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_alter_add_drop_$i" + $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_add_drop_$i" + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_add_drop_$i" done From 172b5025e23170ab3e4275474f6bbeaf2ccc8c10 Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Wed, 17 Aug 2022 11:24:19 +0800 Subject: [PATCH 090/582] use camelCase aNotB --- src/AggregateFunctions/ThetaSketchData.h | 2 +- src/Functions/UniqTheta/FunctionsUniqTheta.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/ThetaSketchData.h b/src/AggregateFunctions/ThetaSketchData.h index 0bce5bf1d5b..cd17719a45a 100644 --- a/src/AggregateFunctions/ThetaSketchData.h +++ b/src/AggregateFunctions/ThetaSketchData.h @@ -106,7 +106,7 @@ public: u->update(theta_intersection.get_result()); } - void a_not_b(const ThetaSketchData & rhs) + void aNotB(const ThetaSketchData & rhs) { datasketches::theta_union * u = getSkUnion(); diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.h b/src/Functions/UniqTheta/FunctionsUniqTheta.h index bd9ad11b9b3..cbbdacedcb3 100644 --- a/src/Functions/UniqTheta/FunctionsUniqTheta.h +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.h @@ -53,7 +53,7 @@ namespace DB { static void apply(AggregateFunctionUniqThetaData & sketch_data_1, const AggregateFunctionUniqThetaData & sketch_data_2) { - sketch_data_1.set.a_not_b(sketch_data_2.set); + sketch_data_1.set.aNotB(sketch_data_2.set); } }; From dd596121b804f3fe387e8de05556ded4b6f0479d Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 17 Aug 2022 11:04:46 +0300 Subject: [PATCH 091/582] Add comment with explain for unfreeze with zero-copy replication --- src/Storages/Freeze.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 24f99e0ed2a..52eeb9bdf3c 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -5,6 +5,19 @@ #include #include +/** + * When ClickHouse has frozen data on remote storage it requred 'smart' data removing during UNFREEZE. + * For remote storage actually frozen not remote data but local metadata with referrers on remote data. + * So remote data can be referred from working and frozen data sets (or two frozen) at same time. + * In this case during UNFREEZE ClickHouse should remove only local metadata and keep remote data. + * But when data was already removed from working data set ClickHouse should remove remote data too. + * To detect is current data used or not in some other place ClickHouse uses + * - ref_count from metadata to check if data used in some other metadata on the same replica; + * - Keeper record to check if data used on other replica. + * StorageReplicatedMergeTree::removeSharedDetachedPart makes required checks, so here this method + * called for each frozen part. + */ + namespace DB { namespace ErrorCodes @@ -61,7 +74,7 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) readIntText(version, buffer); if (version < 1 or version > 2) { - LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version); + LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown frozen metadata version: {}", version); return false; } DB::assertChar('\n', buffer); From 9242c2adb4af1cd930951156bec8c862ee5707f2 Mon Sep 17 00:00:00 2001 From: Stephan <79573800+cyber-moon@users.noreply.github.com> Date: Wed, 17 Aug 2022 11:56:48 +0200 Subject: [PATCH 092/582] Add description of {condition}-keyword Using the {condition}-keyword for Dictionaries is supported since a few weeks (see https://github.com/ClickHouse/ClickHouse/issues/33746 ). This PR adds a corresponding documentation. --- .../external-dicts-dict-lifetime.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md index ab83017f263..e2abdf21025 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md @@ -93,6 +93,21 @@ It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to onl - If the source is HTTP then `update_field` will be added as a query parameter with the last update time as the parameter value. - If the source is Executable then `update_field` will be added as an executable script argument with the last update time as the argument value. - If the source is ClickHouse, MySQL, PostgreSQL, ODBC there will be an additional part of `WHERE`, where `update_field` is compared as greater or equal with the last update time. + - Per default, this `WHERE`-condition is checked at the highest level of the SQL-Query. Alternatively, the condition can be checked in any other `WHERE`-clause within the query using the `{condition}`-keyword. Example: + ```sql + ... + SOURCE(CLICKHOUSE(... + update_field 'added_time' + QUERY ' + SELECT my_arr.1 AS x, my_arr.2 AS y, creation_time + FROM ( + SELECT arrayZip(x_arr, y_arr) AS my_arr, creation_time + FROM dictionary_source + WHERE {condition} + )' + )) + ... + ``` If `update_field` option is set, additional option `update_lag` can be set. Value of `update_lag` option is subtracted from previous update time before request updated data. From 0433b801d2d8e157aca800319755510082b930ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Aug 2022 12:27:15 +0200 Subject: [PATCH 093/582] Configure properly --- docker/test/stress/run.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 9546cbb8d54..f3a4e2ff476 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -179,7 +179,8 @@ install_packages package_folder configure -./setup_minio.sh stateful # to have a proper environment +./setup_minio.sh stateless # to have a proper environment + start From 600d22851f912a7629e7991818229446ff044415 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Aug 2022 12:43:11 +0200 Subject: [PATCH 094/582] Grep dangerous S3 errors --- docker/test/stress/run.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index f3a4e2ff476..d94c79dc6bd 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -277,6 +277,14 @@ zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-serve # Remove file logical_errors.txt if it's empty [ -s /test_output/logical_errors.txt ] || rm /test_output/logical_errors.txt +# No such key errors +zgrep -Ea "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log > /test_output/no_such_key_errors.txt \ + && echo -e 'S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'No lost s3 keys\tOK' >> /test_output/test_results.tsv + +# Remove file no_such_key_errors.txt if it's empty +[ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt + # Crash zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ && echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ From 5ceaeb9e1217a34397d117d0dabef573b4d1925a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 Aug 2022 12:14:13 +0000 Subject: [PATCH 095/582] Sorting mode renaming + sort mode -> sort scope + Stream -> Global + Port -> Stream --- src/Processors/QueryPlan/DistinctStep.cpp | 2 +- src/Processors/QueryPlan/IQueryPlanStep.h | 12 +++---- .../QueryPlan/ITransformingStep.cpp | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 4 +-- .../QueryPlan/ReadFromMergeTree.cpp | 4 +-- src/Processors/QueryPlan/SortingStep.cpp | 16 +++++----- ..._input_stream_properties_explain.reference | 32 +++++++++---------- 7 files changed, 36 insertions(+), 36 deletions(-) diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 7da2b5252f5..e3d29256c23 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -108,7 +108,7 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil return; } /// final distinct for sorted stream (sorting inside and among chunks) - if (input_stream.sort_mode == DataStream::SortMode::Stream) + if (input_stream.sort_scope == DataStream::SortScope::Global) { assert(input_stream.has_single_port); diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index a66582ff06d..c5bd64d66be 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -31,18 +31,18 @@ public: /// QueryPipeline has single port. Totals or extremes ports are not counted. bool has_single_port = false; - /// How data is sorted. - enum class SortMode + /// Sorting scope + enum class SortScope { None, Chunk, /// Separate chunks are sorted - Port, /// Data from each port is sorted - Stream, /// Data is globally sorted + Stream, /// Each data steam is sorted + Global, /// Data is globally sorted }; /// It is not guaranteed that header has columns from sort_description. SortDescription sort_description = {}; - SortMode sort_mode = SortMode::None; + SortScope sort_scope = SortScope::None; /// Things which may be added: /// * limit @@ -54,7 +54,7 @@ public: return distinct_columns == other.distinct_columns && has_single_port == other.has_single_port && sort_description == other.sort_description - && (sort_description.empty() || sort_mode == other.sort_mode); + && (sort_description.empty() || sort_scope == other.sort_scope); } bool hasEqualHeaderWith(const DataStream & other) const diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index 9b9797b6540..64ad2ec5626 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -29,7 +29,7 @@ DataStream ITransformingStep::createOutputStream( if (stream_traits.preserves_sorting) { output_stream.sort_description = input_stream.sort_description; - output_stream.sort_mode = input_stream.sort_mode; + output_stream.sort_scope = input_stream.sort_scope; } return output_stream; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index c27c0c0d318..9b4c0a6e920 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -333,8 +333,8 @@ static void explainStep( { if (step.hasOutputStream()) { - settings.out << prefix << "Sorting (" << step.getOutputStream().sort_mode << ")"; - if (step.getOutputStream().sort_mode != DataStream::SortMode::None) + settings.out << prefix << "Sorting (" << step.getOutputStream().sort_scope << ")"; + if (step.getOutputStream().sort_scope != DataStream::SortScope::None) { settings.out << ": "; dumpSortDescription(step.getOutputStream().sort_description, settings.out); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d7dae54bb1e..ba381d64b50 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -143,9 +143,9 @@ ReadFromMergeTree::ReadFromMergeTree( { auto const & settings = context->getSettingsRef(); if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && query_info.getInputOrderInfo()) - output_stream->sort_mode = DataStream::SortMode::Port; + output_stream->sort_scope = DataStream::SortScope::Stream; else - output_stream->sort_mode = DataStream::SortMode::Chunk; + output_stream->sort_scope = DataStream::SortScope::Chunk; } output_stream->sort_description = std::move(sort_description); diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 38e02eebd44..9bad6a02d53 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -55,7 +55,7 @@ SortingStep::SortingStep( { /// TODO: check input_stream is partially sorted by the same description. output_stream->sort_description = result_description; - output_stream->sort_mode = DataStream::SortMode::Stream; + output_stream->sort_scope = DataStream::SortScope::Global; } SortingStep::SortingStep( @@ -73,7 +73,7 @@ SortingStep::SortingStep( { /// TODO: check input_stream is sorted by prefix_description. output_stream->sort_description = result_description; - output_stream->sort_mode = DataStream::SortMode::Stream; + output_stream->sort_scope = DataStream::SortScope::Global; } SortingStep::SortingStep( @@ -89,14 +89,14 @@ SortingStep::SortingStep( { /// TODO: check input_stream is partially sorted (each port) by the same description. output_stream->sort_description = result_description; - output_stream->sort_mode = DataStream::SortMode::Stream; + output_stream->sort_scope = DataStream::SortScope::Global; } void SortingStep::updateOutputStream() { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); output_stream->sort_description = result_description; - output_stream->sort_mode = DataStream::SortMode::Stream; + output_stream->sort_scope = DataStream::SortScope::Global; } void SortingStep::updateLimit(size_t limit_) @@ -256,23 +256,23 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build return; } - const auto input_sort_mode = input_streams.front().sort_mode; + const auto input_sort_mode = input_streams.front().sort_scope; const SortDescription & input_sort_desc = input_streams.front().sort_description; if (optimize_sorting_by_input_stream_properties) { /// skip sorting if stream is already sorted - if (input_sort_mode == DataStream::SortMode::Stream && input_sort_desc.hasPrefix(result_description)) + if (input_sort_mode == DataStream::SortScope::Global && input_sort_desc.hasPrefix(result_description)) return; /// merge sorted - if (input_sort_mode == DataStream::SortMode::Port && input_sort_desc.hasPrefix(result_description)) + if (input_sort_mode == DataStream::SortScope::Stream && input_sort_desc.hasPrefix(result_description)) { mergingSorted(pipeline, result_description, limit); return; } /// if chunks already sorted according to result_sort_desc, then we can skip chunk sorting - if (input_sort_mode == DataStream::SortMode::Chunk && input_sort_desc.hasPrefix(result_description)) + if (input_sort_mode == DataStream::SortScope::Chunk && input_sort_desc.hasPrefix(result_description)) { const bool skip_partial_sort = true; fullSort(pipeline, result_description, limit, skip_partial_sort); diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index c32f227006c..1ad64150049 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -1,8 +1,8 @@ -- EXPLAIN PLAN sorting for MergeTree w/o sorting key -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a -Sorting (Stream): a ASC +Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Stream): a ASC +Sorting (Global): a ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort @@ -20,22 +20,22 @@ LimitsCheckingTransform PartialSortingTransform -- ExpressionStep preserves sort mode -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a -Sorting (Stream): a ASC +Sorting (Global): a ASC Sorting +Sorting (Global): a ASC +Sorting (Stream): a ASC Sorting (Stream): a ASC -Sorting (Port): a ASC -Sorting (Port): a ASC -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Stream): plus(a, 1) ASC +Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 -Sorting (Stream): plus(a, 1) ASC +Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) -Sorting (Stream): plus(a, 1) ASC +Sorting (Global): plus(a, 1) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode @@ -62,28 +62,28 @@ Sorting (None) Sorting (Chunk): a ASC -- aliases break sorting order -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a -Sorting (Stream): a ASC +Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Stream): a ASC +Sorting (Global): a ASC Sorting (None) Sorting +Sorting (Global): a ASC +Sorting (Stream): a ASC Sorting (Stream): a ASC -Sorting (Port): a ASC -Sorting (Port): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) -Sorting (Stream): x ASC, y ASC +Sorting (Global): x ASC, y ASC Sorting (Sorting for ORDER BY) -Sorting (Stream): x ASC, y ASC +Sorting (Global): x ASC, y ASC Sorting (Chunk): a ASC, b ASC Sorting (Chunk): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Stream): plus(a, 1) ASC +Sorting (Global): plus(a, 1) ASC Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Stream): plus(a, 1) ASC +Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC From 86b1e33eedbca647c93f661a955b323504b5119e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Aug 2022 19:00:53 +0200 Subject: [PATCH 096/582] Disable cache on writes --- docker/test/stress/run.sh | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index d94c79dc6bd..7136701eeae 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -199,18 +199,15 @@ clickhouse-client --query "SHOW TABLES FROM datasets" clickhouse-client --query "SHOW TABLES FROM test" clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" -clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1" +clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" +clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" +clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" -clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" -clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" - -clickhouse-client --query "DROP TABLE datasets.visits_v1" -clickhouse-client --query "DROP TABLE datasets.hits_v1" +clickhouse-client --query "DROP TABLE datasets.visits_v1 SYNC" +clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC" clickhouse-client --query "SHOW TABLES FROM test" From 77b7ed97a04d695301211ce0cf67a2509b2f1ec7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Aug 2022 19:10:13 +0200 Subject: [PATCH 097/582] Remove redundant setting --- tests/config/users.d/s3_cache.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/config/users.d/s3_cache.xml b/tests/config/users.d/s3_cache.xml index 69b24ecbbc4..176128fbf83 100644 --- a/tests/config/users.d/s3_cache.xml +++ b/tests/config/users.d/s3_cache.xml @@ -1,7 +1,6 @@ - 1 1 From 7627f831cd229095f0937c6db9ce06e0061a0d50 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 18 Aug 2022 11:43:50 +0300 Subject: [PATCH 098/582] Fix tests --- src/Storages/Freeze.cpp | 17 ++++++++++++----- src/Storages/Freeze.h | 11 ++--------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 52eeb9bdf3c..52261cf7705 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -6,7 +6,7 @@ #include /** - * When ClickHouse has frozen data on remote storage it requred 'smart' data removing during UNFREEZE. + * When ClickHouse has frozen data on remote storage it required 'smart' data removing during UNFREEZE. * For remote storage actually frozen not remote data but local metadata with referrers on remote data. * So remote data can be referred from working and frozen data sets (or two frozen) at same time. * In this case during UNFREEZE ClickHouse should remove only local metadata and keep remote data. @@ -20,10 +20,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int INVALID_PARTITION_VALUE; -} void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage) { @@ -114,6 +110,17 @@ String FreezeMetaData::getFileName(const String & path) return fs::path(path) / "frozen_metadata.txt"; } +Unfreezer::Unfreezer(ContextPtr context) : local_context(context), zookeeper() +{ + const auto & config = local_context->getConfigRef(); + static constexpr auto config_key = "enable_system_unfreeze"; + if (!config.getBool(config_key, false)) { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); + } + if (local_context->hasZooKeeper()) + zookeeper = local_context->getZooKeeper(); +} + BlockIO Unfreezer::unfreeze(const String & backup_name) { LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name)); diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index eed1900fbd7..31cda41af9d 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -36,14 +36,7 @@ public: class Unfreezer { public: - Unfreezer(ContextPtr context) : local_context(context), zookeeper() { - const auto & config = local_context->getConfigRef(); - static constexpr auto config_key = "enable_system_unfreeze"; - if (!config.getBool(config_key, false)) { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); - } - zookeeper = this->local_context->getZooKeeper(); - } + Unfreezer(ContextPtr context); PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory); BlockIO unfreeze(const String & backup_name); private: @@ -51,7 +44,7 @@ private: zkutil::ZooKeeperPtr zookeeper; Poco::Logger * log = &Poco::Logger::get("Unfreezer"); static constexpr std::string_view backup_directory_prefix = "shadow"; - static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper); + static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper); }; } From 5aca68a8877c30ea750e70d05da565be7bde17f9 Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Thu, 18 Aug 2022 12:33:51 +0300 Subject: [PATCH 099/582] Add option enable_system_unfreeze to unfreeze test config --- .../01417_freeze_partition_verbose.config.xml | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/01417_freeze_partition_verbose.config.xml diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose.config.xml b/tests/queries/0_stateless/01417_freeze_partition_verbose.config.xml new file mode 100644 index 00000000000..7e13604b8e3 --- /dev/null +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose.config.xml @@ -0,0 +1,25 @@ + + + + trace + true + + + true + 9000 + 0 + 0 + . + 0 + + + + + users.xml + + + + ./ + + + From b548979fc5ed276af96b679542e549299306506f Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Thu, 18 Aug 2022 12:44:57 +0300 Subject: [PATCH 100/582] Fix style tests --- src/Storages/Freeze.cpp | 14 +++++++++++--- src/Storages/Freeze.h | 4 ---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 52261cf7705..a588b8b64e9 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -21,6 +21,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_PARTITION_VALUE; +} + void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage) { replica_name = storage.getReplicaName(); @@ -38,7 +43,8 @@ void FreezeMetaData::save(DiskPtr data_disk, const String & path) const writeIntText(version, buffer); buffer.write("\n", 1); - if (version == 1) { + if (version == 1) + { /// is_replicated and is_remote are not used bool is_replicated = true; writeBoolText(is_replicated, buffer); @@ -74,7 +80,8 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) return false; } DB::assertChar('\n', buffer); - if (version == 1) { + if (version == 1) + { /// is_replicated and is_remote are not used bool is_replicated; readBoolText(is_replicated, buffer); @@ -114,7 +121,8 @@ Unfreezer::Unfreezer(ContextPtr context) : local_context(context), zookeeper() { const auto & config = local_context->getConfigRef(); static constexpr auto config_key = "enable_system_unfreeze"; - if (!config.getBool(config_key, false)) { + if (!config.getBool(config_key, false)) + { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); } if (local_context->hasZooKeeper()) diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index 31cda41af9d..cd8dcdf824d 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -6,10 +6,6 @@ namespace DB { -namespace ErrorCodes { - extern const int SUPPORT_IS_DISABLED; -} - /// Special metadata used during freeze table. Required for zero-copy /// replication. struct FreezeMetaData From 46de6997dd7ef0da596a6c2dfa07197ca7355b65 Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Thu, 18 Aug 2022 13:03:45 +0300 Subject: [PATCH 101/582] fix style more --- src/Storages/Freeze.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index a588b8b64e9..99a27553081 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -23,7 +23,7 @@ namespace DB namespace ErrorCodes { - extern const int INVALID_PARTITION_VALUE; + extern const int SUPPORT_IS_DISABLED; } void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage) @@ -43,7 +43,7 @@ void FreezeMetaData::save(DiskPtr data_disk, const String & path) const writeIntText(version, buffer); buffer.write("\n", 1); - if (version == 1) + if (version == 1) { /// is_replicated and is_remote are not used bool is_replicated = true; @@ -80,7 +80,7 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) return false; } DB::assertChar('\n', buffer); - if (version == 1) + if (version == 1) { /// is_replicated and is_remote are not used bool is_replicated; @@ -121,7 +121,7 @@ Unfreezer::Unfreezer(ContextPtr context) : local_context(context), zookeeper() { const auto & config = local_context->getConfigRef(); static constexpr auto config_key = "enable_system_unfreeze"; - if (!config.getBool(config_key, false)) + if (!config.getBool(config_key, false)) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); } From 4c60921da4c355639a96722cec3277c615dfb268 Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Fri, 19 Aug 2022 08:59:13 +0300 Subject: [PATCH 102/582] Move config --- .../config.d/system_unfreeze.xml} | 0 tests/config/install.sh | 1 + 2 files changed, 1 insertion(+) rename tests/{queries/0_stateless/01417_freeze_partition_verbose.config.xml => config/config.d/system_unfreeze.xml} (100%) diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose.config.xml b/tests/config/config.d/system_unfreeze.xml similarity index 100% rename from tests/queries/0_stateless/01417_freeze_partition_verbose.config.xml rename to tests/config/config.d/system_unfreeze.xml diff --git a/tests/config/install.sh b/tests/config/install.sh index a9e66ebb633..6245e9c16b2 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -46,6 +46,7 @@ ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/ssl_certs.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/session_log.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ From f140cedd88380ab8cd7703011ebaae6112e9de86 Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Fri, 19 Aug 2022 10:41:19 +0300 Subject: [PATCH 103/582] fix xml --- tests/config/config.d/system_unfreeze.xml | 23 +---------------------- 1 file changed, 1 insertion(+), 22 deletions(-) diff --git a/tests/config/config.d/system_unfreeze.xml b/tests/config/config.d/system_unfreeze.xml index 7e13604b8e3..9d6e7279e09 100644 --- a/tests/config/config.d/system_unfreeze.xml +++ b/tests/config/config.d/system_unfreeze.xml @@ -1,25 +1,4 @@ - - - trace - true - - + true - 9000 - 0 - 0 - . - 0 - - - - - users.xml - - - - ./ - - From fdbe607515d795b9c8b17d42fe7875e1a8199c77 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Aug 2022 22:00:06 +0200 Subject: [PATCH 104/582] tests: enable back 02400_memory_accounting_on_error Signed-off-by: Azat Khuzhin --- .../0_stateless/02400_memory_accounting_on_error.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02400_memory_accounting_on_error.sql b/tests/queries/0_stateless/02400_memory_accounting_on_error.sql index 32046e854dd..f80c372b81c 100644 --- a/tests/queries/0_stateless/02400_memory_accounting_on_error.sql +++ b/tests/queries/0_stateless/02400_memory_accounting_on_error.sql @@ -1,4 +1,4 @@ -- max_block_size to avoid randomization --- SELECT * FROM generateRandom('i Array(Int8)', 1, 1, 1048577) LIMIT 65536 SETTINGS max_memory_usage='1Gi', max_block_size=65505, log_queries=1; -- { serverError MEMORY_LIMIT_EXCEEDED } --- SYSTEM FLUSH LOGS; --- SELECT * FROM system.query_log WHERE event_date >= yesterday() AND current_database = currentDatabase() AND memory_usage > 100e6; +SELECT * FROM generateRandom('i Array(Int8)', 1, 1, 1048577) LIMIT 65536 SETTINGS max_memory_usage='1Gi', max_block_size=65505, log_queries=1; -- { serverError MEMORY_LIMIT_EXCEEDED } +SYSTEM FLUSH LOGS; +SELECT * FROM system.query_log WHERE event_date >= yesterday() AND current_database = currentDatabase() AND memory_usage > 100e6 FORMAT JSONEachRow; From 28f2607d506847bd2ff6fce34f1e332386103d7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Aug 2022 22:04:10 +0200 Subject: [PATCH 105/582] Fix rare race in updating peak of memory tracker Previously it was possible that allocation, that had been done with locked MEMORY_LIMIT_EXCEEDED (LockMemoryExceptionInThread), will update the peak to the amount that will include some other allocations. Consider the following example (T1/T2 - different threads): Initial state: MemoryTracker::amount = 0 Possible race: LockMemoryExceptionInThread locker; T1: MemoryTracker::allocImpl(1MiB) T2: MemoryTracker::allocImpl(1PiB) will_be = 1T, amount = 1T will_be = 1T+1MiB, amount=1T+1MiB peak = will_be amount = 0 // revert due to exceeding limit throw(MEMORY_LIMIT_EXCEEDED) So as you can see T1 includes allocations from T2 into the peak. Note, that almost every place that has memory_usage column uses MemoryTracker::peak (not MemoryTracker::amount), this is true for query_log (for part_log column name is peak_memory_usage, and obviously it uses MemoryTracker::peak). Signed-off-by: Azat Khuzhin --- src/Common/MemoryTracker.cpp | 114 ++++++++++++++++++++--------------- 1 file changed, 66 insertions(+), 48 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index da3ce3cc3d2..29dbcdd28b0 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -166,27 +166,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } } - std::bernoulli_distribution fault(fault_probability); - if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded) - { - /// Revert - amount.fetch_sub(size, std::memory_order_relaxed); - - /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc - MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - - ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); - const auto * description = description_ptr.load(std::memory_order_relaxed); - throw DB::Exception( - DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - description ? " " : "", - description ? description : "", - formatReadableSizeWithBinarySuffix(will_be), - size, - formatReadableSizeWithBinarySuffix(current_hard_limit)); - } - + bool memory_limit_exceeded_ignored = false; bool allocation_traced = false; if (unlikely(current_profiler_limit && will_be > current_profiler_limit)) @@ -205,54 +185,92 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT allocation_traced = true; } - if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) + std::bernoulli_distribution fault(fault_probability); + if (unlikely(fault_probability && fault(thread_local_rng))) { - OvercommitResult overcommit_result = OvercommitResult::NONE; - if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed); overcommit_tracker_ptr != nullptr && query_tracker != nullptr) - overcommit_result = overcommit_tracker_ptr->needToStopQuery(query_tracker, size); - - if (overcommit_result != OvercommitResult::MEMORY_FREED) + if (memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded) { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}. OvercommitTracker decision: {}.", + "Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", description ? " " : "", description ? description : "", formatReadableSizeWithBinarySuffix(will_be), size, - formatReadableSizeWithBinarySuffix(current_hard_limit), - toDescription(overcommit_result)); + formatReadableSizeWithBinarySuffix(current_hard_limit)); + } + else + memory_limit_exceeded_ignored = true; + } + + + if (unlikely(current_hard_limit && will_be > current_hard_limit)) + { + if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) + { + OvercommitResult overcommit_result = OvercommitResult::NONE; + if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed); overcommit_tracker_ptr != nullptr && query_tracker != nullptr) + overcommit_result = overcommit_tracker_ptr->needToStopQuery(query_tracker, size); + + if (overcommit_result != OvercommitResult::MEMORY_FREED) + { + /// Revert + amount.fetch_sub(size, std::memory_order_relaxed); + + /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc + MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); + const auto * description = description_ptr.load(std::memory_order_relaxed); + throw DB::Exception( + DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}. OvercommitTracker decision: {}.", + description ? " " : "", + description ? description : "", + formatReadableSizeWithBinarySuffix(will_be), + size, + formatReadableSizeWithBinarySuffix(current_hard_limit), + toDescription(overcommit_result)); + } + else + { + // If OvercommitTracker::needToStopQuery returned false, it guarantees that enough memory is freed. + // This memory is already counted in variable `amount` in the moment of `will_be` initialization. + // Now we just need to update value stored in `will_be`, because it should have changed. + will_be = amount.load(std::memory_order_relaxed); + } + } + else + memory_limit_exceeded_ignored = true; + } + + bool peak_updated = false; + /// In case of MEMORY_LIMIT_EXCEEDED was ignored, will_be may include + /// memory of other allocations, that may fail but not reverted yet, and so + /// updating peak will be inaccurate. + if (!memory_limit_exceeded_ignored) + { + if (throw_if_memory_exceeded) + { + /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc + MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); + bool log_memory_usage = true; + peak_updated = updatePeak(will_be, log_memory_usage); } else { - // If OvercommitTracker::needToStopQuery returned false, it guarantees that enough memory is freed. - // This memory is already counted in variable `amount` in the moment of `will_be` initialization. - // Now we just need to update value stored in `will_be`, because it should have changed. - will_be = amount.load(std::memory_order_relaxed); + bool log_memory_usage = false; + peak_updated = updatePeak(will_be, log_memory_usage); } } - bool peak_updated; - if (throw_if_memory_exceeded) - { - /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc - MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - bool log_memory_usage = true; - peak_updated = updatePeak(will_be, log_memory_usage); - } - else - { - bool log_memory_usage = false; - peak_updated = updatePeak(will_be, log_memory_usage); - } - if (peak_updated && allocation_traced) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); From 056960819243a287338d90c4e71600659f503b3e Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Sat, 20 Aug 2022 12:30:39 +0800 Subject: [PATCH 106/582] Use toString --- src/Common/OpenTelemetryTraceContext.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index e2ac0934069..84ee9348ab0 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -16,7 +16,7 @@ void OpenTelemetrySpan::addAttribute(const std::string & name, UInt64 value) if (trace_id == UUID() || name.empty()) return; - this->attributes.push_back(Tuple{name, std::to_string(value)}); + this->attributes.push_back(Tuple{name, toString(value)}); } void OpenTelemetrySpan::addAttribute(const std::string & name, const std::string & value) From 54a530a4e068fc58153289b3d1d54e5247160f50 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Sat, 20 Aug 2022 14:22:33 +0800 Subject: [PATCH 107/582] Make sure right toString is linked --- src/Common/OpenTelemetryTraceContext.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 84ee9348ab0..21710403c8a 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { From ec2e671d202cd86094bb294d12e2e8a4317ab134 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Aug 2022 23:46:06 +0200 Subject: [PATCH 108/582] Remove ThreadStatus::untracked_memory_limit_increase It looks useless nowadays, because operator new cannot throw MEMORY_LIMIT_EXCEEDED today, and so any code that works on Exception is likely safe. Refs: #40249 Refs: #24483 Signed-off-by: Azat Khuzhin --- src/Common/CurrentMemoryTracker.cpp | 7 +------ src/Common/ThreadStatus.h | 2 -- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 921c244da21..720df07efb9 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -52,15 +52,10 @@ void CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) if (current_thread) { Int64 will_be = current_thread->untracked_memory + size; - Int64 limit = current_thread->untracked_memory_limit + current_thread->untracked_memory_limit_increase; - if (will_be > limit) + if (will_be > current_thread->untracked_memory_limit) { - /// Increase limit before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes - /// more. It could be useful to enlarge Exception message in rethrow logic. - current_thread->untracked_memory_limit_increase = current_thread->untracked_memory_limit; memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); - current_thread->untracked_memory_limit_increase = 0; current_thread->untracked_memory = 0; } else diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 2a4ffd229f2..0b01f43a226 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -135,8 +135,6 @@ public: Int64 untracked_memory = 0; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. Int64 untracked_memory_limit = 4 * 1024 * 1024; - /// Increase limit in case of exception. - Int64 untracked_memory_limit_increase = 0; /// Statistics of read and write rows/bytes Progress progress_in; From f8e2ab0a20684a01a5c7ece7c4f4f6dc8154f19d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 22 Aug 2022 08:21:32 -0300 Subject: [PATCH 109/582] Use FileReader::GetRecordBatchReader instead of FileReader::ReadRowGroup to parse Parquet --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 12fa9710c42..427c159314b 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -55,7 +55,16 @@ Chunk ParquetBlockInputFormat::generate() return res; std::shared_ptr table; - arrow::Status read_status = file_reader->ReadRowGroup(row_group_current, column_indices, &table); + + std::unique_ptr<::arrow::RecordBatchReader> rbr; + std::vector row_group_indices { row_group_current }; + arrow::Status get_batch_reader_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, &rbr); + + if (!get_batch_reader_status.ok()) + throw ParsingException{"Error while reading Parquet data: " + get_batch_reader_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; + + arrow::Status read_status = rbr->ReadAll(&table); + if (!read_status.ok()) throw ParsingException{"Error while reading Parquet data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; From e4532da317b915e7003fc3d5b1bd1823eec078f1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Aug 2022 19:28:49 +0200 Subject: [PATCH 110/582] fix test --- .../01730_distributed_group_by_no_merge_order_by_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index 62b578c21d6..3d6a25fe799 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -12,7 +12,7 @@ select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by n -- and the query with GROUP BY on remote servers will first do GROUP BY and then send the block, -- so the initiator will first receive all blocks from remotes and only after start merging, -- and will hit the memory limit. -select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='100Mi'; -- { serverError 241 } +select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='100Mi', max_block_size=1e12; -- { serverError 241 } -- with optimize_aggregation_in_order=1 remote servers will produce blocks more frequently, -- since they don't need to wait until the aggregation will be finished, From b31342ec2cf24e0487466cde11143c8ba6063ffc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Aug 2022 19:29:48 +0200 Subject: [PATCH 111/582] fix --- src/Interpreters/Aggregator.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 84708fc72ea..04fa2aaddae 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1902,6 +1902,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena { out_cols->key_columns[0]->insertDefault(); insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena); + data.hasNullKeyData() = false; } } @@ -1974,6 +1975,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a out_cols->aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); data.getNullKeyData() = nullptr; + data.hasNullKeyData() = false; } } From d81462c0952c35d5d4345883b881c16136d690de Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Tue, 23 Aug 2022 15:08:35 +0800 Subject: [PATCH 112/582] check all ArgumentsDataTypes and use equals --- src/Functions/UniqTheta/FunctionsUniqTheta.h | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.h b/src/Functions/UniqTheta/FunctionsUniqTheta.h index 9231f8e24c2..313f9a15085 100644 --- a/src/Functions/UniqTheta/FunctionsUniqTheta.h +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.h @@ -86,13 +86,25 @@ namespace DB "Second argument for function " + getName() + " must be a uniqTheta but it has type " + arguments[1]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - //todo:check - if (sketch_type0->getArgumentsDataTypes()[0]->getTypeId() != sketch_type1->getArgumentsDataTypes()[0]->getTypeId()) + const DataTypes & arg_data_types0 = sketch_type0->getArgumentsDataTypes(); + const DataTypes & arg_data_types1 = sketch_type1->getArgumentsDataTypes(); + if (arg_data_types0.size() != arg_data_types1.size()) throw Exception( - "The nested type in uniqThetas must be the same, but one is " + sketch_type0->getArgumentsDataTypes()[0]->getName() - + ", and the other is " + sketch_type1->getArgumentsDataTypes()[0]->getName(), + "The nested type in uniqThetas must be the same length, but one is " + std::to_string(arg_data_types0.size()) + + ", and the other is " + std::to_string(arg_data_types1.size()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + size_t types_size = arg_data_types0.size(); + for (size_t i = 0; i < types_size; ++i) + { + if (!arg_data_types0[i]->equals(*arg_data_types1[i])) + throw Exception( + "The " + std::to_string(i) + "th nested type in uniqThetas must be the same, but one is " + arg_data_types0[i]->getName() + + ", and the other is " + arg_data_types1[i]->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return arguments[0]; } From 9c371d518f02c6c4c79f600481bb4aaa0e1585fb Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Tue, 23 Aug 2022 13:53:06 +0300 Subject: [PATCH 113/582] fix enable_unfreeze handler and sanitizer hints --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Storages/Freeze.cpp | 20 ++++++++++--------- src/Storages/Freeze.h | 2 +- .../test_merge_tree_s3/configs/config.xml | 1 + .../configs/config.d/s3.xml | 2 +- 5 files changed, 15 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b50ff4d32c3..62c94fa0ae4 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -501,7 +501,7 @@ BlockIO InterpreterSystemQuery::execute() { getContext()->checkAccess(AccessType::SYSTEM_UNFREEZE); /// The result contains information about deleted parts as a table. It is for compatibility with ALTER TABLE UNFREEZE query. - result = Unfreezer(getContext()).unfreeze(query.backup_name); + result = Unfreezer(getContext()).systemUnfreeze(query.backup_name); break; } default: diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 99a27553081..5a559ae9e2b 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -117,21 +117,23 @@ String FreezeMetaData::getFileName(const String & path) return fs::path(path) / "frozen_metadata.txt"; } -Unfreezer::Unfreezer(ContextPtr context) : local_context(context), zookeeper() +Unfreezer::Unfreezer(ContextPtr context) : local_context(context) { + if (local_context->hasZooKeeper()) + zookeeper = local_context->getZooKeeper(); +} + +BlockIO Unfreezer::systemUnfreeze(const String & backup_name) +{ + LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name)); + const auto & config = local_context->getConfigRef(); static constexpr auto config_key = "enable_system_unfreeze"; if (!config.getBool(config_key, false)) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); } - if (local_context->hasZooKeeper()) - zookeeper = local_context->getZooKeeper(); -} - -BlockIO Unfreezer::unfreeze(const String & backup_name) -{ - LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name)); + auto disks_map = local_context->getDisksMap(); Disks disks; for (auto & [name, disk]: disks_map) @@ -145,7 +147,7 @@ BlockIO Unfreezer::unfreeze(const String & backup_name) for (const auto & disk: disks) { - for (auto store_path: store_paths) + for (const auto& store_path: store_paths) { if (!disk->exists(store_path)) continue; diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index cd8dcdf824d..bd8a8e38ec2 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -34,7 +34,7 @@ class Unfreezer public: Unfreezer(ContextPtr context); PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory); - BlockIO unfreeze(const String & backup_name); + BlockIO system_unfreeze(const String & backup_name); private: ContextPtr local_context; zkutil::ZooKeeperPtr zookeeper; diff --git a/tests/integration/test_merge_tree_s3/configs/config.xml b/tests/integration/test_merge_tree_s3/configs/config.xml index 8e7eada42ed..5e5cf1f37e1 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.xml @@ -16,4 +16,5 @@ 500 ./clickhouse/ users.xml + true diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index 181144b0473..ecab025304b 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -1,5 +1,5 @@ - + true From 4d081543bb2b20c28cd519d99815835855dc7fec Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Tue, 23 Aug 2022 13:59:02 +0300 Subject: [PATCH 114/582] fix underscore --- src/Storages/Freeze.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index bd8a8e38ec2..a64be7465dd 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -34,7 +34,7 @@ class Unfreezer public: Unfreezer(ContextPtr context); PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory); - BlockIO system_unfreeze(const String & backup_name); + BlockIO systemUnfreeze(const String & backup_name); private: ContextPtr local_context; zkutil::ZooKeeperPtr zookeeper; From c2ba9eacd4c7cc20551611e09032529a1e6b51fa Mon Sep 17 00:00:00 2001 From: Vadim Volodin Date: Tue, 23 Aug 2022 14:43:10 +0300 Subject: [PATCH 115/582] fix style --- src/Storages/Freeze.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index 5a559ae9e2b..f8fe1247816 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -133,7 +133,7 @@ BlockIO Unfreezer::systemUnfreeze(const String & backup_name) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key); } - + auto disks_map = local_context->getDisksMap(); Disks disks; for (auto & [name, disk]: disks_map) From ac3cf0beb4cce89bee54dec35848031ff64449aa Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 23 Aug 2022 20:58:57 +0800 Subject: [PATCH 116/582] add read_only option Signed-off-by: Lloyd-Pottiger --- .../integrations/embedded-rocksdb.md | 4 ++- .../RocksDB/StorageEmbeddedRocksDB.cpp | 34 +++++++++++++++---- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 5 ++- 3 files changed, 35 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 5663b1ce834..3c9cdf2ec67 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -15,12 +15,14 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) ENGINE = EmbeddedRocksDB([ttl]) PRIMARY KEY(primary_key_name) +) ENGINE = EmbeddedRocksDB([ttl, rocksdb_dir, read_only]) PRIMARY KEY(primary_key_name) ``` Engine parameters: - `ttl` - time to live for values. TTL is accepted in seconds. If TTL is 0, regular RocksDB instance is used (without TTL). +- `rocksdb_dir` - path to the directory of an existed RocksDB. Open the table with the specified `rocksdb_dir`. +- `read_only` - when `read_only` is set to true, read-only mode is used. For storage with TTL, compaction will not be triggered (neither manual nor automatic), so no expired entries are removed. - `primary_key_name` – any column name in the column list. - `primary key` must be specified, it supports only one column in the primary key. The primary key will be serialized in binary as a `rocksdb key`. - columns other than the primary key will be serialized in binary as `rocksdb` value in corresponding order. diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index c9ddd9147b9..5a162ad2a12 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -166,14 +166,23 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, bool attach, ContextPtr context_, const String & primary_key_, - Int32 ttl_) + Int32 ttl_, + String rocksdb_dir_, + bool read_only_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , primary_key{primary_key_} , ttl(ttl_) + , read_only(read_only_) { setInMemoryMetadata(metadata_); - rocksdb_dir = context_->getPath() + relative_data_path_; + if (rocksdb_dir_.empty()) + { + rocksdb_dir = context_->getPath() + relative_data_path_; + } + else { + rocksdb_dir = rocksdb_dir_; + } if (!attach) { fs::create_directories(rocksdb_dir); @@ -280,7 +289,14 @@ void StorageEmbeddedRocksDB::initDB() else { rocksdb::DB * db; - status = rocksdb::DB::Open(merged, rocksdb_dir, &db); + if (read_only) + { + status = rocksdb::DB::OpenForReadOnly(merged, rocksdb_dir, &db); + } + else + { + status = rocksdb::DB::Open(merged, rocksdb_dir, &db); + } if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}", @@ -351,15 +367,21 @@ static StoragePtr create(const StorageFactory::Arguments & args) { // TODO custom RocksDBSettings, table function auto engine_args = args.engine_args; - if (engine_args.size() > 1) + if (engine_args.size() > 3) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Engine {} requires at most 1 parameter. ({} given). Correct usage: EmbeddedRocksDB([ttl])", + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Engine {} requires at most 3 parameters. ({} given). Correct usage: EmbeddedRocksDB([ttl, rocksdb_dir, read_only])", args.engine_name, engine_args.size()); } Int32 ttl{0}; + String rocksdb_dir; + bool read_only{false}; if (!engine_args.empty()) ttl = checkAndGetLiteralArgument(engine_args[0], "ttl"); + if (engine_args.size() > 1) + rocksdb_dir = checkAndGetLiteralArgument(engine_args[1], "rocksdb_dir"); + if (engine_args.size() > 2) + read_only = checkAndGetLiteralArgument(engine_args[2], "read_only"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -374,7 +396,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); } - return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl); + return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl, rocksdb_dir, read_only); } std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index ab87eac3e66..604976cd402 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -33,7 +33,9 @@ public: bool attach, ContextPtr context_, const String & primary_key_, - Int32 ttl_ = 0); + Int32 ttl_ = 0, + String rocksdb_dir_ = "", + bool read_only_ = false); std::string getName() const override { return "EmbeddedRocksDB"; } @@ -82,6 +84,7 @@ private: mutable std::shared_mutex rocksdb_ptr_mx; String rocksdb_dir; Int32 ttl; + bool read_only; void initDB(); }; From fc33941959e552edc4c2081d08a2e86422bbd85b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Aug 2022 13:15:31 +0000 Subject: [PATCH 117/582] Add support for shared drop/create --- src/Common/ErrorCodes.cpp | 1 + src/Common/ZooKeeper/ZooKeeper.cpp | 16 +- src/Common/ZooKeeper/ZooKeeper.h | 5 +- src/Storages/StorageKeeperMap.cpp | 340 ++++++++++-------- src/Storages/StorageKeeperMap.h | 62 +++- .../queries/0_stateless/02381_keeper_map.sql | 1 - 6 files changed, 255 insertions(+), 170 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f8d8deab08b..a30d6814516 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -635,6 +635,7 @@ M(664, ACCESS_STORAGE_DOESNT_ALLOW_BACKUP) \ M(665, CANNOT_CONNECT_NATS) \ M(666, CANNOT_USE_CACHE) \ + M(667, INVALID_STATE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 74945e3b50c..b47ab35820e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -720,7 +720,7 @@ void ZooKeeper::removeChildren(const std::string & path) } -void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & keep_child_node) +void ZooKeeper::removeChildrenRecursive(const std::string & path, const std::string_view keep_child_node) { Strings children = getChildren(path); while (!children.empty()) @@ -728,9 +728,11 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - removeChildrenRecursive(fs::path(path) / children.back()); if (likely(keep_child_node.empty() || keep_child_node != children.back())) + { + removeChildrenRecursive(fs::path(path) / children.back()); ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); + } children.pop_back(); } multi(ops); @@ -754,13 +756,13 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); + /// Will try to avoid recursive getChildren calls if child_path probably has no children. + /// It may be extremely slow when path contain a lot of leaf children. + if (!probably_flat) + tryRemoveChildrenRecursive(child_path); + if (likely(keep_child_node.empty() || keep_child_node != children.back())) { - /// Will try to avoid recursive getChildren calls if child_path probably has no children. - /// It may be extremely slow when path contain a lot of leaf children. - if (!probably_flat) - tryRemoveChildrenRecursive(child_path); - batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 416fc78d814..d16f1f96271 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -232,9 +232,8 @@ public: void tryRemoveRecursive(const std::string & path); /// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself. - /// If keep_child_node is not empty, this method will not remove path/keep_child_node (but will remove its subtree). - /// It can be useful to keep some child node as a flag which indicates that path is currently removing. - void removeChildrenRecursive(const std::string & path, const String & keep_child_node = {}); + /// If keep_child_node is not empty, this method will not remove path/keep_child_node and its subtree. + void removeChildrenRecursive(const std::string & path, std::string_view keep_child_node = {}); /// If probably_flat is true, this method will optimistically try to remove children non-recursive /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child. /// Returns true if no kind of fallback happened. diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 962d008882e..3477f29fc10 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -23,9 +23,10 @@ #include #include -#include +#include "Common/Exception.h" #include #include +#include #include #include @@ -123,6 +124,15 @@ struct ZooKeeperLock client->tryRemove(sequence_path); } + // release the ownership and return the path + // for the lock + std::string release() + { + assert(locked); + locked = false; + return std::move(sequence_path); + } + private: std::string lock_path; std::string sequence_path; @@ -178,57 +188,17 @@ public: { auto & zookeeper = storage.getClient(); - auto keys_limit = storage.keysLimit(); - Coordination::Requests requests; - if (!keys_limit) + for (const auto & [key, value] : new_values) { - for (const auto & [key, value] : new_values) - { - auto path = storage.fullPathForKey(key); + auto path = storage.fullPathForKey(key); - if (zookeeper->exists(path)) - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - else - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); - } + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); } - else - { - ZooKeeperLock lock(storage.lockPath(), zookeeper); - - auto children = zookeeper->getChildren(storage.rootKeeperPath()); - std::unordered_set children_set(children.begin(), children.end()); - - size_t created_nodes = 0; - for (const auto & [key, value] : new_values) - { - auto path = storage.fullPathForKey(key); - - if (children_set.contains(key)) - { - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - } - else - { - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); - ++created_nodes; - } - } - - size_t keys_num_after_insert = children.size() - 1 + created_nodes; - if (keys_limit && keys_num_after_insert > keys_limit) - { - throw Exception( - DB::ErrorCodes::BAD_ARGUMENTS, - "Cannot insert values. {} key would be created setting the total keys number to {} exceeding the limit of {}", - created_nodes, - keys_num_after_insert, - keys_limit); - } - } - zookeeper->multi(requests); } @@ -290,31 +260,20 @@ public: } }; -namespace -{ - -zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) -{ - if (hosts == default_host) - return context->getZooKeeper()->startNewSession(); - - return std::make_shared(hosts); -} - -} - StorageKeeperMap::StorageKeeperMap( - ContextPtr context, + ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view root_path_, - const std::string & hosts, - bool create_missing_root_path, - size_t keys_limit_, - bool remove_existing_data) - : IStorage(table_id), root_path(root_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)), log(&Poco::Logger::get("StorageKeeperMap")) + const std::string & root_path_, + bool create_missing_root_path) + : IStorage(table_id) + , WithContext(context_->getGlobalContext()) + , root_path(zkutil::extractZooKeeperPath(root_path_, false)) + , primary_key(primary_key_) + , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) + , log(&Poco::Logger::get("StorageKeeperMap")) { setInMemoryMetadata(metadata); @@ -323,28 +282,21 @@ StorageKeeperMap::StorageKeeperMap( if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); - auto client = getClient(); - std::filesystem::path root_path_fs{root_path}; - auto metadata_path_fs = root_path_fs / "__ch_metadata"; + auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; lock_path = metadata_path_fs / "lock"; - auto keys_limit_path = metadata_path_fs / "keys_limit"; + tables_path = metadata_path_fs / "tables"; + dropped_path = metadata_path_fs / "dropped"; if (attach) { - // validate all metadata nodes are present - Coordination::Requests requests; - - requests.push_back(zkutil::makeCheckRequest(root_path, -1)); - requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); - requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); - requests.push_back(zkutil::makeCheckRequest(keys_limit_path, -1)); - - client->multi(requests); + checkTable(); return; } + auto client = getClient(); + if (root_path != "/" && !client->exists(root_path)) { if (!create_missing_root_path) @@ -372,27 +324,62 @@ StorageKeeperMap::StorageKeeperMap( } } - Coordination::Stat stats; - auto exists = client->exists(root_path, &stats); - if (!exists) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); - if (stats.numChildren != 0) + ZooKeeperLock lock(lock_path, client, true); + + bool locked = false; + try { - if (!remove_existing_data) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes. Set remove_existing_data to 1 to clear children nodes", root_path); - - LOG_INFO(log, "Removing children for {} because remove_existing_data was set to true.", root_path); - client->removeChildrenRecursive(root_path); + lock.lock(); + locked = true; + } + catch (const Coordination::Exception & e) + { + if (e.code != Coordination::Error::ZNONODE) + throw; } - // create metadata nodes - Coordination::Requests create_requests; - create_requests.push_back(zkutil::makeCreateRequest(metadata_path, "", zkutil::CreateMode::Persistent)); - create_requests.push_back(zkutil::makeCreateRequest(lock_path, "", zkutil::CreateMode::Persistent)); - create_requests.push_back(zkutil::makeCreateRequest(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent)); + { + Coordination::Stat stats; + auto exists = client->exists(root_path, &stats); + if (!exists) + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); + } - client->multi(create_requests); + + const auto set_tables_num = [&] + { + std::string tables_num_str; + auto exists = client->tryGet(tables_path, tables_num_str); + + if (!exists) + { + client->create(tables_path, toString(1), zkutil::CreateMode::Persistent); + } + else + { + uint64_t tables_num = parse(tables_num_str); + client->set(tables_path, toString(tables_num + 1), zkutil::CreateMode::Persistent); + } + }; + + // we have metadata and lock nodes, because they can't be deleted with the lock acquired + if (locked) + { + if (client->exists(dropped_path)) + { + client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + client->remove(dropped_path); + } + + set_tables_num(); + return; + } + + client->createIfNotExists(metadata_path, ""); + client->createIfNotExists(lock_path, ""); + lock.lock(); + set_tables_num(); } @@ -400,11 +387,12 @@ Pipe StorageKeeperMap::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { + checkTable(); storage_snapshot->check(column_names); FieldVectorPtr filtered_keys; @@ -412,7 +400,7 @@ Pipe StorageKeeperMap::read( Block sample_block = storage_snapshot->metadata->getSampleBlock(); auto primary_key_type = sample_block.getByName(primary_key).type; - std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context); + std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context_); const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { @@ -451,29 +439,62 @@ Pipe StorageKeeperMap::read( SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { + checkTable(); return std::make_shared(*this, metadata_snapshot); } -void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) +void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { + checkTable(); auto client = getClient(); - ZooKeeperLock keeper_lock(lockPath(), client); - client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ true, getBaseName(metadata_path)); + ZooKeeperLock keeper_lock(lock_path, client); + client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); } void StorageKeeperMap::drop() { + checkTable(); auto client = getClient(); - client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ false); + ZooKeeperLock lock(lock_path, client); + + auto tables_num_str = client->get(tables_path); + auto tables_num = parse(tables_num_str); + --tables_num; + + if (tables_num > 0) + { + client->set(tables_path, toString(tables_num)); + return; + } + + client->create(dropped_path, "", zkutil::CreateMode::Persistent); + + client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + + Coordination::Requests requests; + // we need to release lock and delete everything at the same time + // so create doesn't take a lock while delete is being run + auto current_lock_path = lock.release(); + requests.push_back(zkutil::makeRemoveRequest(current_lock_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(lock_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(tables_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(dropped_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(metadata_path, -1)); + + client->multi(requests); } zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { - if (zookeeper_client->expired()) + std::lock_guard lock{zookeeper_mutex}; + if (!zookeeper_client || zookeeper_client->expired()) { - zookeeper_client = zookeeper_client->startNewSession(); - zookeeper_client->sync("/"); + zookeeper_client = nullptr; + if (zookeeper_name == "default") + zookeeper_client = getContext()->getZooKeeper(); + else + zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name); } return zookeeper_client; @@ -494,9 +515,45 @@ const std::string & StorageKeeperMap::lockPath() const return lock_path; } -UInt64 StorageKeeperMap::keysLimit() const +std::optional StorageKeeperMap::isTableValid() const { - return keys_limit; + std::lock_guard lock{init_mutex}; + if (table_is_valid.has_value()) + return *table_is_valid; + + try + { + // validate all metadata nodes are present + Coordination::Requests requests; + + requests.push_back(zkutil::makeCheckRequest(root_path, -1)); + requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); + requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); + requests.push_back(zkutil::makeCheckRequest(tables_path, -1)); + + Coordination::Responses responses; + auto client = getClient(); + auto res = client->tryMulti(requests, responses); + table_is_valid = res == Coordination::Error::ZOK; + } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log); + + std::array retriable_errors{ + Coordination::Error::ZCONNECTIONLOSS, Coordination::Error::ZSESSIONEXPIRED, Coordination::Error::ZOPERATIONTIMEOUT}; + bool retriable_error + = std::any_of(retriable_errors.begin(), retriable_errors.end(), [&](const auto error) { return error == e.code; }); + if (!retriable_error) + table_is_valid = false; + } + catch (const Exception &) + { + tryLogCurrentException(log); + table_is_valid = false; + } + + return table_is_valid; } Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const @@ -586,53 +643,40 @@ Block StorageKeeperMap::getSampleBlock(const Names &) const namespace { -StoragePtr create(const StorageFactory::Arguments & args) -{ - ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 4) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-5 arguments:\n" - "root_path: path in the Keeper where the values will be stored (required)\n" - "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" - "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" - "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", - default_host); + StoragePtr create(const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 4) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1-5 arguments:\n" + "root_path: path in the Keeper where the values will be stored (required)\n" + "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" + "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", + "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", + default_host); - auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); + auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); - std::string hosts = "default"; - if (engine_args.size() > 1) - hosts = checkAndGetLiteralArgument(engine_args[1], "hosts"); + bool create_missing_root_path = true; + if (engine_args.size() > 1) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); - size_t keys_limit = 0; - if (engine_args.size() > 2) - keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); - bool create_missing_root_path = true; - if (engine_args.size() > 3) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[3], "create_missing_root_path"); + if (!args.storage_def->primary_key) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - bool remove_existing_data = false; - if (engine_args.size() > 4) - remove_existing_data = checkAndGetLiteralArgument(engine_args[4], "remove_existing_data"); + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - StorageInMemoryMetadata metadata; - metadata.setColumns(args.columns); - metadata.setConstraints(args.constraints); - - if (!args.storage_def->primary_key) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); - auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); - if (primary_key_names.size() != 1) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - - return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); -} + return std::make_shared( + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path); + } } void registerStorageKeeperMap(StorageFactory & factory) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 5f62d58bfa3..e09e8923e5c 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -14,21 +14,23 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_STATE; +} + // KV store using (Zoo|CH)Keeper -class StorageKeeperMap final : public IStorage, public IKeyValueEntity +class StorageKeeperMap final : public IStorage, public IKeyValueEntity, WithContext { public: StorageKeeperMap( - ContextPtr context, + ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view root_path_, - const std::string & hosts, - bool create_missing_root_path, - size_t keys_limit, - bool remove_existing_data); + const std::string & root_path_, + bool create_missing_root_path); Pipe read( const Names & column_names, @@ -41,7 +43,7 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; - void truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; void drop() override; std::string getName() const override { return "KeeperMap"; } @@ -65,16 +67,54 @@ public: std::string fullPathForKey(std::string_view key) const; const std::string & lockPath() const; - UInt64 keysLimit() const; + std::optional isTableValid() const; + + template + void checkTable() const + { + auto is_table_valid = isTableValid(); + if (!is_table_valid.has_value()) + { + static constexpr std::string_view error_msg = "Failed to activate table because of connection issues. It will be activated " + "once a connection is established and metadata is verified"; + if constexpr (throw_on_error) + throw Exception(ErrorCodes::INVALID_STATE, error_msg); + else + { + LOG_ERROR(log, fmt::runtime(error_msg)); + return; + } + } + + if (!*is_table_valid) + { + static constexpr std::string_view error_msg + = "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table"; + if constexpr (throw_on_error) + throw Exception(ErrorCodes::INVALID_STATE, error_msg); + else + { + LOG_ERROR(log, fmt::runtime(error_msg)); + return; + } + } + } private: std::string root_path; std::string primary_key; std::string metadata_path; std::string lock_path; - UInt64 keys_limit{0}; + std::string tables_path; + std::string dropped_path; - mutable zkutil::ZooKeeperPtr zookeeper_client; + std::string zookeeper_name; + + mutable std::mutex zookeeper_mutex; + mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; + + mutable std::mutex init_mutex; + mutable std::optional table_is_valid; Poco::Logger * log; }; diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql index 9a4f0b367f7..903d530299c 100644 --- a/tests/queries/0_stateless/02381_keeper_map.sql +++ b/tests/queries/0_stateless/02381_keeper_map.sql @@ -30,7 +30,6 @@ INSERT INTO 02381_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), INSERT INTO 02381_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; - SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test_memory) B USING a ORDER BY a; TRUNCATE TABLE 02381_test; From 52ec268d75bce0a29c6883d829eaac63a76cc5bb Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 23 Aug 2022 21:26:15 +0800 Subject: [PATCH 118/582] format check Signed-off-by: Lloyd-Pottiger --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 5a162ad2a12..1a47fa6ba33 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -180,7 +180,8 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, { rocksdb_dir = context_->getPath() + relative_data_path_; } - else { + else + { rocksdb_dir = rocksdb_dir_; } if (!attach) From f0acc4a027dc07d75aafad569b672dea6ac8eb48 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 23 Aug 2022 16:49:46 +0200 Subject: [PATCH 119/582] Check what will happen if disable cache --- docker/test/stress/run.sh | 4 ++-- tests/config/config.d/storage_conf.xml | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index d23a96c508c..a6df29f91fe 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -198,8 +198,8 @@ start clickhouse-client --query "SHOW TABLES FROM datasets" clickhouse-client --query "SHOW TABLES FROM test" -clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" -clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" +clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" +clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index de41478f74a..13e3172101f 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -2,6 +2,13 @@ + + s3 + s3_simple/ + http://localhost:11111/test/00170_test/ + clickhouse + clickhouse + s3 s3_disk/ @@ -148,6 +155,13 @@ + + +
+ s3_simple +
+
+
From 0b6e1462a25473e21d36a50e5f8c5bbe4003284d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 23 Aug 2022 16:51:35 +0200 Subject: [PATCH 120/582] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index a6df29f91fe..6a019a721b3 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -200,7 +200,7 @@ clickhouse-client --query "SHOW TABLES FROM test" clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" -clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" +clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" From 365a65219e2d3e83e63a2e326b86a91ae87f3339 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Aug 2022 19:20:43 +0200 Subject: [PATCH 121/582] try fix tests --- .../01064_incremental_streaming_from_2_src_with_feedback.sql | 3 +-- .../0_stateless/01281_group_by_limit_memory_tracking.sh | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index a63aa768051..9a439180265 100644 --- a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -1,4 +1,5 @@ SET joined_subquery_requires_alias = 0; +SET max_threads = 1; -- incremental streaming usecase -- that has sense only if data filling order has guarantees of chronological order @@ -23,8 +24,6 @@ AS FROM numbers(50000) GROUP BY id; -OPTIMIZE TABLE target_table FINAL; - -- source table #1 CREATE TABLE logins ( diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh index 2f4164ee0d1..7f353b84a80 100755 --- a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings, no-s3-storage +# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings, no-s3-storage, no-msan # Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently # Regression for MemoryTracker that had been incorrectly accounted From efc6a60a60b13162a3d9300da95526d052974b4d Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 24 Aug 2022 15:24:21 +0800 Subject: [PATCH 122/582] Clean code --- src/Common/OpenTelemetryTraceContext.cpp | 84 ++++++++++--------- src/Common/OpenTelemetryTraceContext.h | 64 +++++++------- src/Common/ThreadPool.cpp | 10 ++- src/Interpreters/Session.h | 2 +- src/Interpreters/executeQuery.cpp | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 4 +- 6 files changed, 89 insertions(+), 77 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 21710403c8a..293439a129c 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -12,59 +12,71 @@ namespace DB thread_local OpenTelemetryThreadTraceContext current_thread_trace_context; -void OpenTelemetrySpan::addAttribute(const std::string & name, UInt64 value) +void OpenTelemetrySpan::addAttribute(std::string_view name, UInt64 value) { - if (trace_id == UUID() || name.empty()) + if (!this->isTraceEnabled() || name.empty()) return; this->attributes.push_back(Tuple{name, toString(value)}); } -void OpenTelemetrySpan::addAttribute(const std::string & name, const std::string & value) +void OpenTelemetrySpan::addAttributeIfNotZero(std::string_view name, UInt64 value) { - if (trace_id == UUID() || name.empty() || value.empty()) + if (value != 0) + addAttribute(name, value); +} + +void OpenTelemetrySpan::addAttribute(std::string_view name, std::string_view value) +{ + if (!this->isTraceEnabled() || name.empty()) return; this->attributes.push_back(Tuple{name, value}); } -void OpenTelemetrySpan::addAttribute(const std::string & name, std::function value_supplier) +void OpenTelemetrySpan::addAttribute(std::string_view name, std::function value_supplier) { if (!this->isTraceEnabled() || !value_supplier) return; - std::string value = value_supplier(); + String value = value_supplier(); if (value.empty()) return; this->attributes.push_back(Tuple{name, value}); } -void OpenTelemetrySpan::addAttribute(const Exception & e) +void OpenTelemetrySpan::addAttribute(const Exception & e) noexcept { - if (trace_id == UUID()) + if (!this->isTraceEnabled()) return; - this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); -} - -void OpenTelemetrySpan::addAttribute(std::exception_ptr e) -{ - if (trace_id == UUID() || e == nullptr) - return; - - this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); -} - -OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_name) -{ - if (current_thread_trace_context.trace_id == UUID()) + try { - this->trace_id = 0; - this->span_id = 0; - this->parent_span_id = 0; + this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); } - else + catch(...) + { + } +} + +void OpenTelemetrySpan::addAttribute(std::exception_ptr e) noexcept +{ + if (!this->isTraceEnabled() || e == nullptr) + return; + + try + { + this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); + } + catch(...) + { + } +} + +OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(std::string_view _operation_name) +{ + if (current_thread_trace_context.isTraceEnabled()) { this->trace_id = current_thread_trace_context.trace_id; this->parent_span_id = current_thread_trace_context.span_id; @@ -80,10 +92,10 @@ OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_ void OpenTelemetrySpanHolder::finish() { - if (trace_id == UUID()) + if (!this->isTraceEnabled()) return; - // First of all, return old value of current span. + // First of all, restore old value of current span. assert(current_thread_trace_context.span_id == span_id); current_thread_trace_context.span_id = parent_span_id; @@ -114,8 +126,7 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() finish(); } - -bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, std::string & error) +bool OpenTelemetryTraceContext::parseTraceparentHeader(std::string_view traceparent, String & error) { trace_id = 0; @@ -174,8 +185,7 @@ bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & trace return true; } - -std::string OpenTelemetryTraceContext::composeTraceparentHeader() const +String OpenTelemetryTraceContext::composeTraceparentHeader() const { // This span is a parent for its children, so we specify this span_id as a // parent id. @@ -198,13 +208,13 @@ void OpenTelemetryThreadTraceContext::reset() { this->trace_id = UUID(); this->span_id = 0; - this->trace_flags = 0; + this->trace_flags = TRACE_FLAG_NONE; this->tracestate = ""; this->span_log.reset(); } OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( - const std::string & _operation_name, + std::string_view _operation_name, OpenTelemetryTraceContext _parent_trace_context, const Settings * settings_ptr, const std::weak_ptr & _span_log) @@ -233,9 +243,6 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( return; } - this->root_span.trace_id = UUID(); - this->root_span.span_id = 0; - if (!_parent_trace_context.isTraceEnabled()) { if (settings_ptr == nullptr) @@ -267,7 +274,7 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( /// set up trace context on current thread current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; - current_thread_trace_context.trace_flags = 1; + current_thread_trace_context.trace_flags = TRACE_FLAG_SAMPLED; current_thread_trace_context.span_log = _span_log; } @@ -299,5 +306,4 @@ OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() } } - } diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index abb69880b3a..edc23efd551 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -9,25 +9,23 @@ struct Settings; struct OpenTelemetrySpan { - UUID trace_id; - UInt64 span_id; - UInt64 parent_span_id; - std::string operation_name; - UInt64 start_time_us; - UInt64 finish_time_us; + UUID trace_id{}; + UInt64 span_id = 0; + UInt64 parent_span_id = 0; + String operation_name; + UInt64 start_time_us = 0; + UInt64 finish_time_us = 0; Map attributes; - void addAttribute(const std::string& name, UInt64 value); - void addAttributeIfNotZero(const std::string& name, UInt64 value) - { - if (value != 0) - addAttribute(name, value); - } + void addAttribute(std::string_view name, UInt64 value); + void addAttributeIfNotZero(std::string_view name, UInt64 value); + void addAttribute(std::string_view name, std::string_view value); + void addAttribute(std::string_view name, std::function value_supplier); - void addAttribute(const std::string& name, const std::string& value); - void addAttribute(const std::string& name, std::function value_supplier); - void addAttribute(const Exception & e); - void addAttribute(std::exception_ptr e); + /// Following two methods are declared as noexcept to make sure they're exception safe + /// This is because they're usually called in exception handler + void addAttribute(const Exception & e) noexcept; + void addAttribute(std::exception_ptr e) noexcept; bool isTraceEnabled() const { @@ -35,11 +33,15 @@ struct OpenTelemetrySpan } }; -class Context; -using ContextPtr = std::shared_ptr; - class OpenTelemetrySpanLog; +/// See https://www.w3.org/TR/trace-context/ for trace_flags definition +enum TraceFlags : UInt8 +{ + TRACE_FLAG_NONE = 0, + TRACE_FLAG_SAMPLED = 1, +}; + // The runtime info we need to create new OpenTelemetry spans. struct OpenTelemetryTraceContext { @@ -48,11 +50,11 @@ struct OpenTelemetryTraceContext // The incoming tracestate header and the trace flags, we just pass them // downstream. See https://www.w3.org/TR/trace-context/ String tracestate; - UInt8 trace_flags = 0; + UInt8 trace_flags = TRACE_FLAG_NONE; // Parse/compose OpenTelemetry traceparent header. - bool parseTraceparentHeader(const std::string & traceparent, std::string & error); - std::string composeTraceparentHeader() const; + bool parseTraceparentHeader(std::string_view traceparent, String & error); + String composeTraceparentHeader() const; bool isTraceEnabled() const { @@ -60,7 +62,7 @@ struct OpenTelemetryTraceContext } }; -/// tracing context kept on thread local +/// Tracing context kept on thread local struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext { OpenTelemetryThreadTraceContext& operator =(const OpenTelemetryTraceContext& context) @@ -73,6 +75,8 @@ struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext static const OpenTelemetryThreadTraceContext& current(); + /// Use weak_ptr instead of shared_ptr to hold a reference to the underlying system.opentelemetry_span_log table + /// Since this object is kept on threads and passed across threads, a weak_ptr is more safe to prevent potential leak std::weak_ptr span_log; }; @@ -83,11 +87,11 @@ struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext /// but it's not encourage to do so because this is only a protection in case of code changes. struct OpenTelemetryThreadTraceContextScope { - // forbidden copy ctor and assignment to make the destructor safe + /// Forbidden copy ctor and assignment to make the destructor safe OpenTelemetryThreadTraceContextScope(const OpenTelemetryThreadTraceContextScope& scope) = delete; OpenTelemetryThreadTraceContextScope& operator =(const OpenTelemetryThreadTraceContextScope& scope) = delete; - OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, const OpenTelemetryTraceContext& _parent_trace_context, const std::weak_ptr& _log) : OpenTelemetryThreadTraceContextScope(_operation_name, @@ -98,7 +102,7 @@ struct OpenTelemetryThreadTraceContextScope } /// Initialize a tracing context on a child thread based on the context from the parent thread - OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, const OpenTelemetryThreadTraceContext& _parent_thread_trace_context) : OpenTelemetryThreadTraceContextScope(_operation_name, _parent_thread_trace_context, @@ -107,8 +111,8 @@ struct OpenTelemetryThreadTraceContextScope { } - /// For Servers like HTTP/TCP/GRPC to initialize tracing context on thread that process requests from clients - OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + /// For servers like HTTP/TCP/GRPC to initialize tracing context on thread that process requests from clients + OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, OpenTelemetryTraceContext _parent_trace_context, const Settings& _settings, const std::weak_ptr& _log) @@ -119,7 +123,7 @@ struct OpenTelemetryThreadTraceContextScope { } - OpenTelemetryThreadTraceContextScope(const std::string& _operation_name, + OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, OpenTelemetryTraceContext _parent_trace_context, const Settings* settings_ptr, const std::weak_ptr& _log); @@ -137,7 +141,7 @@ using OpenTelemetryThreadTraceContextScopePtr = std::unique_ptr::worker(typename std::list::iterator thread_ setThreadName("ThreadPool"); Job job; - DB::OpenTelemetryThreadTraceContext parent_thead_trace_context; // A copy of parent trace context bool need_shutdown = false; + /// A copy of parent trace context + DB::OpenTelemetryThreadTraceContext parent_thead_trace_context; + { std::unique_lock lock(mutex); new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); @@ -278,7 +280,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!need_shutdown) { - // set up tracing context for this thread by its parent context + /// Set up tracing context for this thread by its parent context DB::OpenTelemetryThreadTraceContextScope thread_trace_context("ThreadPool::worker()" , parent_thead_trace_context); @@ -292,8 +294,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (thread_trace_context.root_span.isTraceEnabled()) { - // Use the thread name as operation name so that the tracing log will be more clear. - // the thread name is usually set in the jobs, we can only get the name after the job finishes + /// Use the thread name as operation name so that the tracing log will be more clear. + /// The thread name is usually set in the jobs, we can only get the name after the job finishes std::string thread_name = getThreadName(); if (!thread_name.empty()) thread_trace_context.root_span.operation_name = thread_name; diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index e7b7c4c2b8d..ed4f7809dee 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -65,7 +65,7 @@ public: ContextMutablePtr sessionContext() { return session_context; } ContextPtr sessionContext() const { return session_context; } - ContextPtr sessionOrGlobalContext() { return session_context ? session_context : global_context; } + ContextPtr sessionOrGlobalContext() const { return session_context ? session_context : global_context; } /// Makes a query context, can be used multiple times, with or without makeSession() called earlier. /// The query context will be created from a copy of a session context if it exists, or from a copy of diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4d37cd53faa..8f6d2abeeb4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -233,7 +233,7 @@ inline UInt64 time_in_seconds(std::chrono::time_point return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); } -static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, std::shared_ptr& query_span) +static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr& query_span) { /// Exception before the query execution. if (auto quota = context->getQuota()) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index f7abac66d1c..ce9c85a567c 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -871,8 +871,8 @@ private: const auto & distributed_header = readDistributedHeader(in, parent.log); OpenTelemetryThreadTraceContextScope thread_trace_context(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - parent.storage.getContext()->getOpenTelemetrySpanLog()); + distributed_header.client_info.client_trace_context, + parent.storage.getContext()->getOpenTelemetrySpanLog()); if (!remote) { From cd19366b44e8d1d3b0c548f5e93bff18d95233b3 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 24 Aug 2022 16:41:40 +0800 Subject: [PATCH 123/582] Move classes into DB::OpenTelemetry namespace --- src/Client/Connection.cpp | 4 +- src/Common/OpenTelemetryTraceContext.cpp | 37 +++++---- src/Common/OpenTelemetryTraceContext.h | 83 +++++++++++-------- src/Common/ThreadPool.cpp | 8 +- src/Common/ThreadPool.h | 4 +- src/Interpreters/ClientInfo.h | 5 +- src/Interpreters/OpenTelemetrySpanLog.h | 6 +- src/Interpreters/executeQuery.cpp | 10 +-- .../Executors/ExecutionThreadContext.cpp | 4 +- src/Server/GRPCServer.cpp | 4 +- src/Server/HTTPHandler.cpp | 4 +- src/Server/TCPHandler.cpp | 4 +- src/Storages/Distributed/DirectoryMonitor.cpp | 16 ++-- src/Storages/Distributed/DistributedSink.cpp | 14 ++-- src/Storages/StorageURL.cpp | 2 +- 15 files changed, 110 insertions(+), 95 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 56759e96a8d..ed048d1f986 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -484,13 +484,13 @@ void Connection::sendQuery( bool with_pending_data, std::function) { - OpenTelemetrySpanHolder span("Connection::sendQuery()"); + SpanHolder span("Connection::sendQuery()"); span.addAttribute("clickhouse.query_id", query_id_); span.addAttribute("clickhouse.query", query); span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); }); ClientInfo new_client_info; - const auto ¤t_trace_context = OpenTelemetryThreadTraceContext::current(); + const auto ¤t_trace_context = TracingContextOnThread::current(); if (client_info && current_trace_context.isTraceEnabled()) { // use current span as the parent of remote span diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 293439a129c..87f3afc77b9 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -9,10 +9,12 @@ namespace DB { +namespace OpenTelemetry +{ -thread_local OpenTelemetryThreadTraceContext current_thread_trace_context; +thread_local TracingContextOnThread current_thread_trace_context; -void OpenTelemetrySpan::addAttribute(std::string_view name, UInt64 value) +void Span::addAttribute(std::string_view name, UInt64 value) { if (!this->isTraceEnabled() || name.empty()) return; @@ -20,13 +22,13 @@ void OpenTelemetrySpan::addAttribute(std::string_view name, UInt64 value) this->attributes.push_back(Tuple{name, toString(value)}); } -void OpenTelemetrySpan::addAttributeIfNotZero(std::string_view name, UInt64 value) +void Span::addAttributeIfNotZero(std::string_view name, UInt64 value) { if (value != 0) addAttribute(name, value); } -void OpenTelemetrySpan::addAttribute(std::string_view name, std::string_view value) +void Span::addAttribute(std::string_view name, std::string_view value) { if (!this->isTraceEnabled() || name.empty()) return; @@ -34,7 +36,7 @@ void OpenTelemetrySpan::addAttribute(std::string_view name, std::string_view val this->attributes.push_back(Tuple{name, value}); } -void OpenTelemetrySpan::addAttribute(std::string_view name, std::function value_supplier) +void Span::addAttribute(std::string_view name, std::function value_supplier) { if (!this->isTraceEnabled() || !value_supplier) return; @@ -46,7 +48,7 @@ void OpenTelemetrySpan::addAttribute(std::string_view name, std::functionattributes.push_back(Tuple{name, value}); } -void OpenTelemetrySpan::addAttribute(const Exception & e) noexcept +void Span::addAttribute(const Exception & e) noexcept { if (!this->isTraceEnabled()) return; @@ -60,7 +62,7 @@ void OpenTelemetrySpan::addAttribute(const Exception & e) noexcept } } -void OpenTelemetrySpan::addAttribute(std::exception_ptr e) noexcept +void Span::addAttribute(std::exception_ptr e) noexcept { if (!this->isTraceEnabled() || e == nullptr) return; @@ -74,7 +76,7 @@ void OpenTelemetrySpan::addAttribute(std::exception_ptr e) noexcept } } -OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(std::string_view _operation_name) +SpanHolder::SpanHolder(std::string_view _operation_name) { if (current_thread_trace_context.isTraceEnabled()) { @@ -90,7 +92,7 @@ OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(std::string_view _operation_nam } } -void OpenTelemetrySpanHolder::finish() +void SpanHolder::finish() { if (!this->isTraceEnabled()) return; @@ -121,12 +123,12 @@ void OpenTelemetrySpanHolder::finish() trace_id = UUID(); } -OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() +SpanHolder::~SpanHolder() { finish(); } -bool OpenTelemetryTraceContext::parseTraceparentHeader(std::string_view traceparent, String & error) +bool TracingContext::parseTraceparentHeader(std::string_view traceparent, String & error) { trace_id = 0; @@ -185,7 +187,7 @@ bool OpenTelemetryTraceContext::parseTraceparentHeader(std::string_view tracepar return true; } -String OpenTelemetryTraceContext::composeTraceparentHeader() const +String TracingContext::composeTraceparentHeader() const { // This span is a parent for its children, so we specify this span_id as a // parent id. @@ -199,12 +201,12 @@ String OpenTelemetryTraceContext::composeTraceparentHeader() const static_cast(trace_flags)); } -const OpenTelemetryThreadTraceContext & OpenTelemetryThreadTraceContext::current() +const TracingContextOnThread & TracingContextOnThread::current() { return current_thread_trace_context; } -void OpenTelemetryThreadTraceContext::reset() +void TracingContextOnThread::reset() { this->trace_id = UUID(); this->span_id = 0; @@ -213,9 +215,9 @@ void OpenTelemetryThreadTraceContext::reset() this->span_log.reset(); } -OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( +TracingContextHolder::TracingContextHolder( std::string_view _operation_name, - OpenTelemetryTraceContext _parent_trace_context, + TracingContext _parent_trace_context, const Settings * settings_ptr, const std::weak_ptr & _span_log) { @@ -278,7 +280,7 @@ OpenTelemetryThreadTraceContextScope::OpenTelemetryThreadTraceContextScope( current_thread_trace_context.span_log = _span_log; } -OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() +TracingContextHolder::~TracingContextHolder() { if (this->root_span.isTraceEnabled()) { @@ -307,3 +309,4 @@ OpenTelemetryThreadTraceContextScope::~OpenTelemetryThreadTraceContextScope() } } +} diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index edc23efd551..e5cd44e9a2e 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -6,8 +6,12 @@ namespace DB { struct Settings; +class OpenTelemetrySpanLog; -struct OpenTelemetrySpan +namespace OpenTelemetry +{ + +struct Span { UUID trace_id{}; UInt64 span_id = 0; @@ -33,8 +37,6 @@ struct OpenTelemetrySpan } }; -class OpenTelemetrySpanLog; - /// See https://www.w3.org/TR/trace-context/ for trace_flags definition enum TraceFlags : UInt8 { @@ -42,8 +44,8 @@ enum TraceFlags : UInt8 TRACE_FLAG_SAMPLED = 1, }; -// The runtime info we need to create new OpenTelemetry spans. -struct OpenTelemetryTraceContext +/// The runtime info we need to create new OpenTelemetry spans. +struct TracingContext { UUID trace_id{}; UInt64 span_id = 0; @@ -62,39 +64,42 @@ struct OpenTelemetryTraceContext } }; -/// Tracing context kept on thread local -struct OpenTelemetryThreadTraceContext : OpenTelemetryTraceContext +/// Tracing context kept on each thread +struct TracingContextOnThread : TracingContext { - OpenTelemetryThreadTraceContext& operator =(const OpenTelemetryTraceContext& context) + TracingContextOnThread& operator =(const TracingContext& context) { - *(static_cast(this)) = context; + *(static_cast(this)) = context; return *this; } void reset(); - static const OpenTelemetryThreadTraceContext& current(); + static const TracingContextOnThread& current(); /// Use weak_ptr instead of shared_ptr to hold a reference to the underlying system.opentelemetry_span_log table /// Since this object is kept on threads and passed across threads, a weak_ptr is more safe to prevent potential leak std::weak_ptr span_log; }; -/// A scoped tracing context, is used to hold the tracing context at the beginning of each thread execution and clear the context automatically when the scope exists. -/// It should be the root of all span logs for one tracing. +/// Holder of tracing context. +/// It should be initialized at the beginning of each thread execution. +/// And once it's destructed, it clears the context automatically. /// -/// It's SAFE to construct this object multiple times on one same thread, -/// but it's not encourage to do so because this is only a protection in case of code changes. -struct OpenTelemetryThreadTraceContextScope +/// It's also the root of all spans on current thread execution. +/// +/// Although it's SAFE to construct this object multiple times on one same thread, +/// but rememeber only use it at the beginning of one thread +struct TracingContextHolder { /// Forbidden copy ctor and assignment to make the destructor safe - OpenTelemetryThreadTraceContextScope(const OpenTelemetryThreadTraceContextScope& scope) = delete; - OpenTelemetryThreadTraceContextScope& operator =(const OpenTelemetryThreadTraceContextScope& scope) = delete; + TracingContextHolder(const TracingContextHolder& scope) = delete; + TracingContextHolder& operator =(const TracingContextHolder& scope) = delete; - OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, - const OpenTelemetryTraceContext& _parent_trace_context, + TracingContextHolder(std::string_view _operation_name, + const TracingContext& _parent_trace_context, const std::weak_ptr& _log) - : OpenTelemetryThreadTraceContextScope(_operation_name, + : TracingContextHolder(_operation_name, _parent_trace_context, nullptr, _log) @@ -102,9 +107,9 @@ struct OpenTelemetryThreadTraceContextScope } /// Initialize a tracing context on a child thread based on the context from the parent thread - OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, - const OpenTelemetryThreadTraceContext& _parent_thread_trace_context) - : OpenTelemetryThreadTraceContextScope(_operation_name, + TracingContextHolder(std::string_view _operation_name, + const TracingContextOnThread& _parent_thread_trace_context) + : TracingContextHolder(_operation_name, _parent_thread_trace_context, nullptr, _parent_thread_trace_context.span_log) @@ -112,39 +117,47 @@ struct OpenTelemetryThreadTraceContextScope } /// For servers like HTTP/TCP/GRPC to initialize tracing context on thread that process requests from clients - OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, - OpenTelemetryTraceContext _parent_trace_context, + TracingContextHolder(std::string_view _operation_name, + TracingContext _parent_trace_context, const Settings& _settings, const std::weak_ptr& _log) - : OpenTelemetryThreadTraceContextScope(_operation_name, + : TracingContextHolder(_operation_name, _parent_trace_context, &_settings, _log) { } - OpenTelemetryThreadTraceContextScope(std::string_view _operation_name, - OpenTelemetryTraceContext _parent_trace_context, + TracingContextHolder(std::string_view _operation_name, + TracingContext _parent_trace_context, const Settings* settings_ptr, const std::weak_ptr& _log); - ~OpenTelemetryThreadTraceContextScope(); + ~TracingContextHolder(); - OpenTelemetrySpan root_span; + Span root_span; private: bool is_context_owner = true; }; -using OpenTelemetryThreadTraceContextScopePtr = std::unique_ptr; +using TracingContextHolderPtr = std::unique_ptr; -/// A span holder is usually used in a function scope -struct OpenTelemetrySpanHolder : public OpenTelemetrySpan +/// A span holder that creates span automatically in a (function) scope if tracing is enabled. +/// Once it's created or destructed, it automatically maitains the tracing context on the thread that it lives. +struct SpanHolder : public Span { - OpenTelemetrySpanHolder(std::string_view); - ~OpenTelemetrySpanHolder(); + SpanHolder(std::string_view); + ~SpanHolder(); + /// Finish a span explicitly if needed. + /// It's safe to call it multiple times void finish(); }; } + +using namespace OpenTelemetry; + +} + diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 4850d9e0703..cbd0e642cf2 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -152,7 +152,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti // this scheduleImpl is called in the parent thread, // the tracing context on this thread is used as parent context for the sub-thread that runs the job - const auto ¤t_thread_context = DB::OpenTelemetryThreadTraceContext::current(); + const auto ¤t_thread_context = DB::TracingContextOnThread::current(); jobs.emplace(std::move(job), priority, current_thread_context); ++scheduled_jobs; new_job_or_shutdown.notify_one(); @@ -255,7 +255,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ bool need_shutdown = false; /// A copy of parent trace context - DB::OpenTelemetryThreadTraceContext parent_thead_trace_context; + DB::TracingContextOnThread parent_thead_trace_context; { std::unique_lock lock(mutex); @@ -267,7 +267,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// boost::priority_queue does not provide interface for getting non-const reference to an element /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); - parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); + parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); jobs.pop(); } else @@ -281,7 +281,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!need_shutdown) { /// Set up tracing context for this thread by its parent context - DB::OpenTelemetryThreadTraceContextScope thread_trace_context("ThreadPool::worker()" , + DB::TracingContextHolder thread_trace_context("ThreadPool::worker()" , parent_thead_trace_context); try diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 0c54600c862..65992e1b76e 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -97,9 +97,9 @@ private: { Job job; int priority; - DB::OpenTelemetryThreadTraceContext thread_trace_context; + DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, int priority_, const DB::OpenTelemetryThreadTraceContext& thread_trace_context_) + JobWithPriority(Job job_, int priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} bool operator< (const JobWithPriority & rhs) const diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 3ea846101f5..2d8e1a1764f 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -62,9 +62,8 @@ public: time_t initial_query_start_time{}; Decimal64 initial_query_start_time_microseconds{}; - // OpenTelemetry trace context we received from client, or which we are going - // to send to server. - OpenTelemetryTraceContext client_trace_context; + /// OpenTelemetry trace context we received from client, or which we are going to send to server. + TracingContext client_trace_context; /// All below are parameters related to initial query. diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 671fd01128b..9a7bbad4ee4 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -7,11 +7,11 @@ namespace DB { -struct OpenTelemetrySpanLogElement : public OpenTelemetrySpan +struct OpenTelemetrySpanLogElement : public Span { OpenTelemetrySpanLogElement() = default; - OpenTelemetrySpanLogElement(const OpenTelemetrySpan & span) - : OpenTelemetrySpan(span) {} + OpenTelemetrySpanLogElement(const Span & span) + : Span(span) {} static std::string name() { return "OpenTelemetrySpanLog"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8f6d2abeeb4..e467d6fb80e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -233,7 +233,7 @@ inline UInt64 time_in_seconds(std::chrono::time_point return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); } -static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr& query_span) +static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr & query_span) { /// Exception before the query execution. if (auto quota = context->getQuota()) @@ -345,7 +345,7 @@ static std::tuple executeQueryImpl( QueryProcessingStage::Enum stage, ReadBuffer * istr) { - std::shared_ptr query_span = std::make_shared("query"); + std::shared_ptr query_span = std::make_shared("query"); const auto current_time = std::chrono::system_clock::now(); @@ -667,12 +667,12 @@ static std::tuple executeQueryImpl( } { - std::unique_ptr span; + std::unique_ptr span; if (query_span->isTraceEnabled()) { auto * raw_interpreter_ptr = interpreter.get(); std::string class_name(demangle(typeid(*raw_interpreter_ptr).name())); - span = std::make_unique(class_name + "::execute()"); + span = std::make_unique(class_name + "::execute()"); } res = interpreter->execute(); } @@ -930,7 +930,7 @@ static std::tuple executeQueryImpl( query_span->addAttribute("db.statement", elem.query); query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); - query_span->addAttribute("clickhouse.tracestate", OpenTelemetryThreadTraceContext::current().tracestate); + query_span->addAttribute("clickhouse.tracestate", TracingContextOnThread::current().tracestate); query_span->addAttribute("clickhouse.query_status", "QueryFinish"); query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); diff --git a/src/Processors/Executors/ExecutionThreadContext.cpp b/src/Processors/Executors/ExecutionThreadContext.cpp index 7631cb09f61..8825e3f131d 100644 --- a/src/Processors/Executors/ExecutionThreadContext.cpp +++ b/src/Processors/Executors/ExecutionThreadContext.cpp @@ -71,11 +71,11 @@ static void executeJob(ExecutingGraph::Node * node, ReadProgressCallback * read_ bool ExecutionThreadContext::executeTask() { - std::unique_ptr span; + std::unique_ptr span; if (trace_processors) { - span = std::make_unique("ExecutionThreadContext::executeTask() " + node->processor->getName()); + span = std::make_unique("ExecutionThreadContext::executeTask() " + node->processor->getName()); span->addAttribute("thread_number", thread_number); } std::optional execution_time_watch; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index cdf1e0247b1..390c8a23cf8 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -662,7 +662,7 @@ namespace std::optional session; ContextMutablePtr query_context; std::optional query_scope; - OpenTelemetryThreadTraceContextScopePtr thread_trace_context; + TracingContextHolderPtr thread_trace_context; String query_text; ASTPtr ast; ASTInsertQuery * insert_query = nullptr; @@ -842,7 +842,7 @@ namespace query_scope.emplace(query_context); /// Set up tracing context for this query on current thread - thread_trace_context = std::make_unique("GRPCServer", + thread_trace_context = std::make_unique("GRPCServer", query_context->getClientInfo().client_trace_context, query_context->getSettingsRef(), query_context->getOpenTelemetrySpanLog()); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 6f5bc2c1357..251a3b3beb3 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -933,7 +933,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// In case of exception, send stack trace to client. bool with_stacktrace = false; - OpenTelemetryThreadTraceContextScopePtr thread_trace_context; + TracingContextHolderPtr thread_trace_context; SCOPE_EXIT({ // make sure the response status is recorded if (thread_trace_context) @@ -963,7 +963,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); - thread_trace_context = std::make_unique("HTTPHandler", + thread_trace_context = std::make_unique("HTTPHandler", client_info.client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog()); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 790dc599609..242a61ccaec 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -219,7 +219,7 @@ void TCPHandler::runImpl() /// Initialized later. std::optional query_scope; - OpenTelemetryThreadTraceContextScopePtr thread_trace_context; + TracingContextHolderPtr thread_trace_context; /** An exception during the execution of request (it must be sent over the network to the client). * The client will be able to accept it, if it did not happen while sending another packet and the client has not disconnected yet. @@ -246,7 +246,7 @@ void TCPHandler::runImpl() continue; /// Set up tracing context for this query on current thread - thread_trace_context = std::make_unique("TCPHandler", + thread_trace_context = std::make_unique("TCPHandler", query_context->getClientInfo().client_trace_context, query_context->getSettingsRef(), query_context->getOpenTelemetrySpanLog()); diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index ce9c85a567c..89673a76582 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -609,7 +609,7 @@ bool StorageDistributedDirectoryMonitor::processFiles(const std::mapgetSettingsRef()); @@ -629,9 +629,9 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa formatReadableQuantity(distributed_header.rows), formatReadableSizeWithBinarySuffix(distributed_header.bytes)); - thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - this->storage.getContext()->getOpenTelemetrySpanLog()); + thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + this->storage.getContext()->getOpenTelemetrySpanLog()); RemoteInserter remote{*connection, timeouts, distributed_header.insert_query, @@ -870,7 +870,7 @@ private: ReadBufferFromFile in(file_path->second); const auto & distributed_header = readDistributedHeader(in, parent.log); - OpenTelemetryThreadTraceContextScope thread_trace_context(__PRETTY_FUNCTION__, + TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, parent.storage.getContext()->getOpenTelemetrySpanLog()); @@ -909,9 +909,9 @@ private: const auto & distributed_header = readDistributedHeader(in, parent.log); // this function is called in a separated thread, so we set up the trace context from the file - OpenTelemetryThreadTraceContextScope thread_trace_context(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - parent.storage.getContext()->getOpenTelemetrySpanLog()); + TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + parent.storage.getContext()->getOpenTelemetrySpanLog()); RemoteInserter remote(connection, timeouts, distributed_header.insert_query, diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index f47c2e93750..2dec4c5abb6 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -336,7 +336,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si if (rows == 0) return; - OpenTelemetrySpanHolder span(__PRETTY_FUNCTION__); + SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); span.addAttribute("clickhouse.written_rows", rows); @@ -419,7 +419,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si void DistributedSink::writeSync(const Block & block) { - OpenTelemetrySpanHolder span(__PRETTY_FUNCTION__); + SpanHolder span(__PRETTY_FUNCTION__); const Settings & settings = context->getSettingsRef(); const auto & shards_info = cluster->getShardsInfo(); @@ -610,7 +610,7 @@ void DistributedSink::writeSplitAsync(const Block & block) void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) { - OpenTelemetrySpanHolder span("DistributedSink::writeAsyncImpl()"); + SpanHolder span("DistributedSink::writeAsyncImpl()"); const auto & shard_info = cluster->getShardsInfo()[shard_id]; const auto & settings = context->getSettingsRef(); @@ -652,7 +652,7 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) void DistributedSink::writeToLocal(const Block & block, size_t repeats) { - OpenTelemetrySpanHolder span(__PRETTY_FUNCTION__); + SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("db.statement", this->query_string); InterpreterInsertQuery interp(query_ast, context, allow_materialized); @@ -668,7 +668,7 @@ void DistributedSink::writeToLocal(const Block & block, size_t repeats) void DistributedSink::writeToShard(const Block & block, const std::vector & dir_names) { - OpenTelemetrySpanHolder span(__PRETTY_FUNCTION__); + SpanHolder span(__PRETTY_FUNCTION__); const auto & settings = context->getSettingsRef(); const auto & distributed_settings = storage.getDistributedSettingsRef(); @@ -737,11 +737,11 @@ void DistributedSink::writeToShard(const Block & block, const std::vectorgetSettingsRef().write(header_buf); - if (OpenTelemetryThreadTraceContext::current().isTraceEnabled()) + if (TracingContextOnThread::current().isTraceEnabled()) { // if the distributed tracing is enabled, use the trace context in current thread as parent of next span auto client_info = context->getClientInfo(); - client_info.client_trace_context = OpenTelemetryThreadTraceContext::current(); + client_info.client_trace_context = TracingContextOnThread::current(); client_info.write(header_buf, DBMS_TCP_PROTOCOL_VERSION); } else diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d4688918b5f..d0a28a891f3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -102,7 +102,7 @@ namespace ReadWriteBufferFromHTTP::HTTPHeaderEntries headers(headers_.begin(), headers_.end()); // Propagate OpenTelemetry trace context, if any, downstream. - const auto ¤t_trace_context = OpenTelemetryThreadTraceContext::current(); + const auto ¤t_trace_context = TracingContextOnThread::current(); if (current_trace_context.isTraceEnabled()) { headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); From 03007606bab5294a5d81e229421f305a6d8eef1b Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 24 Aug 2022 17:01:31 +0800 Subject: [PATCH 124/582] Fix format Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.h | 37 +++++++++++++------------- src/Common/ThreadPool.cpp | 3 +-- 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index e5cd44e9a2e..3be814f1bee 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -97,41 +97,40 @@ struct TracingContextHolder TracingContextHolder& operator =(const TracingContextHolder& scope) = delete; TracingContextHolder(std::string_view _operation_name, - const TracingContext& _parent_trace_context, - const std::weak_ptr& _log) + const TracingContext& _parent_trace_context, + const std::weak_ptr& _log) : TracingContextHolder(_operation_name, - _parent_trace_context, - nullptr, - _log) + _parent_trace_context, + nullptr, + _log) { } /// Initialize a tracing context on a child thread based on the context from the parent thread - TracingContextHolder(std::string_view _operation_name, - const TracingContextOnThread& _parent_thread_trace_context) + TracingContextHolder(std::string_view _operation_name, const TracingContextOnThread & _parent_thread_trace_context) : TracingContextHolder(_operation_name, - _parent_thread_trace_context, - nullptr, - _parent_thread_trace_context.span_log) + _parent_thread_trace_context, + nullptr, + _parent_thread_trace_context.span_log) { } /// For servers like HTTP/TCP/GRPC to initialize tracing context on thread that process requests from clients TracingContextHolder(std::string_view _operation_name, - TracingContext _parent_trace_context, - const Settings& _settings, - const std::weak_ptr& _log) + TracingContext _parent_trace_context, + const Settings & _settings, + const std::weak_ptr & _log) : TracingContextHolder(_operation_name, - _parent_trace_context, - &_settings, - _log) + _parent_trace_context, + &_settings, + _log) { } TracingContextHolder(std::string_view _operation_name, - TracingContext _parent_trace_context, - const Settings* settings_ptr, - const std::weak_ptr& _log); + TracingContext _parent_trace_context, + const Settings* settings_ptr, + const std::weak_ptr & _log); ~TracingContextHolder(); diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index cbd0e642cf2..358863d46c7 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -281,8 +281,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!need_shutdown) { /// Set up tracing context for this thread by its parent context - DB::TracingContextHolder thread_trace_context("ThreadPool::worker()" , - parent_thead_trace_context); + DB::TracingContextHolder thread_trace_context("ThreadPool::worker()", parent_thead_trace_context); try { From dd0f2cbe6b676bcafebedb9a8d263b0773babae3 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 24 Aug 2022 17:04:02 +0800 Subject: [PATCH 125/582] Fix style checking Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 6 ++++-- src/Common/OpenTelemetryTraceContext.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 87f3afc77b9..42c99ebcfd7 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -57,8 +57,9 @@ void Span::addAttribute(const Exception & e) noexcept { this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); } - catch(...) + catch (...) { + /// Ignore exceptions } } @@ -71,8 +72,9 @@ void Span::addAttribute(std::exception_ptr e) noexcept { this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); } - catch(...) + catch (...) { + /// Ignore exceptions } } diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 3be814f1bee..9b55aff3e68 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -83,7 +83,7 @@ struct TracingContextOnThread : TracingContext }; /// Holder of tracing context. -/// It should be initialized at the beginning of each thread execution. +/// It should be initialized at the beginning of each thread execution. /// And once it's destructed, it clears the context automatically. /// /// It's also the root of all spans on current thread execution. From 1cf3659f26609128647d0746f7d8f1e77542631f Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 24 Aug 2022 18:03:14 +0800 Subject: [PATCH 126/582] Fix style checking Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 9b55aff3e68..8a9792b2d30 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -98,7 +98,7 @@ struct TracingContextHolder TracingContextHolder(std::string_view _operation_name, const TracingContext& _parent_trace_context, - const std::weak_ptr& _log) + const std::weak_ptr& _log) : TracingContextHolder(_operation_name, _parent_trace_context, nullptr, From 72bc451a8a6b59e75beed134806f8347f2db4da8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 16:50:48 +0200 Subject: [PATCH 127/582] Revert "Update run.sh" This reverts commit 0b6e1462a25473e21d36a50e5f8c5bbe4003284d. --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6a019a721b3..a6df29f91fe 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -200,7 +200,7 @@ clickhouse-client --query "SHOW TABLES FROM test" clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" -clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" +clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" From b7e8b727bcd357fd1e817f78f49c9b01b042f76a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 16:50:56 +0200 Subject: [PATCH 128/582] Revert "Check what will happen if disable cache" This reverts commit f0acc4a027dc07d75aafad569b672dea6ac8eb48. --- docker/test/stress/run.sh | 4 ++-- tests/config/config.d/storage_conf.xml | 14 -------------- 2 files changed, 2 insertions(+), 16 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index a6df29f91fe..d23a96c508c 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -198,8 +198,8 @@ start clickhouse-client --query "SHOW TABLES FROM datasets" clickhouse-client --query "SHOW TABLES FROM test" -clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" -clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_simple'" +clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" +clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 13e3172101f..de41478f74a 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -2,13 +2,6 @@ - - s3 - s3_simple/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - s3 s3_disk/ @@ -155,13 +148,6 @@ - - -
- s3_simple -
-
-
From 96b32c6380569f65079e58fd6a4f4f50a2b69f9c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 16:51:21 +0200 Subject: [PATCH 129/582] Fix batch delete --- .../ObjectStorages/DiskObjectStorage.cpp | 7 ++ src/Disks/ObjectStorages/DiskObjectStorage.h | 2 + .../DiskObjectStorageTransaction.cpp | 92 +++++++++++++++++-- 3 files changed, 93 insertions(+), 8 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 75579247752..694b1f719da 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -240,6 +240,13 @@ void DiskObjectStorage::removeSharedFile(const String & path, bool delete_metada transaction->commit(); } +void DiskObjectStorage::removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) +{ + auto transaction = createObjectStorageTransaction(); + transaction->removeSharedFiles(files, keep_all_batch_data, file_names_remove_metadata_only); + transaction->commit(); +} + UInt32 DiskObjectStorage::getRefCount(const String & path) const { return metadata_storage->getHardlinkCount(path); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 1b53cd514a0..e08d5069a91 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -94,6 +94,8 @@ public: void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override; + void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override; + MetadataStoragePtr getMetadataStorage() override { return metadata_storage; } UInt32 getRefCount(const String & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 28015ff0b04..da704952c95 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -138,6 +138,88 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation } }; +struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperation +{ + RemoveBatchRequest remove_paths; + bool keep_all_batch_data; + NameSet file_names_remove_metadata_only; + StoredObjects objects_to_remove; + bool remove_from_cache = false; + + RemoveManyObjectStorageOperation( + IObjectStorage & object_storage_, + IMetadataStorage & metadata_storage_, + const RemoveBatchRequest & remove_paths_, + bool keep_all_batch_data_, + const NameSet & file_names_remove_metadata_only_) + : IDiskObjectStorageOperation(object_storage_, metadata_storage_) + , remove_paths(remove_paths_) + , keep_all_batch_data(keep_all_batch_data_) + , file_names_remove_metadata_only(file_names_remove_metadata_only_) + {} + + std::string getInfoForLog() const override + { + return fmt::format("RemoveManyObjectStorageOperation (paths size: {}, keep all batch {}, files to keep {})", remove_paths.size(), keep_all_batch_data, fmt::join(file_names_remove_metadata_only, ", ")); + } + + void execute(MetadataTransactionPtr tx) override + { + for (const auto & [path, if_exists] : remove_paths) + { + + if (!metadata_storage.exists(path)) + { + if (if_exists) + return; + + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata path '{}' doesn't exist", path); + } + + if (!metadata_storage.isFile(path)) + throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path '{}' is not a regular file", path); + + try + { + uint32_t hardlink_count = metadata_storage.getHardlinkCount(path); + auto objects = metadata_storage.getStorageObjects(path); + + tx->unlinkMetadata(path); + + /// File is really redundant + if (hardlink_count == 0 && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) + objects_to_remove.insert(objects_to_remove.end(), objects.begin(), objects.end()); + } + catch (const Exception & e) + { + /// If it's impossible to read meta - just remove it from FS. + if (e.code() == ErrorCodes::UNKNOWN_FORMAT + || e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF + || e.code() == ErrorCodes::CANNOT_READ_ALL_DATA + || e.code() == ErrorCodes::CANNOT_OPEN_FILE) + { + tx->unlinkFile(path); + } + else + throw; + } + } + } + + void undo() override + { + + } + + void finalize() override + { + if (!objects_to_remove.empty()) + object_storage.removeObjects(objects_to_remove); + } +}; + + + struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOperation { std::string path; @@ -479,14 +561,8 @@ void DiskObjectStorageTransaction::removeFileIfExists(const std::string & path) void DiskObjectStorageTransaction::removeSharedFiles( const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) { - for (const auto & file : files) - { - bool keep_file = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename()); - if (file.if_exists) - removeSharedFileIfExists(file.path, keep_file); - else - removeSharedFile(file.path, keep_file); - } + auto operation = std::make_unique(object_storage, metadata_storage, files, keep_all_batch_data, file_names_remove_metadata_only); + operations_to_execute.emplace_back(std::move(operation)); } namespace From ac54f15c0680b668d47a9c49390d6a0c7f23665e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 17:06:27 +0200 Subject: [PATCH 130/582] Remove more whitespaces --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 286b6427380..be23b6c129d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -4,7 +4,6 @@ #include #include - namespace DB { @@ -219,7 +218,6 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati }; - struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOperation { std::string path; From dc64341876643c9a6b751acc9b9f3f27f74ee67a Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 24 Aug 2022 23:42:13 +0800 Subject: [PATCH 131/582] add tests Signed-off-by: Lloyd-Pottiger --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- .../integration/test_rocksdb_options/test.py | 36 +++++++++++++++++++ .../02404_rocksdb_read_only.reference | 16 +++++++++ .../0_stateless/02404_rocksdb_read_only.sh | 26 ++++++++++++++ 4 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02404_rocksdb_read_only.reference create mode 100644 tests/queries/0_stateless/02404_rocksdb_read_only.sh diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 1a47fa6ba33..34f39d8b5ad 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -279,7 +279,7 @@ void StorageEmbeddedRocksDB::initDB() if (ttl > 0) { rocksdb::DBWithTTL * db; - status = rocksdb::DBWithTTL::Open(merged, rocksdb_dir, &db, ttl); + status = rocksdb::DBWithTTL::Open(merged, rocksdb_dir, &db, ttl, read_only); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}", diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index a9e12eae4fd..2964a6db20d 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -42,6 +42,42 @@ def test_valid_options(start_cluster): DROP TABLE test; """ ) + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); + DROP TABLE test; + """ + ) + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + DROP TABLE test; + """ + ) + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); + INSERT INTO test (key, value) VALUES (0, 'a'); + """ + ) + node.stop_clickhouse() + node.exec_in_container( + [ + "bash", + "-c", + "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only", + ] + ) + node.start_clickhouse() + result = node.query( + """SELECT * FROM test; + """ + ) + assert result.strip() == "0\ta" + result = node.query( + """DROP TABLE test; + """ + ) def test_invalid_options(start_cluster): diff --git a/tests/queries/0_stateless/02404_rocksdb_read_only.reference b/tests/queries/0_stateless/02404_rocksdb_read_only.reference new file mode 100644 index 00000000000..c955fd6034f --- /dev/null +++ b/tests/queries/0_stateless/02404_rocksdb_read_only.reference @@ -0,0 +1,16 @@ +OK +---- +FAIL +---- +OK +OK +---- +FAIL +OK +OK +---- +3 +FAIL +OK +---- +FAIL \ No newline at end of file diff --git a/tests/queries/0_stateless/02404_rocksdb_read_only.sh b/tests/queries/0_stateless/02404_rocksdb_read_only.sh new file mode 100644 index 00000000000..c2e723b26c3 --- /dev/null +++ b/tests/queries/0_stateless/02404_rocksdb_read_only.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ROCKSDB_DIR="/var/lib/clickhouse/store/test_rocksdb_read_only" + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_02404;" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +echo "----" +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +echo "----" +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}') PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (0, 'a'), (1, 'b'), (3, 'c');" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +echo "----" +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +echo "----" +$CLICKHOUSE_CLIENT --query="SELECT * FROM test_02404;" 2>&1 | grep -F "rows in set." | awk '{print $1}'; +$CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (4, 'd');" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +echo "----" +rm -r ${ROCKSDB_DIR} +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; From 41dd49356bb1965b97e5c03912c83971be1ee8ab Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Aug 2022 17:27:07 +0000 Subject: [PATCH 132/582] Table create better --- src/Storages/StorageKeeperMap.cpp | 131 ++++++++++++++++-------------- src/Storages/StorageKeeperMap.h | 7 ++ 2 files changed, 77 insertions(+), 61 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 3477f29fc10..9be462d15a3 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -28,6 +28,7 @@ #include #include #include +#include "Core/UUID.h" #include #include @@ -275,6 +276,9 @@ StorageKeeperMap::StorageKeeperMap( , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) , log(&Poco::Logger::get("StorageKeeperMap")) { + if (table_id.uuid == UUIDHelpers::Nil) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); + setInMemoryMetadata(metadata); if (root_path.empty()) @@ -287,7 +291,9 @@ StorageKeeperMap::StorageKeeperMap( metadata_path = metadata_path_fs; lock_path = metadata_path_fs / "lock"; tables_path = metadata_path_fs / "tables"; + table_path = fs::path(tables_path) / toString(table_id.uuid); dropped_path = metadata_path_fs / "dropped"; + dropped_lock_path = fs::path(dropped_path) / "lock"; if (attach) { @@ -309,77 +315,49 @@ StorageKeeperMap::StorageKeeperMap( else { LOG_TRACE(log, "Creating root path {}", root_path); - - size_t cur_pos = 0; - do - { - size_t search_start = cur_pos + 1; - cur_pos = root_path.find('/', search_start); - if (search_start == cur_pos) - throw Exception("root_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); - - auto path = root_path.substr(0, cur_pos); - client->createIfNotExists(path, ""); - } while (cur_pos != std::string_view::npos); + client->createAncestors(root_path); + client->createIfNotExists(root_path, ""); } } - - ZooKeeperLock lock(lock_path, client, true); - - bool locked = false; - try - { - lock.lock(); - locked = true; - } - catch (const Coordination::Exception & e) - { - if (e.code != Coordination::Error::ZNONODE) - throw; - } - - { - Coordination::Stat stats; - auto exists = client->exists(root_path, &stats); - if (!exists) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); - } - - - const auto set_tables_num = [&] - { - std::string tables_num_str; - auto exists = client->tryGet(tables_path, tables_num_str); - - if (!exists) - { - client->create(tables_path, toString(1), zkutil::CreateMode::Persistent); - } - else - { - uint64_t tables_num = parse(tables_num_str); - client->set(tables_path, toString(tables_num + 1), zkutil::CreateMode::Persistent); - } - }; - - // we have metadata and lock nodes, because they can't be deleted with the lock acquired - if (locked) + for (size_t i = 0; i < 1000; ++i) { if (client->exists(dropped_path)) { - client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); - client->remove(dropped_path); + LOG_INFO(log, "Removing leftover nodes"); + auto code = client->tryCreate(dropped_lock_path, "", zkutil::CreateMode::Ephemeral); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "Someone else removed leftovers"); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception(code, dropped_lock_path); + } + else + { + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); + if (!removeMetadataNodes(client, metadata_drop_lock)) + continue; + } } - set_tables_num(); - return; + client->createIfNotExists(metadata_path, ""); + client->createIfNotExists(tables_path, ""); + + auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); + + if (code == Coordination::Error::ZOK) + return; + + if (code == Coordination::Error::ZNONODE) + LOG_INFO(log, "Metadata nodes were deleted in background, will retry"); + else + throw Coordination::Exception(code, table_path); } - client->createIfNotExists(metadata_path, ""); - client->createIfNotExists(lock_path, ""); - lock.lock(); - set_tables_num(); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or becauyse of wrong root_path ({})", root_path); } @@ -452,6 +430,37 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); } +bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) +{ + bool completely_removed = false; + Coordination::Requests ops; + ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); + ops.emplace_back(zkutil::makeRemoveRequest(dropped_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(metadata_path, -1)); + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + using enum Coordination::Error; + switch (code) + { + case ZOK: + { + metadata_drop_lock->setAlreadyRemoved(); + completely_removed = true; + LOG_INFO(log, "Metadata in {} was successfully removed from ZooKeeper", metadata_path); + break; + } + case ZNONODE: + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of metadata. It's a bug"); + case ZNOTEMPTY: + LOG_ERROR(log, "Metadata was not completely removed from ZooKeeper"); + break; + default: + zkutil::KeeperMultiException::check(code, ops, responses); + } + return completely_removed; +} + void StorageKeeperMap::drop() { checkTable(); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index e09e8923e5c..083d4c4c9ed 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -101,12 +101,19 @@ public: } private: + bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); + std::string root_path; std::string primary_key; + std::string metadata_path; std::string lock_path; + std::string tables_path; + std::string table_path; + std::string dropped_path; + std::string dropped_lock_path; std::string zookeeper_name; From 6afd9c176b2bc12a0c5fed93a3d7599dc1333ee6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 24 Aug 2022 21:05:00 +0000 Subject: [PATCH 133/582] inode based singleton --- .../decompressor.cpp | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 8d8d137a2ac..14e6e4171c7 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -13,6 +13,8 @@ #include #include #include +#include +#include #if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include @@ -359,6 +361,27 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #endif +uint32_t get_inode(const char * self) +{ + std::ifstream maps("/proc/self/maps"); + if (maps.fail()) + { + perror("open maps"); + return 0; + } + + for (std::string line; std::getline(maps, line); ) + { + std::stringstream ss(line); + std::string addr, mode, offset, id, path; + uint32_t inode = 0; + if (ss >> addr >> mode >> offset >> id >> inode >> path && path == self) + return inode; + } + + return 0; +} + int main(int/* argc*/, char* argv[]) { char self[4096] = {0}; @@ -382,6 +405,52 @@ int main(int/* argc*/, char* argv[]) else name = file_path; + uint32_t inode = get_inode(self); + if (inode == 0) + { + std::cerr << "Unable to obtain inode." << std::endl; + return 1; + } + + std::stringstream lock_path; + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) + { + perror("lock open"); + return 1; + } + + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + struct stat input_info; + if (0 != stat(self, &input_info)) + { + perror("stat"); + return 1; + } + + /// if decompression was performed by another process + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) + { + perror("fstat lock"); + return 1; + } + + if (lock_info.st_size == 1) + execv(self, argv); + + printf("No target executable - decompression only was performed.\n"); + return 0; + } + int input_fd = open(self, O_RDONLY); if (input_fd == -1) { @@ -443,6 +512,8 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { + write(lock, "1", 1); + execv(self, argv); /// This part of code will be reached only if error happened From f043d0305381742d5456b2d01e72b9d630143aa0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 24 Aug 2022 21:28:52 +0000 Subject: [PATCH 134/582] fix style --- utils/self-extracting-executable/decompressor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 14e6e4171c7..bf5be5f15cf 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -372,7 +372,7 @@ uint32_t get_inode(const char * self) for (std::string line; std::getline(maps, line); ) { - std::stringstream ss(line); + std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM std::string addr, mode, offset, id, path; uint32_t inode = 0; if (ss >> addr >> mode >> offset >> id >> inode >> path && path == self) @@ -412,7 +412,7 @@ int main(int/* argc*/, char* argv[]) return 1; } - std::stringstream lock_path; + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); if (lock < 0) From 6c1a7205e34f766da376311de474286ca066991c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 24 Aug 2022 21:47:37 +0000 Subject: [PATCH 135/582] fix style --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index bf5be5f15cf..e687717dacd 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -370,7 +370,7 @@ uint32_t get_inode(const char * self) return 0; } - for (std::string line; std::getline(maps, line); ) + for (std::string line; std::getline(maps, line);) { std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM std::string addr, mode, offset, id, path; From 558b179cb11103d46a3fedc12d2f2c69c94378ae Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 25 Aug 2022 10:11:08 +0800 Subject: [PATCH 136/582] Fix typo Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 8a9792b2d30..1c570da380f 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -88,8 +88,7 @@ struct TracingContextOnThread : TracingContext /// /// It's also the root of all spans on current thread execution. /// -/// Although it's SAFE to construct this object multiple times on one same thread, -/// but rememeber only use it at the beginning of one thread +/// Although it's SAFE to construct this object multiple times on one same thread, it shoud be created at the beginning of one thread execution. struct TracingContextHolder { /// Forbidden copy ctor and assignment to make the destructor safe From 221a71f801506ae2b550fcf8d377b5994d96f6d7 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 25 Aug 2022 15:05:07 +0800 Subject: [PATCH 137/582] Fix typo --- src/Common/OpenTelemetryTraceContext.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 1c570da380f..32557183bfd 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -88,7 +88,7 @@ struct TracingContextOnThread : TracingContext /// /// It's also the root of all spans on current thread execution. /// -/// Although it's SAFE to construct this object multiple times on one same thread, it shoud be created at the beginning of one thread execution. +/// Although it's SAFE to construct this object multiple times on one same thread, it should be created at the beginning of one thread execution. struct TracingContextHolder { /// Forbidden copy ctor and assignment to make the destructor safe From e85e885d40f1940d38b6b156a37146b6402387fd Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Thu, 25 Aug 2022 18:15:01 +0800 Subject: [PATCH 138/582] add blank line --- src/Functions/UniqTheta/FunctionsUniqTheta.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.h b/src/Functions/UniqTheta/FunctionsUniqTheta.h index 313f9a15085..7cdbf587cf7 100644 --- a/src/Functions/UniqTheta/FunctionsUniqTheta.h +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.h @@ -88,6 +88,7 @@ namespace DB const DataTypes & arg_data_types0 = sketch_type0->getArgumentsDataTypes(); const DataTypes & arg_data_types1 = sketch_type1->getArgumentsDataTypes(); + if (arg_data_types0.size() != arg_data_types1.size()) throw Exception( "The nested type in uniqThetas must be the same length, but one is " + std::to_string(arg_data_types0.size()) From ae82c687256e69486723872c70fe1a0cb83afda9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 25 Aug 2022 10:52:56 +0000 Subject: [PATCH 139/582] truncate lock --- utils/self-extracting-executable/decompressor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index e687717dacd..73d34f1fe8e 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -521,6 +521,8 @@ int main(int/* argc*/, char* argv[]) return 1; } + ftruncate(lock, 0); + printf("No target executable - decompression only was performed.\n"); } } From bb00dcc19b96122736df10e080ac6bcd7942f6c5 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 25 Aug 2022 20:20:13 +0800 Subject: [PATCH 140/582] Remove using namespace from header Signed-off-by: Frank Chen --- src/Client/Connection.cpp | 4 ++-- src/Common/OpenTelemetryTraceContext.cpp | 8 ++++++++ src/Common/OpenTelemetryTraceContext.h | 3 +-- src/Common/ThreadPool.cpp | 8 ++++---- src/Interpreters/ClientInfo.h | 2 +- src/Interpreters/OpenTelemetrySpanLog.h | 6 +++--- src/Interpreters/executeQuery.cpp | 10 +++++----- .../Executors/ExecutionThreadContext.cpp | 4 ++-- src/Server/GRPCServer.cpp | 10 +++++----- src/Server/HTTPHandler.cpp | 10 +++++----- src/Server/TCPHandler.cpp | 10 +++++----- src/Storages/Distributed/DirectoryMonitor.cpp | 8 ++++---- src/Storages/Distributed/DistributedSink.cpp | 14 +++++++------- src/Storages/StorageURL.cpp | 2 +- 14 files changed, 53 insertions(+), 46 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index ed048d1f986..b8e1bfa13f9 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -484,13 +484,13 @@ void Connection::sendQuery( bool with_pending_data, std::function) { - SpanHolder span("Connection::sendQuery()"); + OpenTelemetry::SpanHolder span("Connection::sendQuery()"); span.addAttribute("clickhouse.query_id", query_id_); span.addAttribute("clickhouse.query", query); span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); }); ClientInfo new_client_info; - const auto ¤t_trace_context = TracingContextOnThread::current(); + const auto ¤t_trace_context = OpenTelemetry::TracingContextOnThread::current(); if (client_info && current_trace_context.isTraceEnabled()) { // use current span as the parent of remote span diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 42c99ebcfd7..0a41e79cd3c 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -36,6 +36,14 @@ void Span::addAttribute(std::string_view name, std::string_view value) this->attributes.push_back(Tuple{name, value}); } +void Span::addAttributeIfNotEmpty(std::string_view name, std::string_view value) +{ + if (!this->isTraceEnabled() || name.empty() || value.empty()) + return; + + this->attributes.push_back(Tuple{name, value}); +} + void Span::addAttribute(std::string_view name, std::function value_supplier) { if (!this->isTraceEnabled() || !value_supplier) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 32557183bfd..60f2b5b3db5 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -24,6 +24,7 @@ struct Span void addAttribute(std::string_view name, UInt64 value); void addAttributeIfNotZero(std::string_view name, UInt64 value); void addAttribute(std::string_view name, std::string_view value); + void addAttributeIfNotEmpty(std::string_view name, std::string_view value); void addAttribute(std::string_view name, std::function value_supplier); /// Following two methods are declared as noexcept to make sure they're exception safe @@ -155,7 +156,5 @@ struct SpanHolder : public Span } -using namespace OpenTelemetry; - } diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 358863d46c7..4c6ed87df3f 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -152,7 +152,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti // this scheduleImpl is called in the parent thread, // the tracing context on this thread is used as parent context for the sub-thread that runs the job - const auto ¤t_thread_context = DB::TracingContextOnThread::current(); + const auto ¤t_thread_context = DB::OpenTelemetry::TracingContextOnThread::current(); jobs.emplace(std::move(job), priority, current_thread_context); ++scheduled_jobs; new_job_or_shutdown.notify_one(); @@ -255,7 +255,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ bool need_shutdown = false; /// A copy of parent trace context - DB::TracingContextOnThread parent_thead_trace_context; + DB::OpenTelemetry::TracingContextOnThread parent_thead_trace_context; { std::unique_lock lock(mutex); @@ -267,7 +267,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// boost::priority_queue does not provide interface for getting non-const reference to an element /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); - parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); + parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); jobs.pop(); } else @@ -281,7 +281,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!need_shutdown) { /// Set up tracing context for this thread by its parent context - DB::TracingContextHolder thread_trace_context("ThreadPool::worker()", parent_thead_trace_context); + DB::OpenTelemetry::TracingContextHolder thread_trace_context("ThreadPool::worker()", parent_thead_trace_context); try { diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 2d8e1a1764f..a1096b99325 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -63,7 +63,7 @@ public: Decimal64 initial_query_start_time_microseconds{}; /// OpenTelemetry trace context we received from client, or which we are going to send to server. - TracingContext client_trace_context; + OpenTelemetry::TracingContext client_trace_context; /// All below are parameters related to initial query. diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 9a7bbad4ee4..e5a5b082284 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -7,11 +7,11 @@ namespace DB { -struct OpenTelemetrySpanLogElement : public Span +struct OpenTelemetrySpanLogElement : public OpenTelemetry::Span { OpenTelemetrySpanLogElement() = default; - OpenTelemetrySpanLogElement(const Span & span) - : Span(span) {} + OpenTelemetrySpanLogElement(const OpenTelemetry::Span & span) + : OpenTelemetry::Span(span) {} static std::string name() { return "OpenTelemetrySpanLog"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e467d6fb80e..abf7c7a723e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -233,7 +233,7 @@ inline UInt64 time_in_seconds(std::chrono::time_point return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); } -static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr & query_span) +static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr & query_span) { /// Exception before the query execution. if (auto quota = context->getQuota()) @@ -345,7 +345,7 @@ static std::tuple executeQueryImpl( QueryProcessingStage::Enum stage, ReadBuffer * istr) { - std::shared_ptr query_span = std::make_shared("query"); + std::shared_ptr query_span = std::make_shared("query"); const auto current_time = std::chrono::system_clock::now(); @@ -667,12 +667,12 @@ static std::tuple executeQueryImpl( } { - std::unique_ptr span; + std::unique_ptr span; if (query_span->isTraceEnabled()) { auto * raw_interpreter_ptr = interpreter.get(); std::string class_name(demangle(typeid(*raw_interpreter_ptr).name())); - span = std::make_unique(class_name + "::execute()"); + span = std::make_unique(class_name + "::execute()"); } res = interpreter->execute(); } @@ -930,8 +930,8 @@ static std::tuple executeQueryImpl( query_span->addAttribute("db.statement", elem.query); query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); - query_span->addAttribute("clickhouse.tracestate", TracingContextOnThread::current().tracestate); query_span->addAttribute("clickhouse.query_status", "QueryFinish"); + query_span->addAttributeIfNotEmpty("clickhouse.tracestate", OpenTelemetry::TracingContextOnThread::current().tracestate); query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); diff --git a/src/Processors/Executors/ExecutionThreadContext.cpp b/src/Processors/Executors/ExecutionThreadContext.cpp index 8825e3f131d..eddc1b76d8a 100644 --- a/src/Processors/Executors/ExecutionThreadContext.cpp +++ b/src/Processors/Executors/ExecutionThreadContext.cpp @@ -71,11 +71,11 @@ static void executeJob(ExecutingGraph::Node * node, ReadProgressCallback * read_ bool ExecutionThreadContext::executeTask() { - std::unique_ptr span; + std::unique_ptr span; if (trace_processors) { - span = std::make_unique("ExecutionThreadContext::executeTask() " + node->processor->getName()); + span = std::make_unique("ExecutionThreadContext::executeTask() " + node->processor->getName()); span->addAttribute("thread_number", thread_number); } std::optional execution_time_watch; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 390c8a23cf8..48102a7e99e 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -662,7 +662,7 @@ namespace std::optional session; ContextMutablePtr query_context; std::optional query_scope; - TracingContextHolderPtr thread_trace_context; + OpenTelemetry::TracingContextHolderPtr thread_trace_context; String query_text; ASTPtr ast; ASTInsertQuery * insert_query = nullptr; @@ -842,10 +842,10 @@ namespace query_scope.emplace(query_context); /// Set up tracing context for this query on current thread - thread_trace_context = std::make_unique("GRPCServer", - query_context->getClientInfo().client_trace_context, - query_context->getSettingsRef(), - query_context->getOpenTelemetrySpanLog()); + thread_trace_context = std::make_unique("GRPCServer", + query_context->getClientInfo().client_trace_context, + query_context->getSettingsRef(), + query_context->getOpenTelemetrySpanLog()); /// Prepare for sending exceptions and logs. const Settings & settings = query_context->getSettingsRef(); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 251a3b3beb3..442233ab408 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -933,7 +933,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// In case of exception, send stack trace to client. bool with_stacktrace = false; - TracingContextHolderPtr thread_trace_context; + OpenTelemetry::TracingContextHolderPtr thread_trace_context; SCOPE_EXIT({ // make sure the response status is recorded if (thread_trace_context) @@ -963,10 +963,10 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); - thread_trace_context = std::make_unique("HTTPHandler", - client_info.client_trace_context, - context->getSettingsRef(), - context->getOpenTelemetrySpanLog()); + thread_trace_context = std::make_unique("HTTPHandler", + client_info.client_trace_context, + context->getSettingsRef(), + context->getOpenTelemetrySpanLog()); thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); response.setContentType("text/plain; charset=UTF-8"); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 491bb2a10bb..550ae1bff31 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -219,7 +219,7 @@ void TCPHandler::runImpl() /// Initialized later. std::optional query_scope; - TracingContextHolderPtr thread_trace_context; + OpenTelemetry::TracingContextHolderPtr thread_trace_context; /** An exception during the execution of request (it must be sent over the network to the client). * The client will be able to accept it, if it did not happen while sending another packet and the client has not disconnected yet. @@ -246,10 +246,10 @@ void TCPHandler::runImpl() continue; /// Set up tracing context for this query on current thread - thread_trace_context = std::make_unique("TCPHandler", - query_context->getClientInfo().client_trace_context, - query_context->getSettingsRef(), - query_context->getOpenTelemetrySpanLog()); + thread_trace_context = std::make_unique("TCPHandler", + query_context->getClientInfo().client_trace_context, + query_context->getSettingsRef(), + query_context->getOpenTelemetrySpanLog()); query_scope.emplace(query_context); diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 89673a76582..16981d26146 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -609,7 +609,7 @@ bool StorageDistributedDirectoryMonitor::processFiles(const std::mapgetSettingsRef()); @@ -629,7 +629,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa formatReadableQuantity(distributed_header.rows), formatReadableSizeWithBinarySuffix(distributed_header.bytes)); - thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, + thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, this->storage.getContext()->getOpenTelemetrySpanLog()); @@ -870,7 +870,7 @@ private: ReadBufferFromFile in(file_path->second); const auto & distributed_header = readDistributedHeader(in, parent.log); - TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, + OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, parent.storage.getContext()->getOpenTelemetrySpanLog()); @@ -909,7 +909,7 @@ private: const auto & distributed_header = readDistributedHeader(in, parent.log); // this function is called in a separated thread, so we set up the trace context from the file - TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, + OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, parent.storage.getContext()->getOpenTelemetrySpanLog()); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index ffecaf4c1b4..630234f1e08 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -336,7 +336,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si if (rows == 0) return; - SpanHolder span(__PRETTY_FUNCTION__); + OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); span.addAttribute("clickhouse.written_rows", rows); @@ -419,7 +419,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si void DistributedSink::writeSync(const Block & block) { - SpanHolder span(__PRETTY_FUNCTION__); + OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); const Settings & settings = context->getSettingsRef(); const auto & shards_info = cluster->getShardsInfo(); @@ -610,7 +610,7 @@ void DistributedSink::writeSplitAsync(const Block & block) void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) { - SpanHolder span("DistributedSink::writeAsyncImpl()"); + OpenTelemetry::SpanHolder span("DistributedSink::writeAsyncImpl()"); const auto & shard_info = cluster->getShardsInfo()[shard_id]; const auto & settings = context->getSettingsRef(); @@ -652,7 +652,7 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) void DistributedSink::writeToLocal(const Block & block, size_t repeats) { - SpanHolder span(__PRETTY_FUNCTION__); + OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("db.statement", this->query_string); InterpreterInsertQuery interp(query_ast, context, allow_materialized); @@ -668,7 +668,7 @@ void DistributedSink::writeToLocal(const Block & block, size_t repeats) void DistributedSink::writeToShard(const Block & block, const std::vector & dir_names) { - SpanHolder span(__PRETTY_FUNCTION__); + OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); const auto & settings = context->getSettingsRef(); const auto & distributed_settings = storage.getDistributedSettingsRef(); @@ -737,11 +737,11 @@ void DistributedSink::writeToShard(const Block & block, const std::vectorgetSettingsRef().write(header_buf); - if (TracingContextOnThread::current().isTraceEnabled()) + if (OpenTelemetry::TracingContextOnThread::current().isTraceEnabled()) { // if the distributed tracing is enabled, use the trace context in current thread as parent of next span auto client_info = context->getClientInfo(); - client_info.client_trace_context = TracingContextOnThread::current(); + client_info.client_trace_context = OpenTelemetry::TracingContextOnThread::current(); client_info.write(header_buf, DBMS_TCP_PROTOCOL_VERSION); } else diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 163fd51eeba..7e7885617a3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -102,7 +102,7 @@ namespace ReadWriteBufferFromHTTP::HTTPHeaderEntries headers(headers_.begin(), headers_.end()); // Propagate OpenTelemetry trace context, if any, downstream. - const auto ¤t_trace_context = TracingContextOnThread::current(); + const auto ¤t_trace_context = OpenTelemetry::TracingContextOnThread::current(); if (current_trace_context.isTraceEnabled()) { headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); From 92f7ca361635ae7ece249010156f0bff47498270 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 25 Aug 2022 20:23:56 +0800 Subject: [PATCH 141/582] Move TracingContextOnThread::current() out of class for simplicity Signed-off-by: Frank Chen --- src/Client/Connection.cpp | 2 +- src/Common/OpenTelemetryTraceContext.cpp | 2 +- src/Common/OpenTelemetryTraceContext.h | 5 +++-- src/Common/ThreadPool.cpp | 2 +- src/Interpreters/executeQuery.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 4 ++-- src/Storages/StorageURL.cpp | 2 +- 7 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index b8e1bfa13f9..30f5af5a540 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -490,7 +490,7 @@ void Connection::sendQuery( span.addAttribute("target", [this] () { return this->getHost() + ":" + std::to_string(this->getPort()); }); ClientInfo new_client_info; - const auto ¤t_trace_context = OpenTelemetry::TracingContextOnThread::current(); + const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); if (client_info && current_trace_context.isTraceEnabled()) { // use current span as the parent of remote span diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 0a41e79cd3c..adc37f83790 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -211,7 +211,7 @@ String TracingContext::composeTraceparentHeader() const static_cast(trace_flags)); } -const TracingContextOnThread & TracingContextOnThread::current() +const TracingContextOnThread & CurrentContext() { return current_thread_trace_context; } diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 60f2b5b3db5..99412f41286 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -76,13 +76,14 @@ struct TracingContextOnThread : TracingContext void reset(); - static const TracingContextOnThread& current(); - /// Use weak_ptr instead of shared_ptr to hold a reference to the underlying system.opentelemetry_span_log table /// Since this object is kept on threads and passed across threads, a weak_ptr is more safe to prevent potential leak std::weak_ptr span_log; }; +/// Get tracing context on current thread +const TracingContextOnThread& CurrentContext(); + /// Holder of tracing context. /// It should be initialized at the beginning of each thread execution. /// And once it's destructed, it clears the context automatically. diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 4c6ed87df3f..447b3d98f83 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -152,7 +152,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti // this scheduleImpl is called in the parent thread, // the tracing context on this thread is used as parent context for the sub-thread that runs the job - const auto ¤t_thread_context = DB::OpenTelemetry::TracingContextOnThread::current(); + const auto ¤t_thread_context = DB::OpenTelemetry::CurrentContext(); jobs.emplace(std::move(job), priority, current_thread_context); ++scheduled_jobs; new_job_or_shutdown.notify_one(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index abf7c7a723e..ee7879bbbc7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -931,7 +931,7 @@ static std::tuple executeQueryImpl( query_span->addAttribute("db.statement", elem.query); query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); query_span->addAttribute("clickhouse.query_status", "QueryFinish"); - query_span->addAttributeIfNotEmpty("clickhouse.tracestate", OpenTelemetry::TracingContextOnThread::current().tracestate); + query_span->addAttributeIfNotEmpty("clickhouse.tracestate", OpenTelemetry::CurrentContext().tracestate); query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 630234f1e08..ae72fdd84e2 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -737,11 +737,11 @@ void DistributedSink::writeToShard(const Block & block, const std::vectorgetSettingsRef().write(header_buf); - if (OpenTelemetry::TracingContextOnThread::current().isTraceEnabled()) + if (OpenTelemetry::CurrentContext().isTraceEnabled()) { // if the distributed tracing is enabled, use the trace context in current thread as parent of next span auto client_info = context->getClientInfo(); - client_info.client_trace_context = OpenTelemetry::TracingContextOnThread::current(); + client_info.client_trace_context = OpenTelemetry::CurrentContext(); client_info.write(header_buf, DBMS_TCP_PROTOCOL_VERSION); } else diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 7e7885617a3..5c03b872283 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -102,7 +102,7 @@ namespace ReadWriteBufferFromHTTP::HTTPHeaderEntries headers(headers_.begin(), headers_.end()); // Propagate OpenTelemetry trace context, if any, downstream. - const auto ¤t_trace_context = OpenTelemetry::TracingContextOnThread::current(); + const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); if (current_trace_context.isTraceEnabled()) { headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); From 10fdc1e286a717a368b2f0a9933f4ec83ec6aa05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Aug 2022 14:52:01 +0200 Subject: [PATCH 142/582] Fix stupid bug --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index be23b6c129d..5d1b5f1dc33 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -170,7 +170,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati if (!metadata_storage.exists(path)) { if (if_exists) - return; + continue; throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata path '{}' doesn't exist", path); } From 7c668be23a909d5c481d2c2806adca1b13472b42 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Aug 2022 14:10:43 +0000 Subject: [PATCH 143/582] Fix drop --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 9be462d15a3..f3917fe6d7d 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -357,7 +357,7 @@ StorageKeeperMap::StorageKeeperMap( throw Coordination::Exception(code, table_path); } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or becauyse of wrong root_path ({})", root_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or because of wrong root_path ({})", root_path); } From 09aaa2b5dd477dfa03ba1c47055d776e6a0f8e05 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Aug 2022 20:44:40 +0200 Subject: [PATCH 144/582] Do not obtain storage snapshot for each INSERT block (improves performance) It depends on how much parts do you have, but for some workload with InMemory only parts w/o merges, I got 5% increase. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeSink.cpp | 2 +- src/Storages/MergeTree/MergeTreeSink.h | 3 +++ src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeSink.h | 4 ++++ 4 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 83203939903..5eaa8ec8004 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -23,6 +23,7 @@ MergeTreeSink::MergeTreeSink( , metadata_snapshot(metadata_snapshot_) , max_parts_per_block(max_parts_per_block_) , context(context_) + , storage_snapshot(storage.getStorageSnapshot(metadata_snapshot, context)) { } @@ -54,7 +55,6 @@ struct MergeTreeSink::DelayedChunk void MergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - auto storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, context); storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreeSink.h b/src/Storages/MergeTree/MergeTreeSink.h index 65a565d7f57..68f11d86a25 100644 --- a/src/Storages/MergeTree/MergeTreeSink.h +++ b/src/Storages/MergeTree/MergeTreeSink.h @@ -9,6 +9,8 @@ namespace DB class Block; class StorageMergeTree; +struct StorageSnapshot; +using StorageSnapshotPtr = std::shared_ptr; class MergeTreeSink : public SinkToStorage @@ -32,6 +34,7 @@ private: StorageMetadataPtr metadata_snapshot; size_t max_parts_per_block; ContextPtr context; + StorageSnapshotPtr storage_snapshot; uint64_t chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token /// We can delay processing for previous chunk and start writing a new one. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 34d64b92d69..dc01df22886 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -65,6 +65,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( , deduplicate(deduplicate_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) , context(context_) + , storage_snapshot(storage.getStorageSnapshot(metadata_snapshot, context)) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) @@ -150,7 +151,6 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (quorum) checkQuorumPrecondition(zookeeper); - auto storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, context); storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index f7504d2f784..3a329b8d6c6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -17,6 +17,8 @@ namespace DB { class StorageReplicatedMergeTree; +struct StorageSnapshot; +using StorageSnapshotPtr = std::shared_ptr; class ReplicatedMergeTreeSink : public SinkToStorage @@ -96,6 +98,8 @@ private: Poco::Logger * log; ContextPtr context; + StorageSnapshotPtr storage_snapshot; + UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token /// We can delay processing for previous chunk and start writing a new one. From cf1081eada5a5553027c88774702b46bde6a5c6f Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 26 Aug 2022 11:18:16 +0800 Subject: [PATCH 145/582] Exception safe from destructor Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 29 ++++++++++++++++++------ src/Common/OpenTelemetryTraceContext.h | 2 +- 2 files changed, 23 insertions(+), 8 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index adc37f83790..d296ed54422 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -102,7 +102,7 @@ SpanHolder::SpanHolder(std::string_view _operation_name) } } -void SpanHolder::finish() +void SpanHolder::finish() noexcept { if (!this->isTraceEnabled()) return; @@ -294,14 +294,29 @@ TracingContextHolder::~TracingContextHolder() { if (this->root_span.isTraceEnabled()) { - auto shared_span_log = current_thread_trace_context.span_log.lock(); - if (shared_span_log) + try { - this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); - this->root_span.finish_time_us - = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + auto shared_span_log = current_thread_trace_context.span_log.lock(); + if (shared_span_log) + { + try + { + this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); + } + catch (...) + { + /// Ignore any exceptions + } - shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); + this->root_span.finish_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); + } + } + catch (...) + { + tryLogCurrentException(__FUNCTION__); } this->root_span.trace_id = UUID(); diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 99412f41286..3964b5030fc 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -152,7 +152,7 @@ struct SpanHolder : public Span /// Finish a span explicitly if needed. /// It's safe to call it multiple times - void finish(); + void finish() noexcept; }; } From 704c4b2c5bf3d653378a2e513cfb56cc2917bb4a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Aug 2022 11:54:54 +0200 Subject: [PATCH 146/582] Stop thread fuzzer on shutdown --- docker/test/stress/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 3def6fecdef..ee38f6f7c5d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -211,6 +211,8 @@ clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC" clickhouse-client --query "SHOW TABLES FROM test" +clickhouse-client --query "SYSTEM STOP THREAD FUZZER" + stop # Let's enable S3 storage by default From 3c478da003707f3c07fd1184bb5d8f8aa5c8840c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Aug 2022 23:32:00 +0200 Subject: [PATCH 147/582] Move committing InMemory parts to WAL out of NOEXCEPT_SCOPE() Since this commit can definitelly throw (i.e. due to ENOSPC). Note, that it should be safe, since rollback() will call dropPart() for those parts. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 37 +++++++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 3 +- 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c19e4f5f559..ad8e1cfb02d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4975,6 +4975,8 @@ void MergeTreeData::Transaction::rollbackPartsToTemporaryState() void MergeTreeData::Transaction::addPart(MutableDataPartPtr & part, DataPartStorageBuilderPtr builder) { precommitted_parts.insert(part); + if (asInMemoryPart(part)) + has_in_memory_parts = true; part_builders.push_back(builder); } @@ -4997,6 +4999,12 @@ void MergeTreeData::Transaction::rollback() clear(); } +void MergeTreeData::Transaction::clear() +{ + precommitted_parts.clear(); + has_in_memory_parts = false; +} + MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData::DataPartsLock * acquired_parts_lock) { DataPartsVector total_covered_parts; @@ -5004,20 +5012,30 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: if (!isEmpty()) { auto settings = data.getSettings(); - MergeTreeData::WriteAheadLogPtr wal; auto parts_lock = acquired_parts_lock ? MergeTreeData::DataPartsLock() : data.lockParts(); auto * owing_parts_lock = acquired_parts_lock ? acquired_parts_lock : &parts_lock; for (auto & builder : part_builders) builder->commit(); - if (txn) + bool commit_to_wal = has_in_memory_parts && settings->in_memory_parts_enable_wal; + if (txn || commit_to_wal) { + MergeTreeData::WriteAheadLogPtr wal; + if (commit_to_wal) + wal = data.getWriteAheadLog(); + for (const DataPartPtr & part : precommitted_parts) { - DataPartPtr covering_part; - DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); - MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, txn); + if (txn) + { + DataPartPtr covering_part; + DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); + MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, txn); + } + + if (auto part_in_memory = asInMemoryPart(part)) + wal->addPart(part_in_memory); } } @@ -5034,15 +5052,6 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: for (const DataPartPtr & part : precommitted_parts) { - auto part_in_memory = asInMemoryPart(part); - if (part_in_memory && settings->in_memory_parts_enable_wal) - { - if (!wal) - wal = data.getWriteAheadLog(); - - wal->addPart(part_in_memory); - } - DataPartPtr covering_part; DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); if (covering_part) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 68ec191412b..f4becdf5892 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -290,8 +290,9 @@ public: DataParts precommitted_parts; std::vector part_builders; DataParts locked_parts; + bool has_in_memory_parts = false; - void clear() { precommitted_parts.clear(); } + void clear(); }; using TransactionUniquePtr = std::unique_ptr; From b42d26acfe9a5a7c459f738c7cbc50c4f5a6e519 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Aug 2022 02:44:02 +0200 Subject: [PATCH 148/582] Remove one line from XML, because we do not care --- programs/diagnostics/testdata/configs/xml/config.xml | 1 - programs/diagnostics/testdata/configs/xml/users.xml | 1 - programs/diagnostics/testdata/configs/yandex_xml/config.xml | 1 - programs/server/config.xml | 1 - programs/server/embedded.xml | 1 - programs/server/users.d/allow_introspection_functions.xml | 1 - programs/server/users.d/allow_only_from_localhost.xml | 1 - programs/server/users.xml | 1 - src/Common/tests/gtest_getMultipleValuesFromConfig.cpp | 1 - src/Common/tests/gtest_sensitive_data_masker.cpp | 4 ---- tests/config/config.d/query_masking_rules.xml | 1 - tests/config/users.d/readonly.xml | 1 - tests/config/users.d/session_log_test.xml | 1 - tests/integration/helpers/0_common_enable_dictionaries.xml | 1 - .../test_allowed_client_hosts/configs/users.d/network.xml | 1 - .../test_alter_update_cast_keep_nullable/configs/users.xml | 1 - .../test_async_drain_connection/configs/config.xml | 1 - .../test_backup_restore_new/configs/backups_disk.xml | 1 - .../test_backup_restore_on_cluster/configs/backups_disk.xml | 1 - .../configs/lesser_timeouts.xml | 1 - .../test_cluster_copier/configs/conf.d/clusters.xml | 1 - .../test_cluster_copier/configs/conf.d/clusters_trivial.xml | 1 - tests/integration/test_cluster_copier/configs/users.xml | 1 - .../configs_three_nodes/conf.d/clusters.xml | 1 - .../test_cluster_copier/configs_three_nodes/conf.d/ddl.xml | 1 - .../test_cluster_copier/configs_three_nodes/config-copier.xml | 1 - .../test_cluster_copier/configs_three_nodes/users.xml | 1 - .../test_cluster_copier/configs_two_nodes/conf.d/clusters.xml | 1 - .../test_cluster_copier/configs_two_nodes/conf.d/ddl.xml | 1 - .../test_cluster_copier/configs_two_nodes/config-copier.xml | 1 - .../test_cluster_copier/configs_two_nodes/users.xml | 1 - tests/integration/test_cluster_copier/task0_description.xml | 1 - .../test_cluster_copier/task_drop_target_partition.xml | 1 - .../test_cluster_copier/task_month_to_week_description.xml | 1 - tests/integration/test_cluster_copier/task_self_copy.xml | 1 - tests/integration/test_cluster_copier/task_skip_index.xml | 1 - tests/integration/test_cluster_copier/task_taxi_data.xml | 1 - .../integration/test_cluster_copier/task_test_block_size.xml | 1 - tests/integration/test_cluster_copier/task_trivial.xml | 1 - .../test_cluster_copier/task_trivial_without_arguments.xml | 1 - tests/integration/test_cluster_copier/task_ttl_columns.xml | 1 - .../test_cluster_copier/task_ttl_move_to_volume.xml | 1 - .../test_cluster_copier/task_with_different_schema.xml | 1 - tests/integration/test_codec_encrypted/test.py | 1 - .../configs/config_default.xml | 1 - .../configs/config_defined_1.xml | 1 - .../configs/config_defined_50.xml | 1 - .../configs/config_limit_reached.xml | 1 - .../test_concurrent_threads_soft_limit/configs/users.xml | 1 - .../test_config_corresponding_root/configs/config.xml | 1 - .../test_config_corresponding_root/configs/users.xml | 1 - .../configs/config.d/query_masking_rules.xml | 1 - tests/integration/test_config_xml_full/configs/config.xml | 1 - tests/integration/test_config_xml_full/configs/embedded.xml | 1 - .../configs/users.d/allow_introspection_functions.xml | 1 - tests/integration/test_config_xml_full/configs/users.xml | 1 - tests/integration/test_config_xml_main/configs/config.xml | 1 - tests/integration/test_config_xml_main/configs/embedded.xml | 1 - tests/integration/test_config_xml_main/configs/users.xml | 1 - .../configs/config.d/query_masking_rules.xml | 1 - tests/integration/test_config_xml_yaml_mix/configs/config.xml | 1 - .../integration/test_config_xml_yaml_mix/configs/embedded.xml | 1 - .../configs/users.d/allow_introspection_functions.xml | 1 - tests/integration/test_config_yaml_full/configs/embedded.xml | 1 - .../configs/config.d/query_masking_rules.xml | 1 - tests/integration/test_config_yaml_main/configs/embedded.xml | 1 - .../configs/users.d/allow_introspection_functions.xml | 1 - tests/integration/test_create_user_and_login/test.py | 1 - .../configs/config.xml | 1 - .../configs/disable_ssl_verification.xml | 1 - .../configs/users.xml | 1 - .../configs/config.xml | 1 - .../configs/users.xml | 1 - .../integration/test_dictionaries_ddl/configs/user_admin.xml | 1 - .../test_dictionaries_ddl/configs/user_default.xml | 1 - .../test_dictionaries_dependency_xml/configs/config.xml | 1 - .../test_dictionaries_dependency_xml/configs/users.xml | 1 - tests/integration/test_dictionaries_mysql/configs/config.xml | 1 - .../configs/dictionaries/mysql_dict1.xml | 1 - .../configs/dictionaries/mysql_dict2.xml | 1 - tests/integration/test_dictionaries_mysql/configs/users.xml | 1 - .../test_dictionaries_null_value/configs/config.xml | 1 - .../test_dictionaries_null_value/configs/users.xml | 1 - .../test_dictionaries_postgresql/configs/config.xml | 1 - .../configs/dictionaries/postgres_dict.xml | 1 - .../test_dictionaries_select_all/configs/config.xml | 1 - .../test_dictionaries_select_all/configs/users.xml | 1 - .../test_dictionaries_update_and_reload/configs/config.xml | 1 - .../configs/dictionaries/executable.xml | 1 - .../configs/dictionaries/file.xml | 1 - .../configs/dictionaries/slow.xml | 1 - .../test_dictionaries_update_and_reload/configs/users.xml | 1 - .../test_dictionaries_update_field/configs/config.xml | 1 - .../test_dictionaries_update_field/configs/users.xml | 1 - .../configs/config.xml | 1 - .../test_dictionary_allow_read_expired_keys/configs/users.xml | 1 - .../test_dictionary_custom_settings/configs/config.xml | 1 - .../test_dictionary_custom_settings/configs/users.xml | 1 - .../test_disabled_access_control_improvements/all_rows.xml | 1 - .../configs/users.d/another_user.xml | 1 - .../configs/users.xml | 1 - .../test_disabled_access_control_improvements/no_filters.xml | 1 - .../test_disabled_access_control_improvements/no_rows.xml | 1 - .../normal_filters.xml | 1 - .../test_disk_over_web_server/configs/storage_conf.xml | 1 - .../test_disk_over_web_server/configs/storage_conf_web.xml | 1 - tests/integration/test_disk_types/configs/config.xml | 1 - tests/integration/test_disk_types/configs/storage.xml | 1 - tests/integration/test_disk_types/configs/users.xml | 1 - tests/integration/test_disks_app_func/config.xml | 1 - .../test_distributed_ddl_parallel/configs/dict.xml | 1 - .../test_distributed_default_database/configs/users.xml | 1 - .../test_distributed_inter_server_secret/configs/users.xml | 1 - .../test_distributed_load_balancing/configs/users.xml | 1 - tests/integration/test_dotnet_client/configs/config.xml | 1 - tests/integration/test_dotnet_client/configs/users.xml | 1 - tests/integration/test_encrypted_disk/configs/storage.xml | 1 - tests/integration/test_encrypted_disk/test.py | 1 - .../configs/config.d/query_log.xml | 1 - .../configs/config.d/schema_cache.xml | 1 - tests/integration/test_graphite_merge_tree/configs/users.xml | 1 - .../test_graphite_merge_tree_typed/configs/users.xml | 1 - tests/integration/test_hedged_requests/configs/users.xml | 1 - tests/integration/test_hedged_requests/configs/users1.xml | 1 - .../test_hedged_requests_parallel/configs/users.xml | 1 - .../test_hedged_requests_parallel/configs/users1.xml | 1 - .../test_defaults_handlers/config.xml | 1 - .../test_http_handlers_config/test_dynamic_handler/config.xml | 1 - .../test_predefined_handler/config.xml | 1 - .../test_prometheus_handler/config.xml | 1 - .../test_replicas_status_handler/config.xml | 1 - .../test_http_handlers_config/test_static_handler/config.xml | 1 - tests/integration/test_https_replication/configs/config.xml | 1 - tests/integration/test_jdbc_bridge/configs/jdbc_bridge.xml | 1 - tests/integration/test_join_set_family_s3/configs/config.xml | 1 - tests/integration/test_join_set_family_s3/configs/minio.xml | 1 - tests/integration/test_join_set_family_s3/configs/ssl.xml | 1 - tests/integration/test_join_set_family_s3/configs/users.xml | 1 - .../test_library_bridge/configs/dictionaries/dict1.xml | 1 - .../integration/test_log_family_hdfs/configs/storage_conf.xml | 1 - tests/integration/test_log_family_s3/configs/config.xml | 1 - tests/integration/test_log_family_s3/configs/minio.xml | 1 - tests/integration/test_log_family_s3/configs/ssl.xml | 1 - tests/integration/test_log_family_s3/configs/users.xml | 1 - .../test_logs_level/configs/config_information.xml | 1 - .../configs/config.xml | 1 - .../configs/users.xml | 1 - .../test_materialized_mysql_database/configs/users.xml | 1 - .../configs/users_disable_bytes_settings.xml | 1 - .../configs/users_disable_rows_settings.xml | 1 - .../test_merge_tree_azure_blob_storage/configs/config.xml | 1 - tests/integration/test_merge_tree_s3/configs/config.xml | 1 - .../test_merge_tree_s3_failover/configs/config.d/part_log.xml | 1 - .../test_merge_tree_s3_failover/configs/config.xml | 1 - .../test_merge_tree_s3_restore/configs/config.d/clusters.xml | 1 - .../integration/test_merge_tree_s3_restore/configs/config.xml | 1 - .../configs/config.d/query_log.xml | 1 - .../configs/config.d/ssl_conf.xml | 1 - .../test_merge_tree_s3_with_cache/configs/config.xml | 1 - .../test_mutations_with_merge_tree/configs/config.xml | 1 - .../test_mutations_with_merge_tree/configs/users.xml | 1 - .../test_mutations_with_projection/configs/config.xml | 1 - .../test_mutations_with_projection/configs/users.xml | 1 - tests/integration/test_mysql_protocol/configs/config.xml | 1 - tests/integration/test_mysql_protocol/configs/mysql.xml | 1 - tests/integration/test_mysql_protocol/configs/ssl_conf.xml | 1 - tests/integration/test_mysql_protocol/configs/users.xml | 1 - tests/integration/test_nlp/configs/dicts_config.xml | 1 - tests/integration/test_odbc_interaction/configs/config.xml | 1 - .../test_odbc_interaction/configs/odbc_logging.xml | 1 - tests/integration/test_odbc_interaction/configs/openssl.xml | 1 - tests/integration/test_odbc_interaction/configs/users.xml | 1 - tests/integration/test_postgresql_protocol/configs/config.xml | 1 - .../test_postgresql_protocol/configs/default_passwd.xml | 1 - tests/integration/test_postgresql_protocol/configs/log.xml | 1 - .../test_postgresql_protocol/configs/postresql.xml | 1 - .../integration/test_postgresql_protocol/configs/ssl_conf.xml | 1 - tests/integration/test_postgresql_protocol/configs/users.xml | 1 - .../configs/users.xml | 1 - .../configs/users.xml | 1 - tests/integration/test_profile_events_s3/configs/config.xml | 1 - tests/integration/test_profile_events_s3/configs/log.xml | 1 - .../integration/test_profile_events_s3/configs/query_log.xml | 1 - tests/integration/test_profile_events_s3/configs/ssl_conf.xml | 1 - tests/integration/test_profile_events_s3/configs/users.xml | 1 - .../test_quorum_inserts/configs/users.d/settings.xml | 1 - tests/integration/test_quota/no_quotas.xml | 1 - tests/integration/test_quota/normal_limits.xml | 1 - tests/integration/test_quota/simpliest.xml | 1 - tests/integration/test_quota/tiny_limits.xml | 1 - tests/integration/test_quota/tracking.xml | 1 - tests/integration/test_quota/two_intervals.xml | 1 - tests/integration/test_quota/two_quotas.xml | 1 - tests/integration/test_relative_filepath/configs/config.xml | 1 - .../test_reload_auxiliary_zookeepers/configs/config.xml | 1 - .../test_reload_auxiliary_zookeepers/configs/users.xml | 1 - tests/integration/test_reload_certificate/configs/cert.xml | 1 - tests/integration/test_reload_certificate/test.py | 1 - .../test_reload_max_table_size_to_drop/configs/config.xml | 1 - .../configs/max_table_size_to_drop.xml | 1 - .../test_reload_max_table_size_to_drop/configs/users.xml | 1 - tests/integration/test_reload_zookeeper/configs/config.xml | 1 - tests/integration/test_reload_zookeeper/configs/users.xml | 1 - .../configs/changed_settings.xml | 1 - .../configs/normal_settings.xml | 1 - .../configs/unexpected_setting_enum.xml | 1 - .../configs/unexpected_setting_int.xml | 1 - .../configs/unknown_setting.xml | 1 - .../configs/limit_replication_config.xml | 1 - .../test_replicated_merge_tree_config/configs/config.xml | 1 - .../configs/config.d/clusters.xml | 1 - tests/integration/test_row_policy/all_rows.xml | 1 - .../test_row_policy/configs/users.d/another_user.xml | 1 - .../configs/users.d/any_join_distinct_right_table_keys.xml | 1 - tests/integration/test_row_policy/configs/users.xml | 1 - tests/integration/test_row_policy/no_filters.xml | 1 - tests/integration/test_row_policy/no_rows.xml | 1 - tests/integration/test_row_policy/normal_filter2_table2.xml | 1 - tests/integration/test_row_policy/normal_filters.xml | 1 - .../test_row_policy/tags_with_db_and_table_names.xml | 1 - tests/integration/test_s3_with_https/configs/config.d/ssl.xml | 1 - tests/integration/test_s3_with_https/configs/config.xml | 1 - tests/integration/test_s3_with_proxy/configs/config.xml | 1 - .../test_select_access_rights/configs/another_user.xml | 1 - .../configs/config_send_crash_reports.xml | 1 - .../integration/test_server_reload/configs/default_passwd.xml | 1 - tests/integration/test_server_reload/configs/ssl_conf.xml | 1 - tests/integration/test_settings_constraints/configs/users.xml | 1 - .../configs/users_with_ssl_auth.xml | 1 - tests/integration/test_storage_dict/configs/conf.xml | 1 - tests/integration/test_storage_kafka/configs/users.xml | 1 - .../test_storage_kerberized_kafka/configs/users.xml | 1 - tests/integration/test_storage_mysql/configs/users.xml | 1 - tests/integration/test_storage_nats/configs/users.xml | 1 - .../integration/test_storage_postgresql/configs/settings.xml | 1 - tests/integration/test_storage_rabbitmq/configs/users.xml | 1 - tests/integration/test_storage_url/configs/conf.xml | 1 - .../test_structured_logging_json/configs/config_json.xml | 1 - .../test_system_queries/configs/config.d/clusters_config.xml | 1 - .../configs/config.d/dictionaries_config.xml | 1 - .../test_system_queries/configs/config.d/query_log.xml | 1 - tests/integration/test_system_queries/configs/users.xml | 1 - tests/integration/test_timezone_config/configs/config.xml | 1 - .../integration/test_tlsv1_3/configs/users_with_ssl_auth.xml | 1 - .../test_user_zero_database_access/configs/config.xml | 1 - .../test_user_zero_database_access/configs/users.xml | 1 - .../0_stateless/01594_too_low_memory_limits.config.xml | 1 - .../01737_clickhouse_server_wait_server_pool_long.config.xml | 1 - .../01954_clickhouse_benchmark_multiple_long.config.xml | 1 - .../02207_allow_plaintext_and_no_password.config.xml | 1 - 250 files changed, 253 deletions(-) diff --git a/programs/diagnostics/testdata/configs/xml/config.xml b/programs/diagnostics/testdata/configs/xml/config.xml index a41c3b8c04e..18997855955 100644 --- a/programs/diagnostics/testdata/configs/xml/config.xml +++ b/programs/diagnostics/testdata/configs/xml/config.xml @@ -1,4 +1,3 @@ - diff --git a/programs/diagnostics/testdata/configs/yandex_xml/config.xml b/programs/diagnostics/testdata/configs/yandex_xml/config.xml index 5133ce93ae5..8854754fc3c 100644 --- a/programs/diagnostics/testdata/configs/yandex_xml/config.xml +++ b/programs/diagnostics/testdata/configs/yandex_xml/config.xml @@ -1,4 +1,3 @@ - diff --git a/programs/server/users.d/allow_introspection_functions.xml b/programs/server/users.d/allow_introspection_functions.xml index cfde1b4525d..ec3057c82d7 100644 --- a/programs/server/users.d/allow_introspection_functions.xml +++ b/programs/server/users.d/allow_introspection_functions.xml @@ -1,4 +1,3 @@ - diff --git a/programs/server/users.d/allow_only_from_localhost.xml b/programs/server/users.d/allow_only_from_localhost.xml index b1b38686133..406b89b4d7c 100644 --- a/programs/server/users.d/allow_only_from_localhost.xml +++ b/programs/server/users.d/allow_only_from_localhost.xml @@ -1,4 +1,3 @@ - diff --git a/programs/server/users.xml b/programs/server/users.xml index 96067d01a6b..9810feb9a53 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -1,4 +1,3 @@ - diff --git a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp index e813cf4c75c..11e9282a1c0 100644 --- a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp +++ b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp @@ -10,7 +10,6 @@ using namespace DB; TEST(Common, getMultipleValuesFromConfig) { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - xml_isteam(R"END( 0 diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index b9ee9025c03..4eedadff92c 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -103,7 +103,6 @@ TEST(Common, SensitiveDataMasker) { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - xml_isteam(R"END( @@ -154,7 +153,6 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - xml_isteam_bad(R"END( @@ -184,7 +182,6 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - xml_isteam_bad(R"END( test @@ -207,7 +204,6 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - xml_isteam_bad(R"END( test())( diff --git a/tests/config/config.d/query_masking_rules.xml b/tests/config/config.d/query_masking_rules.xml index 690d62b9a2b..b26c0b01888 100644 --- a/tests/config/config.d/query_masking_rules.xml +++ b/tests/config/config.d/query_masking_rules.xml @@ -1,4 +1,3 @@ - diff --git a/tests/config/users.d/readonly.xml b/tests/config/users.d/readonly.xml index 8e463205348..0fe1e3fe6d9 100644 --- a/tests/config/users.d/readonly.xml +++ b/tests/config/users.d/readonly.xml @@ -1,4 +1,3 @@ - diff --git a/tests/config/users.d/session_log_test.xml b/tests/config/users.d/session_log_test.xml index 1cd8aadfd09..daddaa6e4b9 100644 --- a/tests/config/users.d/session_log_test.xml +++ b/tests/config/users.d/session_log_test.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/helpers/0_common_enable_dictionaries.xml b/tests/integration/helpers/0_common_enable_dictionaries.xml index c7a763eb942..fee13e3b564 100644 --- a/tests/integration/helpers/0_common_enable_dictionaries.xml +++ b/tests/integration/helpers/0_common_enable_dictionaries.xml @@ -1,4 +1,3 @@ - /etc/clickhouse-server/dictionaries/*.xml diff --git a/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml b/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml index 6c55d61481a..3201ec36f8c 100644 --- a/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml +++ b/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml index f38096f37fc..f436c4e3380 100644 --- a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml +++ b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_async_drain_connection/configs/config.xml b/tests/integration/test_async_drain_connection/configs/config.xml index da3c4f171c4..88862a3001e 100644 --- a/tests/integration/test_async_drain_connection/configs/config.xml +++ b/tests/integration/test_async_drain_connection/configs/config.xml @@ -1,4 +1,3 @@ - 10000 diff --git a/tests/integration/test_backup_restore_new/configs/backups_disk.xml b/tests/integration/test_backup_restore_new/configs/backups_disk.xml index a589610f600..f7d666c6542 100644 --- a/tests/integration/test_backup_restore_new/configs/backups_disk.xml +++ b/tests/integration/test_backup_restore_new/configs/backups_disk.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_backup_restore_on_cluster/configs/backups_disk.xml b/tests/integration/test_backup_restore_on_cluster/configs/backups_disk.xml index beb8d605f39..b99a51cd56d 100644 --- a/tests/integration/test_backup_restore_on_cluster/configs/backups_disk.xml +++ b/tests/integration/test_backup_restore_on_cluster/configs/backups_disk.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml b/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml index 9caf52fcca4..5878cd5c41f 100644 --- a/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml +++ b/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml @@ -1,4 +1,3 @@ - 1000 diff --git a/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml index ebb914f54b0..07363e6e43a 100644 --- a/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml +++ b/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/configs/conf.d/clusters_trivial.xml b/tests/integration/test_cluster_copier/configs/conf.d/clusters_trivial.xml index 64ee762fc2c..c91ec627580 100644 --- a/tests/integration/test_cluster_copier/configs/conf.d/clusters_trivial.xml +++ b/tests/integration/test_cluster_copier/configs/conf.d/clusters_trivial.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/configs/users.xml b/tests/integration/test_cluster_copier/configs/users.xml index 2542642f6df..b463dfc81e7 100644 --- a/tests/integration/test_cluster_copier/configs/users.xml +++ b/tests/integration/test_cluster_copier/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/clusters.xml index 8105afb155b..9de7b57de27 100644 --- a/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/clusters.xml +++ b/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/clusters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/ddl.xml b/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/ddl.xml index 4f5a6a6d1f5..64fa32335ab 100644 --- a/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/ddl.xml +++ b/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/ddl.xml @@ -1,4 +1,3 @@ - /clickhouse/task_queue/ddl diff --git a/tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml b/tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml index fd79d8850d8..d0cab0fafb7 100644 --- a/tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml +++ b/tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml @@ -1,4 +1,3 @@ - information diff --git a/tests/integration/test_cluster_copier/configs_three_nodes/users.xml b/tests/integration/test_cluster_copier/configs_three_nodes/users.xml index f017daff974..badaf46a5ca 100644 --- a/tests/integration/test_cluster_copier/configs_three_nodes/users.xml +++ b/tests/integration/test_cluster_copier/configs_three_nodes/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml index b46faf9b0b7..38d88308631 100644 --- a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml +++ b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml index 4f5a6a6d1f5..64fa32335ab 100644 --- a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml +++ b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml @@ -1,4 +1,3 @@ - /clickhouse/task_queue/ddl diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml b/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml index d65ab52807d..55bd24816ae 100644 --- a/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml +++ b/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml @@ -1,4 +1,3 @@ - information diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/users.xml b/tests/integration/test_cluster_copier/configs_two_nodes/users.xml index f017daff974..badaf46a5ca 100644 --- a/tests/integration/test_cluster_copier/configs_two_nodes/users.xml +++ b/tests/integration/test_cluster_copier/configs_two_nodes/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/task0_description.xml b/tests/integration/test_cluster_copier/task0_description.xml index 9b903444741..8d74d0bdde0 100644 --- a/tests/integration/test_cluster_copier/task0_description.xml +++ b/tests/integration/test_cluster_copier/task0_description.xml @@ -1,4 +1,3 @@ - 3 diff --git a/tests/integration/test_cluster_copier/task_drop_target_partition.xml b/tests/integration/test_cluster_copier/task_drop_target_partition.xml index df39009f7f1..dc8e6452243 100644 --- a/tests/integration/test_cluster_copier/task_drop_target_partition.xml +++ b/tests/integration/test_cluster_copier/task_drop_target_partition.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/tests/integration/test_cluster_copier/task_month_to_week_description.xml index 6839ba930aa..bc290ca397f 100644 --- a/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -1,4 +1,3 @@ - 4 diff --git a/tests/integration/test_cluster_copier/task_self_copy.xml b/tests/integration/test_cluster_copier/task_self_copy.xml index 4d56369233a..fb8eaa775cd 100644 --- a/tests/integration/test_cluster_copier/task_self_copy.xml +++ b/tests/integration/test_cluster_copier/task_self_copy.xml @@ -1,4 +1,3 @@ - 9440 diff --git a/tests/integration/test_cluster_copier/task_skip_index.xml b/tests/integration/test_cluster_copier/task_skip_index.xml index 13e80086ea8..b04cec963d4 100644 --- a/tests/integration/test_cluster_copier/task_skip_index.xml +++ b/tests/integration/test_cluster_copier/task_skip_index.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/task_taxi_data.xml b/tests/integration/test_cluster_copier/task_taxi_data.xml index 554621cea21..94fa5087338 100644 --- a/tests/integration/test_cluster_copier/task_taxi_data.xml +++ b/tests/integration/test_cluster_copier/task_taxi_data.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/task_test_block_size.xml b/tests/integration/test_cluster_copier/task_test_block_size.xml index 1c7fc5d0363..bf29c7e1832 100644 --- a/tests/integration/test_cluster_copier/task_test_block_size.xml +++ b/tests/integration/test_cluster_copier/task_test_block_size.xml @@ -1,4 +1,3 @@ - 1 diff --git a/tests/integration/test_cluster_copier/task_trivial.xml b/tests/integration/test_cluster_copier/task_trivial.xml index c1531e904c5..a3b8bc03888 100644 --- a/tests/integration/test_cluster_copier/task_trivial.xml +++ b/tests/integration/test_cluster_copier/task_trivial.xml @@ -1,4 +1,3 @@ - 3 diff --git a/tests/integration/test_cluster_copier/task_trivial_without_arguments.xml b/tests/integration/test_cluster_copier/task_trivial_without_arguments.xml index 27a2b084148..0197dee0181 100644 --- a/tests/integration/test_cluster_copier/task_trivial_without_arguments.xml +++ b/tests/integration/test_cluster_copier/task_trivial_without_arguments.xml @@ -1,4 +1,3 @@ - 3 diff --git a/tests/integration/test_cluster_copier/task_ttl_columns.xml b/tests/integration/test_cluster_copier/task_ttl_columns.xml index d1fbf5f8be6..2069c509c87 100644 --- a/tests/integration/test_cluster_copier/task_ttl_columns.xml +++ b/tests/integration/test_cluster_copier/task_ttl_columns.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml b/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml index 63d2865f7f9..2a51fa7a66d 100644 --- a/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml +++ b/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_cluster_copier/task_with_different_schema.xml b/tests/integration/test_cluster_copier/task_with_different_schema.xml index eb5bf7d5945..e1e6ee4dc42 100644 --- a/tests/integration/test_cluster_copier/task_with_different_schema.xml +++ b/tests/integration/test_cluster_copier/task_with_different_schema.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_codec_encrypted/test.py b/tests/integration/test_codec_encrypted/test.py index ebe837c9e3a..64b8bf29640 100644 --- a/tests/integration/test_codec_encrypted/test.py +++ b/tests/integration/test_codec_encrypted/test.py @@ -24,7 +24,6 @@ def make_storage_with_key(id): "bash", "-c", """cat > /etc/clickhouse-server/config.d/storage_keys_config.xml << EOF - diff --git a/tests/integration/test_concurrent_threads_soft_limit/configs/config_default.xml b/tests/integration/test_concurrent_threads_soft_limit/configs/config_default.xml index 6c1f8f33de1..a63e91f41fb 100644 --- a/tests/integration/test_concurrent_threads_soft_limit/configs/config_default.xml +++ b/tests/integration/test_concurrent_threads_soft_limit/configs/config_default.xml @@ -1,4 +1,3 @@ - system diff --git a/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_1.xml b/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_1.xml index 5f93f2dc319..43d9d9ecf2d 100644 --- a/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_1.xml +++ b/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_1.xml @@ -1,4 +1,3 @@ - 1 diff --git a/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_50.xml b/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_50.xml index f3b8c6a03b4..33ed0030acf 100644 --- a/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_50.xml +++ b/tests/integration/test_concurrent_threads_soft_limit/configs/config_defined_50.xml @@ -1,4 +1,3 @@ - 50 diff --git a/tests/integration/test_concurrent_threads_soft_limit/configs/config_limit_reached.xml b/tests/integration/test_concurrent_threads_soft_limit/configs/config_limit_reached.xml index 77303cb0c35..d32417ca6e3 100644 --- a/tests/integration/test_concurrent_threads_soft_limit/configs/config_limit_reached.xml +++ b/tests/integration/test_concurrent_threads_soft_limit/configs/config_limit_reached.xml @@ -1,4 +1,3 @@ - 10 diff --git a/tests/integration/test_concurrent_threads_soft_limit/configs/users.xml b/tests/integration/test_concurrent_threads_soft_limit/configs/users.xml index 63fefbb803b..704c9220554 100644 --- a/tests/integration/test_concurrent_threads_soft_limit/configs/users.xml +++ b/tests/integration/test_concurrent_threads_soft_limit/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_corresponding_root/configs/config.xml b/tests/integration/test_config_corresponding_root/configs/config.xml index 914d4e6cb0e..72014646161 100644 --- a/tests/integration/test_config_corresponding_root/configs/config.xml +++ b/tests/integration/test_config_corresponding_root/configs/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_corresponding_root/configs/users.xml b/tests/integration/test_config_corresponding_root/configs/users.xml index bffce084828..9d8e6e02acc 100644 --- a/tests/integration/test_config_corresponding_root/configs/users.xml +++ b/tests/integration/test_config_corresponding_root/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_full/configs/config.d/query_masking_rules.xml b/tests/integration/test_config_xml_full/configs/config.d/query_masking_rules.xml index 690d62b9a2b..b26c0b01888 100644 --- a/tests/integration/test_config_xml_full/configs/config.d/query_masking_rules.xml +++ b/tests/integration/test_config_xml_full/configs/config.d/query_masking_rules.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_full/configs/config.xml b/tests/integration/test_config_xml_full/configs/config.xml index 55e84568af1..4e3d1def5fc 100644 --- a/tests/integration/test_config_xml_full/configs/config.xml +++ b/tests/integration/test_config_xml_full/configs/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_full/configs/users.d/allow_introspection_functions.xml b/tests/integration/test_config_xml_full/configs/users.d/allow_introspection_functions.xml index cfde1b4525d..ec3057c82d7 100644 --- a/tests/integration/test_config_xml_full/configs/users.d/allow_introspection_functions.xml +++ b/tests/integration/test_config_xml_full/configs/users.d/allow_introspection_functions.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_full/configs/users.xml b/tests/integration/test_config_xml_full/configs/users.xml index 378e4e4c570..ee07ea863c5 100644 --- a/tests/integration/test_config_xml_full/configs/users.xml +++ b/tests/integration/test_config_xml_full/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_main/configs/config.xml b/tests/integration/test_config_xml_main/configs/config.xml index 515575a1798..cd178668bda 100644 --- a/tests/integration/test_config_xml_main/configs/config.xml +++ b/tests/integration/test_config_xml_main/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_config_xml_main/configs/embedded.xml b/tests/integration/test_config_xml_main/configs/embedded.xml index 2b6c4d9f770..c9d269a5355 100644 --- a/tests/integration/test_config_xml_main/configs/embedded.xml +++ b/tests/integration/test_config_xml_main/configs/embedded.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_main/configs/users.xml b/tests/integration/test_config_xml_main/configs/users.xml index ac785c12577..6f94d1696e3 100644 --- a/tests/integration/test_config_xml_main/configs/users.xml +++ b/tests/integration/test_config_xml_main/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_masking_rules.xml b/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_masking_rules.xml index 690d62b9a2b..b26c0b01888 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_masking_rules.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_masking_rules.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.xml b/tests/integration/test_config_xml_yaml_mix/configs/config.xml index 870379618f2..dc1cb85e155 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/config.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_config_xml_yaml_mix/configs/embedded.xml b/tests/integration/test_config_xml_yaml_mix/configs/embedded.xml index 2b6c4d9f770..c9d269a5355 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/embedded.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/embedded.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_xml_yaml_mix/configs/users.d/allow_introspection_functions.xml b/tests/integration/test_config_xml_yaml_mix/configs/users.d/allow_introspection_functions.xml index cfde1b4525d..ec3057c82d7 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/users.d/allow_introspection_functions.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/users.d/allow_introspection_functions.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_yaml_full/configs/embedded.xml b/tests/integration/test_config_yaml_full/configs/embedded.xml index 2b6c4d9f770..c9d269a5355 100644 --- a/tests/integration/test_config_yaml_full/configs/embedded.xml +++ b/tests/integration/test_config_yaml_full/configs/embedded.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_yaml_main/configs/config.d/query_masking_rules.xml b/tests/integration/test_config_yaml_main/configs/config.d/query_masking_rules.xml index 690d62b9a2b..b26c0b01888 100644 --- a/tests/integration/test_config_yaml_main/configs/config.d/query_masking_rules.xml +++ b/tests/integration/test_config_yaml_main/configs/config.d/query_masking_rules.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_yaml_main/configs/embedded.xml b/tests/integration/test_config_yaml_main/configs/embedded.xml index 2b6c4d9f770..c9d269a5355 100644 --- a/tests/integration/test_config_yaml_main/configs/embedded.xml +++ b/tests/integration/test_config_yaml_main/configs/embedded.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_config_yaml_main/configs/users.d/allow_introspection_functions.xml b/tests/integration/test_config_yaml_main/configs/users.d/allow_introspection_functions.xml index cfde1b4525d..ec3057c82d7 100644 --- a/tests/integration/test_config_yaml_main/configs/users.d/allow_introspection_functions.xml +++ b/tests/integration/test_config_yaml_main/configs/users.d/allow_introspection_functions.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_create_user_and_login/test.py b/tests/integration/test_create_user_and_login/test.py index 25346c22d3b..1b59089fa11 100644 --- a/tests/integration/test_create_user_and_login/test.py +++ b/tests/integration/test_create_user_and_login/test.py @@ -61,7 +61,6 @@ def test_login_as_dropped_user_xml(): "-c", """ cat > /etc/clickhouse-server/users.d/user_c.xml << EOF - diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/config.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/config.xml index 4bd62c34536..13cf3d0d1bd 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/config.xml +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/disable_ssl_verification.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/disable_ssl_verification.xml index 8bfe1fff690..d2a014f5bc1 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/disable_ssl_verification.xml +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/disable_ssl_verification.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/users.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/users.xml +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/config.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/config.xml index 835814671e7..cb6f0e228c5 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/config.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/users.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/users.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_ddl/configs/user_admin.xml b/tests/integration/test_dictionaries_ddl/configs/user_admin.xml index abbca144c35..b5e94aed63a 100644 --- a/tests/integration/test_dictionaries_ddl/configs/user_admin.xml +++ b/tests/integration/test_dictionaries_ddl/configs/user_admin.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_ddl/configs/user_default.xml b/tests/integration/test_dictionaries_ddl/configs/user_default.xml index 9b01205192b..bf004102329 100644 --- a/tests/integration/test_dictionaries_ddl/configs/user_default.xml +++ b/tests/integration/test_dictionaries_ddl/configs/user_default.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_dependency_xml/configs/config.xml b/tests/integration/test_dictionaries_dependency_xml/configs/config.xml index 7ed6d759dac..099515eeccc 100644 --- a/tests/integration/test_dictionaries_dependency_xml/configs/config.xml +++ b/tests/integration/test_dictionaries_dependency_xml/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_dependency_xml/configs/users.xml b/tests/integration/test_dictionaries_dependency_xml/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_dependency_xml/configs/users.xml +++ b/tests/integration/test_dictionaries_dependency_xml/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_mysql/configs/config.xml b/tests/integration/test_dictionaries_mysql/configs/config.xml index 7ed6d759dac..099515eeccc 100644 --- a/tests/integration/test_dictionaries_mysql/configs/config.xml +++ b/tests/integration/test_dictionaries_mysql/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml index 9e149ca8575..d4d2466b7bf 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml @@ -1,4 +1,3 @@ - dict0 diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml index 7587936ae99..031afbba636 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml @@ -1,4 +1,3 @@ - dict2 diff --git a/tests/integration/test_dictionaries_mysql/configs/users.xml b/tests/integration/test_dictionaries_mysql/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_mysql/configs/users.xml +++ b/tests/integration/test_dictionaries_mysql/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_null_value/configs/config.xml b/tests/integration/test_dictionaries_null_value/configs/config.xml index 95c52a19611..3d5fca55918 100644 --- a/tests/integration/test_dictionaries_null_value/configs/config.xml +++ b/tests/integration/test_dictionaries_null_value/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_null_value/configs/users.xml b/tests/integration/test_dictionaries_null_value/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_null_value/configs/users.xml +++ b/tests/integration/test_dictionaries_null_value/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_postgresql/configs/config.xml b/tests/integration/test_dictionaries_postgresql/configs/config.xml index 48c9dff364a..a897b5bb60e 100644 --- a/tests/integration/test_dictionaries_postgresql/configs/config.xml +++ b/tests/integration/test_dictionaries_postgresql/configs/config.xml @@ -1,4 +1,3 @@ - /etc/clickhouse-server/config.d/postgres_dict.xml diff --git a/tests/integration/test_dictionaries_postgresql/configs/dictionaries/postgres_dict.xml b/tests/integration/test_dictionaries_postgresql/configs/dictionaries/postgres_dict.xml index dc4e474f125..66a5386793f 100644 --- a/tests/integration/test_dictionaries_postgresql/configs/dictionaries/postgres_dict.xml +++ b/tests/integration/test_dictionaries_postgresql/configs/dictionaries/postgres_dict.xml @@ -1,4 +1,3 @@ - dict0 diff --git a/tests/integration/test_dictionaries_select_all/configs/config.xml b/tests/integration/test_dictionaries_select_all/configs/config.xml index 95c52a19611..3d5fca55918 100644 --- a/tests/integration/test_dictionaries_select_all/configs/config.xml +++ b/tests/integration/test_dictionaries_select_all/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_select_all/configs/users.xml b/tests/integration/test_dictionaries_select_all/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_select_all/configs/users.xml +++ b/tests/integration/test_dictionaries_select_all/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/config.xml b/tests/integration/test_dictionaries_update_and_reload/configs/config.xml index d7d0c7a0c89..fbd44095364 100644 --- a/tests/integration/test_dictionaries_update_and_reload/configs/config.xml +++ b/tests/integration/test_dictionaries_update_and_reload/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml b/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml index c34107e30d9..be056cd4f14 100644 --- a/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml +++ b/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml @@ -1,4 +1,3 @@ - executable diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml b/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml index 1679801aed0..74d9135e521 100644 --- a/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml +++ b/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml @@ -1,4 +1,3 @@ - file diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml b/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml index 16b00ecd76a..7c30ba76da0 100644 --- a/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml +++ b/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml @@ -1,4 +1,3 @@ - slow diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/users.xml b/tests/integration/test_dictionaries_update_and_reload/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_update_and_reload/configs/users.xml +++ b/tests/integration/test_dictionaries_update_and_reload/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionaries_update_field/configs/config.xml b/tests/integration/test_dictionaries_update_field/configs/config.xml index 835814671e7..cb6f0e228c5 100644 --- a/tests/integration/test_dictionaries_update_field/configs/config.xml +++ b/tests/integration/test_dictionaries_update_field/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionaries_update_field/configs/users.xml b/tests/integration/test_dictionaries_update_field/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionaries_update_field/configs/users.xml +++ b/tests/integration/test_dictionaries_update_field/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/configs/config.xml b/tests/integration/test_dictionary_allow_read_expired_keys/configs/config.xml index 835814671e7..cb6f0e228c5 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/configs/config.xml +++ b/tests/integration/test_dictionary_allow_read_expired_keys/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/configs/users.xml b/tests/integration/test_dictionary_allow_read_expired_keys/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/configs/users.xml +++ b/tests/integration/test_dictionary_allow_read_expired_keys/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dictionary_custom_settings/configs/config.xml b/tests/integration/test_dictionary_custom_settings/configs/config.xml index 95c52a19611..3d5fca55918 100644 --- a/tests/integration/test_dictionary_custom_settings/configs/config.xml +++ b/tests/integration/test_dictionary_custom_settings/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_dictionary_custom_settings/configs/users.xml b/tests/integration/test_dictionary_custom_settings/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_dictionary_custom_settings/configs/users.xml +++ b/tests/integration/test_dictionary_custom_settings/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disabled_access_control_improvements/all_rows.xml b/tests/integration/test_disabled_access_control_improvements/all_rows.xml index 948aed1a874..bbeccc1c51a 100644 --- a/tests/integration/test_disabled_access_control_improvements/all_rows.xml +++ b/tests/integration/test_disabled_access_control_improvements/all_rows.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml b/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml index 476072bd138..d1d47ba2ec8 100644 --- a/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml +++ b/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disabled_access_control_improvements/configs/users.xml b/tests/integration/test_disabled_access_control_improvements/configs/users.xml index 5dc65384a2d..a4b5c2f8daf 100644 --- a/tests/integration/test_disabled_access_control_improvements/configs/users.xml +++ b/tests/integration/test_disabled_access_control_improvements/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disabled_access_control_improvements/no_filters.xml b/tests/integration/test_disabled_access_control_improvements/no_filters.xml index fddc67fc3f9..3cbf717bb67 100644 --- a/tests/integration/test_disabled_access_control_improvements/no_filters.xml +++ b/tests/integration/test_disabled_access_control_improvements/no_filters.xml @@ -1,3 +1,2 @@ - diff --git a/tests/integration/test_disabled_access_control_improvements/no_rows.xml b/tests/integration/test_disabled_access_control_improvements/no_rows.xml index 760203210b2..b7e179bc5c0 100644 --- a/tests/integration/test_disabled_access_control_improvements/no_rows.xml +++ b/tests/integration/test_disabled_access_control_improvements/no_rows.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disabled_access_control_improvements/normal_filters.xml b/tests/integration/test_disabled_access_control_improvements/normal_filters.xml index 6f5cd8701f6..ab4e08eb308 100644 --- a/tests/integration/test_disabled_access_control_improvements/normal_filters.xml +++ b/tests/integration/test_disabled_access_control_improvements/normal_filters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disk_over_web_server/configs/storage_conf.xml b/tests/integration/test_disk_over_web_server/configs/storage_conf.xml index 7e5c22c6e4d..ca81daf015e 100644 --- a/tests/integration/test_disk_over_web_server/configs/storage_conf.xml +++ b/tests/integration/test_disk_over_web_server/configs/storage_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml b/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml index e69794638f1..56ee3e532c4 100644 --- a/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml +++ b/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disk_types/configs/config.xml b/tests/integration/test_disk_types/configs/config.xml index 699e8834701..7047dca698f 100644 --- a/tests/integration/test_disk_types/configs/config.xml +++ b/tests/integration/test_disk_types/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_disk_types/configs/storage.xml b/tests/integration/test_disk_types/configs/storage.xml index 78e7fb01553..c55d589d19e 100644 --- a/tests/integration/test_disk_types/configs/storage.xml +++ b/tests/integration/test_disk_types/configs/storage.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disk_types/configs/users.xml b/tests/integration/test_disk_types/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_disk_types/configs/users.xml +++ b/tests/integration/test_disk_types/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_disks_app_func/config.xml b/tests/integration/test_disks_app_func/config.xml index b3c96e3c0ca..779232b3341 100644 --- a/tests/integration/test_disks_app_func/config.xml +++ b/tests/integration/test_disks_app_func/config.xml @@ -1,4 +1,3 @@ - /var/lib/clickhouse/ diff --git a/tests/integration/test_distributed_ddl_parallel/configs/dict.xml b/tests/integration/test_distributed_ddl_parallel/configs/dict.xml index 580d95c6e5e..6335f171480 100644 --- a/tests/integration/test_distributed_ddl_parallel/configs/dict.xml +++ b/tests/integration/test_distributed_ddl_parallel/configs/dict.xml @@ -1,4 +1,3 @@ - slow_dict_7 diff --git a/tests/integration/test_distributed_default_database/configs/users.xml b/tests/integration/test_distributed_default_database/configs/users.xml index a49b1363433..e7fe138fbeb 100644 --- a/tests/integration/test_distributed_default_database/configs/users.xml +++ b/tests/integration/test_distributed_default_database/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_distributed_inter_server_secret/configs/users.xml b/tests/integration/test_distributed_inter_server_secret/configs/users.xml index d73fbc2183b..31ad3c3a4c5 100644 --- a/tests/integration/test_distributed_inter_server_secret/configs/users.xml +++ b/tests/integration/test_distributed_inter_server_secret/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_distributed_load_balancing/configs/users.xml b/tests/integration/test_distributed_load_balancing/configs/users.xml index f4c52d0f211..dfa6bf9e652 100644 --- a/tests/integration/test_distributed_load_balancing/configs/users.xml +++ b/tests/integration/test_distributed_load_balancing/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dotnet_client/configs/config.xml b/tests/integration/test_dotnet_client/configs/config.xml index fe64e47d384..7f77f14f376 100644 --- a/tests/integration/test_dotnet_client/configs/config.xml +++ b/tests/integration/test_dotnet_client/configs/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_dotnet_client/configs/users.xml b/tests/integration/test_dotnet_client/configs/users.xml index 1874371871a..7682741488a 100644 --- a/tests/integration/test_dotnet_client/configs/users.xml +++ b/tests/integration/test_dotnet_client/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_encrypted_disk/configs/storage.xml b/tests/integration/test_encrypted_disk/configs/storage.xml index 212c1616d6e..41a29455fe5 100644 --- a/tests/integration/test_encrypted_disk/configs/storage.xml +++ b/tests/integration/test_encrypted_disk/configs/storage.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 17a30676f7f..42ce7aab906 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -166,7 +166,6 @@ def test_add_key(): "bash", "-c", """cat > /etc/clickhouse-server/config.d/storage_policy_{policy_name}.xml << EOF - diff --git a/tests/integration/test_file_schema_inference_cache/configs/config.d/query_log.xml b/tests/integration/test_file_schema_inference_cache/configs/config.d/query_log.xml index de7e27921f0..69060e5abf9 100644 --- a/tests/integration/test_file_schema_inference_cache/configs/config.d/query_log.xml +++ b/tests/integration/test_file_schema_inference_cache/configs/config.d/query_log.xml @@ -1,4 +1,3 @@ - system diff --git a/tests/integration/test_file_schema_inference_cache/configs/config.d/schema_cache.xml b/tests/integration/test_file_schema_inference_cache/configs/config.d/schema_cache.xml index de3c4cbe57f..7587fb433eb 100644 --- a/tests/integration/test_file_schema_inference_cache/configs/config.d/schema_cache.xml +++ b/tests/integration/test_file_schema_inference_cache/configs/config.d/schema_cache.xml @@ -1,4 +1,3 @@ - 2 diff --git a/tests/integration/test_graphite_merge_tree/configs/users.xml b/tests/integration/test_graphite_merge_tree/configs/users.xml index 66d0cd7e445..f2e2fd93c0b 100644 --- a/tests/integration/test_graphite_merge_tree/configs/users.xml +++ b/tests/integration/test_graphite_merge_tree/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_graphite_merge_tree_typed/configs/users.xml b/tests/integration/test_graphite_merge_tree_typed/configs/users.xml index 66d0cd7e445..f2e2fd93c0b 100644 --- a/tests/integration/test_graphite_merge_tree_typed/configs/users.xml +++ b/tests/integration/test_graphite_merge_tree_typed/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_hedged_requests/configs/users.xml b/tests/integration/test_hedged_requests/configs/users.xml index fa3fc4d0fd5..184b7857245 100644 --- a/tests/integration/test_hedged_requests/configs/users.xml +++ b/tests/integration/test_hedged_requests/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_hedged_requests/configs/users1.xml b/tests/integration/test_hedged_requests/configs/users1.xml index 3547177eb80..db0dd71de56 100644 --- a/tests/integration/test_hedged_requests/configs/users1.xml +++ b/tests/integration/test_hedged_requests/configs/users1.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_hedged_requests_parallel/configs/users.xml b/tests/integration/test_hedged_requests_parallel/configs/users.xml index d2351c184b2..6fb96f1689c 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_hedged_requests_parallel/configs/users1.xml b/tests/integration/test_hedged_requests_parallel/configs/users1.xml index 3547177eb80..db0dd71de56 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users1.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users1.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_http_handlers_config/test_defaults_handlers/config.xml b/tests/integration/test_http_handlers_config/test_defaults_handlers/config.xml index ae2e5562ff7..a7d35d07a32 100644 --- a/tests/integration/test_http_handlers_config/test_defaults_handlers/config.xml +++ b/tests/integration/test_http_handlers_config/test_defaults_handlers/config.xml @@ -1,4 +1,3 @@ - Default server response diff --git a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml index 68e5ba302db..c9b61c21507 100644 --- a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml index 96fc018ae3e..5b5de63356e 100644 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_http_handlers_config/test_prometheus_handler/config.xml b/tests/integration/test_http_handlers_config/test_prometheus_handler/config.xml index 3402d9ebf96..a7c92427c5c 100644 --- a/tests/integration/test_http_handlers_config/test_prometheus_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_prometheus_handler/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_http_handlers_config/test_replicas_status_handler/config.xml b/tests/integration/test_http_handlers_config/test_replicas_status_handler/config.xml index fdfeb981ee3..d762bfe289e 100644 --- a/tests/integration/test_http_handlers_config/test_replicas_status_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_replicas_status_handler/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_http_handlers_config/test_static_handler/config.xml b/tests/integration/test_http_handlers_config/test_static_handler/config.xml index 48cbdb6c682..ff24e6dec96 100644 --- a/tests/integration/test_http_handlers_config/test_static_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_static_handler/config.xml @@ -1,4 +1,3 @@ - Test get static handler and config content diff --git a/tests/integration/test_https_replication/configs/config.xml b/tests/integration/test_https_replication/configs/config.xml index c1fc5d29c22..675efee8ea6 100644 --- a/tests/integration/test_https_replication/configs/config.xml +++ b/tests/integration/test_https_replication/configs/config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_jdbc_bridge/configs/jdbc_bridge.xml b/tests/integration/test_jdbc_bridge/configs/jdbc_bridge.xml index 2cc29c0c76d..ff3e28cdb5d 100644 --- a/tests/integration/test_jdbc_bridge/configs/jdbc_bridge.xml +++ b/tests/integration/test_jdbc_bridge/configs/jdbc_bridge.xml @@ -1,4 +1,3 @@ - bridge1 diff --git a/tests/integration/test_join_set_family_s3/configs/config.xml b/tests/integration/test_join_set_family_s3/configs/config.xml index 87270096ccd..d4a2a9cf367 100644 --- a/tests/integration/test_join_set_family_s3/configs/config.xml +++ b/tests/integration/test_join_set_family_s3/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_join_set_family_s3/configs/minio.xml b/tests/integration/test_join_set_family_s3/configs/minio.xml index 8a3222c37a3..d5952a6848f 100644 --- a/tests/integration/test_join_set_family_s3/configs/minio.xml +++ b/tests/integration/test_join_set_family_s3/configs/minio.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_join_set_family_s3/configs/ssl.xml b/tests/integration/test_join_set_family_s3/configs/ssl.xml index 1200dc8c94b..d4cea955c68 100644 --- a/tests/integration/test_join_set_family_s3/configs/ssl.xml +++ b/tests/integration/test_join_set_family_s3/configs/ssl.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_join_set_family_s3/configs/users.xml b/tests/integration/test_join_set_family_s3/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_join_set_family_s3/configs/users.xml +++ b/tests/integration/test_join_set_family_s3/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_library_bridge/configs/dictionaries/dict1.xml b/tests/integration/test_library_bridge/configs/dictionaries/dict1.xml index df0fe2ddf05..b1aca0f0aac 100644 --- a/tests/integration/test_library_bridge/configs/dictionaries/dict1.xml +++ b/tests/integration/test_library_bridge/configs/dictionaries/dict1.xml @@ -1,4 +1,3 @@ - dict1 diff --git a/tests/integration/test_log_family_hdfs/configs/storage_conf.xml b/tests/integration/test_log_family_hdfs/configs/storage_conf.xml index ffa5d516ae0..82cea6730ff 100644 --- a/tests/integration/test_log_family_hdfs/configs/storage_conf.xml +++ b/tests/integration/test_log_family_hdfs/configs/storage_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_log_family_s3/configs/config.xml b/tests/integration/test_log_family_s3/configs/config.xml index 87270096ccd..d4a2a9cf367 100644 --- a/tests/integration/test_log_family_s3/configs/config.xml +++ b/tests/integration/test_log_family_s3/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_log_family_s3/configs/minio.xml b/tests/integration/test_log_family_s3/configs/minio.xml index 8a3222c37a3..d5952a6848f 100644 --- a/tests/integration/test_log_family_s3/configs/minio.xml +++ b/tests/integration/test_log_family_s3/configs/minio.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_log_family_s3/configs/ssl.xml b/tests/integration/test_log_family_s3/configs/ssl.xml index 1200dc8c94b..d4cea955c68 100644 --- a/tests/integration/test_log_family_s3/configs/ssl.xml +++ b/tests/integration/test_log_family_s3/configs/ssl.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_log_family_s3/configs/users.xml b/tests/integration/test_log_family_s3/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_log_family_s3/configs/users.xml +++ b/tests/integration/test_log_family_s3/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_logs_level/configs/config_information.xml b/tests/integration/test_logs_level/configs/config_information.xml index 17342b36c57..518657e82ca 100644 --- a/tests/integration/test_logs_level/configs/config_information.xml +++ b/tests/integration/test_logs_level/configs/config_information.xml @@ -1,4 +1,3 @@ - information diff --git a/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml b/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml index 58087e93882..70a8a066661 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml +++ b/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml @@ -1,4 +1,3 @@ - /var/log/clickhouse-server/log.log diff --git a/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml b/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml index 3547177eb80..db0dd71de56 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml +++ b/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_materialized_mysql_database/configs/users.xml b/tests/integration/test_materialized_mysql_database/configs/users.xml index 0e116f115fe..3669fbb46ba 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml b/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml index a00b6ca6b9a..f0aa2ecfa49 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml b/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml index 3a7cc2537e5..857a0035b3c 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml index 8e7eada42ed..feb537ebbce 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml +++ b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_merge_tree_s3/configs/config.xml b/tests/integration/test_merge_tree_s3/configs/config.xml index 8e7eada42ed..feb537ebbce 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/part_log.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/part_log.xml index 7140b923e53..17622e0faff 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/part_log.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/part_log.xml @@ -1,4 +1,3 @@ - system diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.xml index 8e7eada42ed..feb537ebbce 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_merge_tree_s3_restore/configs/config.d/clusters.xml b/tests/integration/test_merge_tree_s3_restore/configs/config.d/clusters.xml index dd383c14116..d02d5ab0bf5 100644 --- a/tests/integration/test_merge_tree_s3_restore/configs/config.d/clusters.xml +++ b/tests/integration/test_merge_tree_s3_restore/configs/config.d/clusters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_merge_tree_s3_restore/configs/config.xml b/tests/integration/test_merge_tree_s3_restore/configs/config.xml index 8e7eada42ed..feb537ebbce 100644 --- a/tests/integration/test_merge_tree_s3_restore/configs/config.xml +++ b/tests/integration/test_merge_tree_s3_restore/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml index 33da9cd1586..24d66fc674e 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml @@ -1,4 +1,3 @@ - system diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml index 1200dc8c94b..d4cea955c68 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.xml index ccba76b71a6..efc501ad901 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/configs/config.xml +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_mutations_with_merge_tree/configs/config.xml b/tests/integration/test_mutations_with_merge_tree/configs/config.xml index 9183a737238..721fc429204 100644 --- a/tests/integration/test_mutations_with_merge_tree/configs/config.xml +++ b/tests/integration/test_mutations_with_merge_tree/configs/config.xml @@ -1,4 +1,3 @@ - /var/log/clickhouse-server/log.log diff --git a/tests/integration/test_mutations_with_merge_tree/configs/users.xml b/tests/integration/test_mutations_with_merge_tree/configs/users.xml index 7b28d23746c..d1a3ae1e859 100644 --- a/tests/integration/test_mutations_with_merge_tree/configs/users.xml +++ b/tests/integration/test_mutations_with_merge_tree/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_mutations_with_projection/configs/config.xml b/tests/integration/test_mutations_with_projection/configs/config.xml index 9183a737238..721fc429204 100644 --- a/tests/integration/test_mutations_with_projection/configs/config.xml +++ b/tests/integration/test_mutations_with_projection/configs/config.xml @@ -1,4 +1,3 @@ - /var/log/clickhouse-server/log.log diff --git a/tests/integration/test_mutations_with_projection/configs/users.xml b/tests/integration/test_mutations_with_projection/configs/users.xml index 37e6c66b5a5..135252f714c 100644 --- a/tests/integration/test_mutations_with_projection/configs/users.xml +++ b/tests/integration/test_mutations_with_projection/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_mysql_protocol/configs/config.xml b/tests/integration/test_mysql_protocol/configs/config.xml index a2ba348eabd..d865d5f11cf 100644 --- a/tests/integration/test_mysql_protocol/configs/config.xml +++ b/tests/integration/test_mysql_protocol/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_mysql_protocol/configs/mysql.xml b/tests/integration/test_mysql_protocol/configs/mysql.xml index 35731682db2..768e6e2e641 100644 --- a/tests/integration/test_mysql_protocol/configs/mysql.xml +++ b/tests/integration/test_mysql_protocol/configs/mysql.xml @@ -1,4 +1,3 @@ - 9001 diff --git a/tests/integration/test_mysql_protocol/configs/ssl_conf.xml b/tests/integration/test_mysql_protocol/configs/ssl_conf.xml index 43b25032059..67f1b64bb85 100644 --- a/tests/integration/test_mysql_protocol/configs/ssl_conf.xml +++ b/tests/integration/test_mysql_protocol/configs/ssl_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_mysql_protocol/configs/users.xml b/tests/integration/test_mysql_protocol/configs/users.xml index 9f82953225f..a5f8232450b 100644 --- a/tests/integration/test_mysql_protocol/configs/users.xml +++ b/tests/integration/test_mysql_protocol/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_nlp/configs/dicts_config.xml b/tests/integration/test_nlp/configs/dicts_config.xml index 8c05ea67e49..1f7c499edff 100644 --- a/tests/integration/test_nlp/configs/dicts_config.xml +++ b/tests/integration/test_nlp/configs/dicts_config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_odbc_interaction/configs/config.xml b/tests/integration/test_odbc_interaction/configs/config.xml index 861c81248d7..abe0a05b65c 100644 --- a/tests/integration/test_odbc_interaction/configs/config.xml +++ b/tests/integration/test_odbc_interaction/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_odbc_interaction/configs/odbc_logging.xml b/tests/integration/test_odbc_interaction/configs/odbc_logging.xml index ad013cbcdac..d14b4b8f96d 100644 --- a/tests/integration/test_odbc_interaction/configs/odbc_logging.xml +++ b/tests/integration/test_odbc_interaction/configs/odbc_logging.xml @@ -1,4 +1,3 @@ - /var/log/clickhouse-server/clickhouse-odbc-bridge.log diff --git a/tests/integration/test_odbc_interaction/configs/openssl.xml b/tests/integration/test_odbc_interaction/configs/openssl.xml index 1200dc8c94b..d4cea955c68 100644 --- a/tests/integration/test_odbc_interaction/configs/openssl.xml +++ b/tests/integration/test_odbc_interaction/configs/openssl.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_odbc_interaction/configs/users.xml b/tests/integration/test_odbc_interaction/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_odbc_interaction/configs/users.xml +++ b/tests/integration/test_odbc_interaction/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_postgresql_protocol/configs/config.xml b/tests/integration/test_postgresql_protocol/configs/config.xml index b3340627751..9a20dfd45e6 100644 --- a/tests/integration/test_postgresql_protocol/configs/config.xml +++ b/tests/integration/test_postgresql_protocol/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_postgresql_protocol/configs/default_passwd.xml b/tests/integration/test_postgresql_protocol/configs/default_passwd.xml index ad56fbfa635..123a9be0c81 100644 --- a/tests/integration/test_postgresql_protocol/configs/default_passwd.xml +++ b/tests/integration/test_postgresql_protocol/configs/default_passwd.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_postgresql_protocol/configs/log.xml b/tests/integration/test_postgresql_protocol/configs/log.xml index 21cd83a1af2..22f95a8bd5d 100644 --- a/tests/integration/test_postgresql_protocol/configs/log.xml +++ b/tests/integration/test_postgresql_protocol/configs/log.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_postgresql_protocol/configs/postresql.xml b/tests/integration/test_postgresql_protocol/configs/postresql.xml index 52f8ce9fb89..e8bc3eadf5e 100644 --- a/tests/integration/test_postgresql_protocol/configs/postresql.xml +++ b/tests/integration/test_postgresql_protocol/configs/postresql.xml @@ -1,4 +1,3 @@ - 5433 diff --git a/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml b/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml index 46a69601de9..778d327c460 100644 --- a/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml +++ b/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_postgresql_protocol/configs/users.xml b/tests/integration/test_postgresql_protocol/configs/users.xml index ad56fbfa635..123a9be0c81 100644 --- a/tests/integration/test_postgresql_protocol/configs/users.xml +++ b/tests/integration/test_postgresql_protocol/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_postgresql_replica_database_engine_1/configs/users.xml b/tests/integration/test_postgresql_replica_database_engine_1/configs/users.xml index 588ebd116c7..26ea20e012f 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/configs/users.xml +++ b/tests/integration/test_postgresql_replica_database_engine_1/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_postgresql_replica_database_engine_2/configs/users.xml b/tests/integration/test_postgresql_replica_database_engine_2/configs/users.xml index 588ebd116c7..26ea20e012f 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/configs/users.xml +++ b/tests/integration/test_postgresql_replica_database_engine_2/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_profile_events_s3/configs/config.xml b/tests/integration/test_profile_events_s3/configs/config.xml index 226e3e7efbe..f00ba1b32fb 100644 --- a/tests/integration/test_profile_events_s3/configs/config.xml +++ b/tests/integration/test_profile_events_s3/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_profile_events_s3/configs/log.xml b/tests/integration/test_profile_events_s3/configs/log.xml index c58040559d8..049fa471846 100644 --- a/tests/integration/test_profile_events_s3/configs/log.xml +++ b/tests/integration/test_profile_events_s3/configs/log.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_profile_events_s3/configs/query_log.xml b/tests/integration/test_profile_events_s3/configs/query_log.xml index 33da9cd1586..24d66fc674e 100644 --- a/tests/integration/test_profile_events_s3/configs/query_log.xml +++ b/tests/integration/test_profile_events_s3/configs/query_log.xml @@ -1,4 +1,3 @@ - system diff --git a/tests/integration/test_profile_events_s3/configs/ssl_conf.xml b/tests/integration/test_profile_events_s3/configs/ssl_conf.xml index 1200dc8c94b..d4cea955c68 100644 --- a/tests/integration/test_profile_events_s3/configs/ssl_conf.xml +++ b/tests/integration/test_profile_events_s3/configs/ssl_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_profile_events_s3/configs/users.xml b/tests/integration/test_profile_events_s3/configs/users.xml index 4b08a2dc4a4..b7c75cf619e 100644 --- a/tests/integration/test_profile_events_s3/configs/users.xml +++ b/tests/integration/test_profile_events_s3/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quorum_inserts/configs/users.d/settings.xml b/tests/integration/test_quorum_inserts/configs/users.d/settings.xml index 234e90689dc..3ed2607b45b 100644 --- a/tests/integration/test_quorum_inserts/configs/users.d/settings.xml +++ b/tests/integration/test_quorum_inserts/configs/users.d/settings.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quota/no_quotas.xml b/tests/integration/test_quota/no_quotas.xml index fddc67fc3f9..3cbf717bb67 100644 --- a/tests/integration/test_quota/no_quotas.xml +++ b/tests/integration/test_quota/no_quotas.xml @@ -1,3 +1,2 @@ - diff --git a/tests/integration/test_quota/normal_limits.xml b/tests/integration/test_quota/normal_limits.xml index b91320d57ba..6061104ec9f 100644 --- a/tests/integration/test_quota/normal_limits.xml +++ b/tests/integration/test_quota/normal_limits.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quota/simpliest.xml b/tests/integration/test_quota/simpliest.xml index 2095457eba2..c89f6ec6890 100644 --- a/tests/integration/test_quota/simpliest.xml +++ b/tests/integration/test_quota/simpliest.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quota/tiny_limits.xml b/tests/integration/test_quota/tiny_limits.xml index 2dd50eba5b7..5821935bb6d 100644 --- a/tests/integration/test_quota/tiny_limits.xml +++ b/tests/integration/test_quota/tiny_limits.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quota/tracking.xml b/tests/integration/test_quota/tracking.xml index f0fc55f5589..3111b7e6cbc 100644 --- a/tests/integration/test_quota/tracking.xml +++ b/tests/integration/test_quota/tracking.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quota/two_intervals.xml b/tests/integration/test_quota/two_intervals.xml index 8362413f4d8..a192b46ea71 100644 --- a/tests/integration/test_quota/two_intervals.xml +++ b/tests/integration/test_quota/two_intervals.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_quota/two_quotas.xml b/tests/integration/test_quota/two_quotas.xml index 1da0f5831b7..13872286dc6 100644 --- a/tests/integration/test_quota/two_quotas.xml +++ b/tests/integration/test_quota/two_quotas.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_relative_filepath/configs/config.xml b/tests/integration/test_relative_filepath/configs/config.xml index 19fb49fed3f..33b0515c35d 100644 --- a/tests/integration/test_relative_filepath/configs/config.xml +++ b/tests/integration/test_relative_filepath/configs/config.xml @@ -1,4 +1,3 @@ - user_files diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml index d1c2e5958c8..66585859e71 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml +++ b/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml +++ b/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reload_certificate/configs/cert.xml b/tests/integration/test_reload_certificate/configs/cert.xml index 91fe8156d1e..03387757258 100644 --- a/tests/integration/test_reload_certificate/configs/cert.xml +++ b/tests/integration/test_reload_certificate/configs/cert.xml @@ -1,4 +1,3 @@ - 8443 diff --git a/tests/integration/test_reload_certificate/test.py b/tests/integration/test_reload_certificate/test.py index a9012b9defd..1718e440629 100644 --- a/tests/integration/test_reload_certificate/test.py +++ b/tests/integration/test_reload_certificate/test.py @@ -46,7 +46,6 @@ def change_config_to_key(name, pass_phrase=""): "bash", "-c", """cat > /etc/clickhouse-server/config.d/cert.xml << EOF - 8443 diff --git a/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml b/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml index d1c2e5958c8..66585859e71 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml +++ b/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml b/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml index 34849e1187a..0500e2ad554 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml +++ b/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml @@ -1,4 +1,3 @@ - 1 1 diff --git a/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml b/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml +++ b/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reload_zookeeper/configs/config.xml b/tests/integration/test_reload_zookeeper/configs/config.xml index d1c2e5958c8..66585859e71 100644 --- a/tests/integration/test_reload_zookeeper/configs/config.xml +++ b/tests/integration/test_reload_zookeeper/configs/config.xml @@ -1,4 +1,3 @@ - trace diff --git a/tests/integration/test_reload_zookeeper/configs/users.xml b/tests/integration/test_reload_zookeeper/configs/users.xml index 7c18d7144a3..4555a2ed494 100644 --- a/tests/integration/test_reload_zookeeper/configs/users.xml +++ b/tests/integration/test_reload_zookeeper/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml index 9692bebf0eb..6b1611079ec 100644 --- a/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml index 0b82d51bdaf..c02c5161683 100644 --- a/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml index 6c80c344af5..e4c3d4b1bf5 100644 --- a/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml index 77947f5eb95..4dee699da52 100644 --- a/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml index 97190ad1e04..0aa0c1828e2 100644 --- a/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml index c979d76b50c..b18f0fbc93f 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml +++ b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_replicated_merge_tree_config/configs/config.xml b/tests/integration/test_replicated_merge_tree_config/configs/config.xml index ca8edc41ab6..82e716919bc 100644 --- a/tests/integration/test_replicated_merge_tree_config/configs/config.xml +++ b/tests/integration/test_replicated_merge_tree_config/configs/config.xml @@ -1,4 +1,3 @@ - 100 diff --git a/tests/integration/test_replicated_merge_tree_s3_restore/configs/config.d/clusters.xml b/tests/integration/test_replicated_merge_tree_s3_restore/configs/config.d/clusters.xml index 31f7e06db3c..e0b94f46d83 100644 --- a/tests/integration/test_replicated_merge_tree_s3_restore/configs/config.d/clusters.xml +++ b/tests/integration/test_replicated_merge_tree_s3_restore/configs/config.d/clusters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/all_rows.xml b/tests/integration/test_row_policy/all_rows.xml index 948aed1a874..bbeccc1c51a 100644 --- a/tests/integration/test_row_policy/all_rows.xml +++ b/tests/integration/test_row_policy/all_rows.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/configs/users.d/another_user.xml b/tests/integration/test_row_policy/configs/users.d/another_user.xml index 19249011968..5007153fa4a 100644 --- a/tests/integration/test_row_policy/configs/users.d/another_user.xml +++ b/tests/integration/test_row_policy/configs/users.d/another_user.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml b/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml index b93613cba09..32ab078b231 100644 --- a/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml +++ b/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/configs/users.xml b/tests/integration/test_row_policy/configs/users.xml index a583770d1f0..9cb6c5bb93b 100644 --- a/tests/integration/test_row_policy/configs/users.xml +++ b/tests/integration/test_row_policy/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/no_filters.xml b/tests/integration/test_row_policy/no_filters.xml index fddc67fc3f9..3cbf717bb67 100644 --- a/tests/integration/test_row_policy/no_filters.xml +++ b/tests/integration/test_row_policy/no_filters.xml @@ -1,3 +1,2 @@ - diff --git a/tests/integration/test_row_policy/no_rows.xml b/tests/integration/test_row_policy/no_rows.xml index 760203210b2..b7e179bc5c0 100644 --- a/tests/integration/test_row_policy/no_rows.xml +++ b/tests/integration/test_row_policy/no_rows.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/normal_filter2_table2.xml b/tests/integration/test_row_policy/normal_filter2_table2.xml index 3eac0403411..03ed68ea7a3 100644 --- a/tests/integration/test_row_policy/normal_filter2_table2.xml +++ b/tests/integration/test_row_policy/normal_filter2_table2.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/normal_filters.xml b/tests/integration/test_row_policy/normal_filters.xml index 6f5cd8701f6..ab4e08eb308 100644 --- a/tests/integration/test_row_policy/normal_filters.xml +++ b/tests/integration/test_row_policy/normal_filters.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_row_policy/tags_with_db_and_table_names.xml b/tests/integration/test_row_policy/tags_with_db_and_table_names.xml index 992a1e63942..5ce438809d4 100644 --- a/tests/integration/test_row_policy/tags_with_db_and_table_names.xml +++ b/tests/integration/test_row_policy/tags_with_db_and_table_names.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_s3_with_https/configs/config.d/ssl.xml b/tests/integration/test_s3_with_https/configs/config.d/ssl.xml index 1200dc8c94b..d4cea955c68 100644 --- a/tests/integration/test_s3_with_https/configs/config.d/ssl.xml +++ b/tests/integration/test_s3_with_https/configs/config.d/ssl.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_s3_with_https/configs/config.xml b/tests/integration/test_s3_with_https/configs/config.xml index 8e7eada42ed..feb537ebbce 100644 --- a/tests/integration/test_s3_with_https/configs/config.xml +++ b/tests/integration/test_s3_with_https/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_s3_with_proxy/configs/config.xml b/tests/integration/test_s3_with_proxy/configs/config.xml index 8e7eada42ed..feb537ebbce 100644 --- a/tests/integration/test_s3_with_proxy/configs/config.xml +++ b/tests/integration/test_s3_with_proxy/configs/config.xml @@ -1,4 +1,3 @@ - 9000 127.0.0.1 diff --git a/tests/integration/test_select_access_rights/configs/another_user.xml b/tests/integration/test_select_access_rights/configs/another_user.xml index 627ebccdada..c9d480ab99a 100644 --- a/tests/integration/test_select_access_rights/configs/another_user.xml +++ b/tests/integration/test_select_access_rights/configs/another_user.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_send_crash_reports/configs/config_send_crash_reports.xml b/tests/integration/test_send_crash_reports/configs/config_send_crash_reports.xml index 0a3b16dbc84..d63b7b41ca2 100644 --- a/tests/integration/test_send_crash_reports/configs/config_send_crash_reports.xml +++ b/tests/integration/test_send_crash_reports/configs/config_send_crash_reports.xml @@ -1,4 +1,3 @@ - true diff --git a/tests/integration/test_server_reload/configs/default_passwd.xml b/tests/integration/test_server_reload/configs/default_passwd.xml index 5c23be0dcb0..f79149e7e23 100644 --- a/tests/integration/test_server_reload/configs/default_passwd.xml +++ b/tests/integration/test_server_reload/configs/default_passwd.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_server_reload/configs/ssl_conf.xml b/tests/integration/test_server_reload/configs/ssl_conf.xml index 43b25032059..67f1b64bb85 100644 --- a/tests/integration/test_server_reload/configs/ssl_conf.xml +++ b/tests/integration/test_server_reload/configs/ssl_conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_settings_constraints/configs/users.xml b/tests/integration/test_settings_constraints/configs/users.xml index 2ff02086e3e..de4d87b95a0 100644 --- a/tests/integration/test_settings_constraints/configs/users.xml +++ b/tests/integration/test_settings_constraints/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml b/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml index c41776f9e78..c6ac737b632 100644 --- a/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml +++ b/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_dict/configs/conf.xml b/tests/integration/test_storage_dict/configs/conf.xml index e37ca358e63..57b149c89d0 100644 --- a/tests/integration/test_storage_dict/configs/conf.xml +++ b/tests/integration/test_storage_dict/configs/conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_kafka/configs/users.xml b/tests/integration/test_storage_kafka/configs/users.xml index 797dfebba0e..2cef0a6de3c 100644 --- a/tests/integration/test_storage_kafka/configs/users.xml +++ b/tests/integration/test_storage_kafka/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_kerberized_kafka/configs/users.xml b/tests/integration/test_storage_kerberized_kafka/configs/users.xml index 797dfebba0e..2cef0a6de3c 100644 --- a/tests/integration/test_storage_kerberized_kafka/configs/users.xml +++ b/tests/integration/test_storage_kerberized_kafka/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_mysql/configs/users.xml b/tests/integration/test_storage_mysql/configs/users.xml index 074bf9f407e..d030ccb0e72 100644 --- a/tests/integration/test_storage_mysql/configs/users.xml +++ b/tests/integration/test_storage_mysql/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_nats/configs/users.xml b/tests/integration/test_storage_nats/configs/users.xml index 797dfebba0e..2cef0a6de3c 100644 --- a/tests/integration/test_storage_nats/configs/users.xml +++ b/tests/integration/test_storage_nats/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_postgresql/configs/settings.xml b/tests/integration/test_storage_postgresql/configs/settings.xml index 7054c274771..5d4c8f4c523 100644 --- a/tests/integration/test_storage_postgresql/configs/settings.xml +++ b/tests/integration/test_storage_postgresql/configs/settings.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_rabbitmq/configs/users.xml b/tests/integration/test_storage_rabbitmq/configs/users.xml index 797dfebba0e..2cef0a6de3c 100644 --- a/tests/integration/test_storage_rabbitmq/configs/users.xml +++ b/tests/integration/test_storage_rabbitmq/configs/users.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_storage_url/configs/conf.xml b/tests/integration/test_storage_url/configs/conf.xml index e3e8627d95e..790752c5c04 100644 --- a/tests/integration/test_storage_url/configs/conf.xml +++ b/tests/integration/test_storage_url/configs/conf.xml @@ -1,4 +1,3 @@ - diff --git a/tests/integration/test_structured_logging_json/configs/config_json.xml b/tests/integration/test_structured_logging_json/configs/config_json.xml index c0bd4e86a01..f5b00091439 100644 --- a/tests/integration/test_structured_logging_json/configs/config_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_json.xml @@ -1,4 +1,3 @@ - @@ -153,7 +153,7 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( test @@ -182,7 +182,7 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( test @@ -204,7 +204,7 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( test())( From abf51bccfd22c66321a6df26c6b8ac11fb908fd3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 29 Aug 2022 01:18:44 +0800 Subject: [PATCH 154/582] Disable projection when grouping set is used. --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ ...1710_aggregate_projection_with_grouping_set.reference | 6 ++++++ .../01710_aggregate_projection_with_grouping_set.sql | 9 +++++++++ 3 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c19e4f5f559..2a9eae1a026 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5541,6 +5541,10 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (select_query->interpolate() && !select_query->interpolate()->children.empty()) return std::nullopt; + // Currently projections don't support GROUPING SET yet. + if (select_query->group_by_with_grouping_sets) + return std::nullopt; + auto query_options = SelectQueryOptions( QueryProcessingStage::WithMergeableState, /* depth */ 1, diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference new file mode 100644 index 00000000000..fd34e25e370 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference @@ -0,0 +1,6 @@ +a 2 +a x 1 +a y 1 +b 2 +b x 1 +b y 1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql new file mode 100644 index 00000000000..2b393482b43 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql @@ -0,0 +1,9 @@ +drop table if exists test; + +create table test(dim1 String, dim2 String, projection p1 (select dim1, dim2, count() group by dim1, dim2)) engine MergeTree order by dim1; + +insert into test values ('a', 'x') ('a', 'y') ('b', 'x') ('b', 'y'); + +select dim1, dim2, count() from test group by grouping sets ((dim1, dim2), dim1) order by dim1, dim2, count(); + +drop table test; From 6493fb4cbb46fe2e4d8a8a5225dce3f69b5b4054 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 29 Aug 2022 10:09:47 +0800 Subject: [PATCH 155/582] Add more test cases --- ...ate_projection_with_grouping_set.reference | 22 +++++++++++++++++++ ...aggregate_projection_with_grouping_set.sql | 6 +++++ 2 files changed, 28 insertions(+) diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference index fd34e25e370..b233507ce6d 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference @@ -4,3 +4,25 @@ a y 1 b 2 b x 1 b y 1 + 4 +a 2 +a x 1 +a y 1 +b 2 +b x 1 +b y 1 + 4 + x 2 + y 2 +a 2 +a x 1 +a y 1 +b 2 +b x 1 +b y 1 +a x 1 +a y 1 +b x 1 +b y 1 + + 4 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql index 2b393482b43..652ce786b5d 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql @@ -6,4 +6,10 @@ insert into test values ('a', 'x') ('a', 'y') ('b', 'x') ('b', 'y'); select dim1, dim2, count() from test group by grouping sets ((dim1, dim2), dim1) order by dim1, dim2, count(); +select dim1, dim2, count() from test group by dim1, dim2 with rollup order by dim1, dim2, count(); + +select dim1, dim2, count() from test group by dim1, dim2 with cube order by dim1, dim2, count(); + +select dim1, dim2, count() from test group by dim1, dim2 with totals order by dim1, dim2, count(); + drop table test; From 40fb163e8cc84dab39607810cc6fe525b8b45f54 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Aug 2022 11:37:19 +0200 Subject: [PATCH 156/582] Update s3_cache.xml --- tests/config/users.d/s3_cache.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/users.d/s3_cache.xml b/tests/config/users.d/s3_cache.xml index 176128fbf83..69b24ecbbc4 100644 --- a/tests/config/users.d/s3_cache.xml +++ b/tests/config/users.d/s3_cache.xml @@ -1,6 +1,7 @@ + 1 1 From c5bc7b0a0c3a29bb906bc23d7c01d8b52ae5d67e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Jul 2022 14:07:44 +0000 Subject: [PATCH 157/582] Resize pipeline after full sort join --- src/Processors/QueryPlan/JoinStep.cpp | 6 +++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 6 ++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 909933fbed2..6e212a53bc6 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -34,8 +34,12 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); if (join->pipelineType() == JoinPipelineType::YShaped) - return QueryPipelineBuilder::joinPipelinesYShaped( + { + auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); + joined_pipeline->resize(max_streams); + return joined_pipeline; + } return QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 82907d883bc..a7db7442cba 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -348,8 +348,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped left->pipe.dropExtremes(); right->pipe.dropExtremes(); - - if (left->pipe.output_ports.size() != 1 || right->pipe.output_ports.size() != 1) + if (left->getNumStreams() != 1 || right->getNumStreams() != 1) throw Exception("Join is supported only for pipelines with one output port", ErrorCodes::LOGICAL_ERROR); if (left->hasTotals() || right->hasTotals()) @@ -359,8 +358,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped auto joining = std::make_shared(join, inputs, out_header, max_block_size); - auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); - return result; + return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); } std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLeft( From 031aaf3a4545ba491e02bec1af0001df06add20b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Jul 2022 17:43:18 +0000 Subject: [PATCH 158/582] Add Creating/FilterBySetsOnTheFlyStep for full sorting join --- src/Core/Settings.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 26 ++++ src/Interpreters/Set.cpp | 16 ++- src/Interpreters/Set.h | 8 +- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 134 +++++++++++++++++ .../QueryPlan/CreatingSetOnTheFlyStep.h | 58 ++++++++ .../CreatingSetsOnTheFlyTransform.cpp | 135 ++++++++++++++++++ .../CreatingSetsOnTheFlyTransform.h | 71 +++++++++ 8 files changed, 445 insertions(+), 5 deletions(-) create mode 100644 src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp create mode 100644 src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h create mode 100644 src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp create mode 100644 src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index af32c15a867..60f588f1618 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,6 +366,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ + M(UInt64, max_bytes_in_set_to_optimize_join, 1_GiB, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ + \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ M(String, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 76f199d8e41..cb33f3657b2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -1445,9 +1446,34 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

SetPtr + { + SizeLimits size_limits(0, settings.max_bytes_in_set_to_optimize_join, OverflowMode::BREAK); + auto creating_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, size_limits); + SetPtr set = creating_set_step->getSet(); + plan.addStep(std::move(creating_set_step)); + return set; + }; + + auto add_filter_by_set = [](QueryPlan & plan, const Names & key_names, SetPtr set) + { + auto filter_by_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, set); + plan.addStep(std::move(filter_by_set_step)); + }; + if (expressions.join->pipelineType() == JoinPipelineType::YShaped) { const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); + + if (settings.max_bytes_in_set_to_optimize_join > 0) + { + SetPtr left_set = add_create_set(query_plan, join_clause.key_names_left); + SetPtr right_set = add_create_set(*joined_plan, join_clause.key_names_right); + + add_filter_by_set(query_plan, join_clause.key_names_left, right_set); + add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set); + } + add_sorting(query_plan, join_clause.key_names_left, false); add_sorting(*joined_plan, join_clause.key_names_right, true); } diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index af05b33c1f6..7d279750f4c 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -22,6 +22,8 @@ #include #include +#include + #include #include @@ -162,8 +164,16 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) data.init(data.chooseMethod(key_columns, key_sizes)); } - bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) +{ + Columns cols; + cols.reserve(columns.size()); + for (const auto & column : columns) + cols.emplace_back(column.column); + return insertFromBlock(cols); +} + +bool Set::insertFromBlock(const Columns & columns) { std::lock_guard lock(rwlock); @@ -179,11 +189,11 @@ bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(columns.at(i).column->convertToFullIfNeeded()); + materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); key_columns.emplace_back(materialized_columns.back().get()); } - size_t rows = columns.at(0).column->size(); + size_t rows = columns.at(0)->size(); /// We will insert to the Set only keys, where all components are not NULL. ConstNullMapPtr null_map{}; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 6a3b28407ee..44f543ce222 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -20,6 +20,7 @@ class Context; class IFunctionBase; using FunctionBasePtr = std::shared_ptr; +class Chunk; /** Data structure for implementation of IN expression. */ @@ -45,11 +46,14 @@ public: void setHeader(const ColumnsWithTypeAndName & header); /// Returns false, if some limit was exceeded and no need to insert more data. + bool insertFromBlock(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); + /// Call after all blocks were inserted. To get the information that set is already created. void finishInsert() { is_created = true; } - bool isCreated() const { return is_created; } + /// finishInsert and isCreated are thread-safe + bool isCreated() const { return is_created.load(); } /** For columns of 'block', check belonging of corresponding rows to the set. * Return UInt8 column with the result. @@ -111,7 +115,7 @@ private: bool transform_null_in; /// Check if set contains all the data. - bool is_created = false; + std::atomic is_created = false; /// If in the left part columns contains the same types as the elements of the set. void executeOrdinary( diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp new file mode 100644 index 00000000000..7a718a17ad7 --- /dev/null +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -0,0 +1,134 @@ +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static ITransformingStep::Traits getTraits(bool is_filter) +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = !is_filter, + } + }; +} + +CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, const SizeLimits & size_limits) + : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) + , column_names(column_names_) +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + + set = std::make_shared(size_limits, false, true); + + { + ColumnsWithTypeAndName header; + for (const auto & name : column_names) + { + ColumnWithTypeAndName column = input_streams[0].header.getByName(name); + header.emplace_back(column); + } + set->setHeader(header); + } +} + +void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) +{ + UNUSED(settings); + size_t num_streams = pipeline.getNumStreams(); + + pipeline.resize(1); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + return std::make_shared(header, column_names, set); + }); + pipeline.resize(num_streams); +} + +void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const +{ + map.add(getName(), true); +} + +void CreatingSetOnTheFlyStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + settings.out << prefix << getName(); + + settings.out << '\n'; +} + +void CreatingSetOnTheFlyStep::updateOutputStream() +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + + output_stream = input_streams[0]; +} + + +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetPtr set_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) + , column_names(column_names_) + , set(set_) +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); +} + +void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) +{ + UNUSED(settings); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + return std::make_shared(header, column_names, set); + }); +} + +void FilterBySetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const +{ + map.add(getName(), true); +} + +void FilterBySetOnTheFlyStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + settings.out << prefix << getName(); + + settings.out << '\n'; +} + +void FilterBySetOnTheFlyStep::updateOutputStream() +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + + output_stream = input_streams[0]; +} + + +} diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h new file mode 100644 index 00000000000..9482428534e --- /dev/null +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -0,0 +1,58 @@ +#pragma once +#include +#include + +#include + + +namespace DB +{ + +using SetPtr = std::shared_ptr; + +class CreatingSetOnTheFlyStep : public ITransformingStep +{ +public: + explicit CreatingSetOnTheFlyStep( + const DataStream & input_stream_, + const Names & column_names_, + const SizeLimits & size_limits = {}); + + String getName() const override { return "CreatingSetsOnTheFly"; } + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + + SetPtr getSet() const { return set; } + +private: + void updateOutputStream() override; + + Names column_names; + SetPtr set; +}; + + +class FilterBySetOnTheFlyStep : public ITransformingStep +{ +public: + FilterBySetOnTheFlyStep( + const DataStream & input_stream_, + const Names & column_names_, + SetPtr set_); + + String getName() const override { return "FilterBySetOnTheFly"; } + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + +private: + void updateOutputStream() override; + + Names column_names; + SetPtr set; +}; + +} diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp new file mode 100644 index 00000000000..10b8513ee2b --- /dev/null +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -0,0 +1,135 @@ +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +std::vector getColumnIndices(const Block & block, const Names & column_names) +{ + std::vector indices; + for (const auto & name : column_names) + indices.push_back(block.getPositionByName(name)); + return indices; +} + +Columns getColumnsByIndices(const Chunk & chunk, const std::vector & indices) +{ + Columns columns; + const Columns & all_cols = chunk.getColumns(); + for (const auto & index : indices) + columns.push_back(all_cols.at(index)); + return columns; +} + +ColumnsWithTypeAndName getColumnsByIndices(const Block & sample_block, const Chunk & chunk, const std::vector & indices) +{ + Block block = sample_block.cloneEmpty(); + block.setColumns(getColumnsByIndices(chunk, indices)); + return block.getColumnsWithTypeAndName(); +} + +} + + +CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) + : ISimpleTransform(header_, header_, true) + , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) + , set(set_) + , log(&Poco::Logger::get(getName())) +{ +} + +IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() +{ + auto status = ISimpleTransform::prepare(); + return status; +} + +void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) +{ + if (!set) + return; + + if (chunk.getNumRows()) + { + Columns key_cols = getColumnsByIndices(chunk, key_column_indices); + bool limit_exceeded = !set->insertFromBlock(key_cols); + if (limit_exceeded) + { + LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); + // set->clear(); + // LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); + set.reset(); + } + } + + if (input.isFinished()) + { + set->finishInsert(); + LOG_DEBUG(log, "Finish building set with {} rows, set size is {} MB", set->getTotalRowCount(), set->getTotalByteCount() / 1024 / 1024); + + /// Release pointer to make it possible destroy it by consumer + set.reset(); + } +} + +FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) + : ISimpleTransform(header_, header_, true) + , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) + , set(set_) + , log(&Poco::Logger::get(getName())) +{ + const auto & header = inputs.front().getHeader(); + for (size_t idx : key_column_indices) + key_sample_block.insert(header.getByPosition(idx)); +} + +IProcessor::Status FilterBySetOnTheFlyTransform::prepare() +{ + auto status = ISimpleTransform::prepare(); + return status; +} + +void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) +{ + + if (!set) + return; + + if (!set->isCreated()) + return; + + if (chunk.getNumRows()) + { + auto key_columns = getColumnsByIndices(key_sample_block, chunk, key_column_indices); + ColumnPtr mask_col = set->execute(key_columns, false); + const auto & mask = assert_cast(mask_col.get())->getData(); + + Columns columns = chunk.detachColumns(); + size_t num_rows = 0; + for (auto & col : columns) + { + col = col->filter(mask, 0); + num_rows = col->size(); + } + chunk.setColumns(std::move(columns), num_rows); + } + + if (input.isFinished()) + { + /// Release set to free memory + set.reset(); + } +} + +} diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h new file mode 100644 index 00000000000..e36e850030f --- /dev/null +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h @@ -0,0 +1,71 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +class Set; +using SetPtr = std::shared_ptr; + +/* + * Create a set on the fly. + * The set is created from the key columns of the input block. + * Data is not changed and returned as is. + * Can be executed only in one stream. + */ +class CreatingSetsOnTheFlyTransform : public ISimpleTransform +{ +public: + explicit CreatingSetsOnTheFlyTransform( + const Block & header_, const Names & column_names_, SetPtr set_); + + String getName() const override { return "CreatingSetsOnTheFlyTransform"; } + + Status prepare() override; + + void transform(Chunk & chunk) override; + +private: + std::vector key_column_indices; + + /// Set to fill + SetPtr set; + + Poco::Logger * log; +}; + + +/* + * Filter the input chunk by the set. + * When set building is not comleted, just return the source data. + */ +class FilterBySetOnTheFlyTransform : public ISimpleTransform +{ +public: + explicit FilterBySetOnTheFlyTransform( + const Block & header_, const Names & column_names_, SetPtr set_); + + String getName() const override { return "FilterBySetOnTheFlyTransform"; } + + Status prepare() override; + + void transform(Chunk & chunk) override; + +private: + /// Set::execute requires ColumnsWithTypesAndNames, so we need to convert Chunk to that format + Block key_sample_block; + + std::vector key_column_indices; + + /// Filter by this set when it's created + SetPtr set; + + Poco::Logger * log; +}; + + + +} From 3292566603df6511dd65365e9fd1f8b53e76e4cc Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Jul 2022 10:21:04 +0000 Subject: [PATCH 159/582] Format bytes in CreatingSetsOnTheFlyTransform logs --- .../CreatingSetsOnTheFlyTransform.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index 10b8513ee2b..4a1ea45cdef 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -38,6 +38,17 @@ ColumnsWithTypeAndName getColumnsByIndices(const Block & sample_block, const Chu return block.getColumnsWithTypeAndName(); } +std::string formatBytesHumanReadable(size_t bytes) +{ + if (bytes >= 1_GiB) + return fmt::format("{:.2f} GB", static_cast(bytes) / 1_GiB); + if (bytes >= 1_MiB) + return fmt::format("{:.2f} MB", static_cast(bytes) / 1_MiB); + if (bytes >= 1_KiB) + return fmt::format("{:.2f} KB", static_cast(bytes) / 1_KiB); + return fmt::format("{:.2f} B", static_cast(bytes)); +} + } @@ -66,9 +77,8 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) bool limit_exceeded = !set->insertFromBlock(key_cols); if (limit_exceeded) { - LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); + LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {}", formatBytesHumanReadable(set->getTotalByteCount())); // set->clear(); - // LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); set.reset(); } } @@ -76,7 +86,8 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (input.isFinished()) { set->finishInsert(); - LOG_DEBUG(log, "Finish building set with {} rows, set size is {} MB", set->getTotalRowCount(), set->getTotalByteCount() / 1024 / 1024); + LOG_DEBUG(log, "Finish building set with {} rows, set size is {}", + set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); /// Release pointer to make it possible destroy it by consumer set.reset(); From 8f06430ebda36d52c80a0933ba0dc15b8b7c9fab Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Jul 2022 13:35:22 +0000 Subject: [PATCH 160/582] Create sets for joins: upd --- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++--- src/Interpreters/TableJoin.h | 24 ++++++-- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 17 ++++-- .../QueryPlan/CreatingSetOnTheFlyStep.h | 12 ++-- .../Optimizations/filterPushDown.cpp | 48 +++++++++------ .../CreatingSetsOnTheFlyTransform.cpp | 59 +++++++++++++------ .../CreatingSetsOnTheFlyTransform.h | 36 +++++++++-- src/QueryPipeline/Pipe.h | 2 +- 8 files changed, 146 insertions(+), 68 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cb33f3657b2..3229313941f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1446,18 +1446,20 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

SetPtr + auto add_create_set = [&settings](QueryPlan & plan, const Names & key_names, bool is_right) { SizeLimits size_limits(0, settings.max_bytes_in_set_to_optimize_join, OverflowMode::BREAK); auto creating_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, size_limits); - SetPtr set = creating_set_step->getSet(); + creating_set_step->setStepDescription(fmt::format("Create set for {} stream", is_right ? "right" : "left")); + auto set = creating_set_step->getSet(); plan.addStep(std::move(creating_set_step)); return set; }; - auto add_filter_by_set = [](QueryPlan & plan, const Names & key_names, SetPtr set) + auto add_filter_by_set = [](QueryPlan & plan, const Names & key_names, auto set, bool is_right) { auto filter_by_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, set); + filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); plan.addStep(std::move(filter_by_set_step)); }; @@ -1467,11 +1469,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

0) { - SetPtr left_set = add_create_set(query_plan, join_clause.key_names_left); - SetPtr right_set = add_create_set(*joined_plan, join_clause.key_names_right); + auto left_set = add_create_set(query_plan, join_clause.key_names_left, false); + auto right_set = add_create_set(*joined_plan, join_clause.key_names_right, true); - add_filter_by_set(query_plan, join_clause.key_names_left, right_set); - add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set); + add_filter_by_set(query_plan, join_clause.key_names_left, right_set, false); + add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set, true); } add_sorting(query_plan, join_clause.key_names_left, false); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index d0bf64fdebe..3835ef77deb 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -73,16 +73,32 @@ public: return key_names_right.size(); } - String formatDebug() const + String formatDebug(bool short_format = false) const { - return fmt::format("Left keys: [{}] Right keys [{}] Condition columns: '{}', '{}'", - fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), - condColumnNames().first, condColumnNames().second); + const auto & [left_cond, right_cond] = condColumnNames(); + + if (short_format) + { + return fmt::format("({}) = ({}){}{}", fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), + !left_cond.empty() ? " AND " + left_cond : "", !right_cond.empty() ? " AND " + right_cond : ""); + } + + return fmt::format( + "Left keys: [{}] Right keys [{}] Condition columns: '{}', '{}'", + fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), left_cond, right_cond); } }; using Clauses = std::vector; + static std::string formatClauses(const Clauses & clauses, bool short_format = false) + { + std::vector res; + for (const auto & clause : clauses) + res.push_back("[" + clause.formatDebug(short_format) + "]"); + return fmt::format("{}", fmt::join(res, "; ")); + } + private: /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` * The join is made by column k. diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index 7a718a17ad7..8eaeecb597f 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -22,9 +22,9 @@ static ITransformingStep::Traits getTraits(bool is_filter) { { .preserves_distinct_columns = true, - .returns_single_stream = true, + .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = true, + .preserves_sorting = false, /// resize doesn't perserve sorting (TODO fixit) }, { .preserves_number_of_rows = !is_filter, @@ -39,7 +39,7 @@ CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep(const DataStream & input_stream if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); - set = std::make_shared(size_limits, false, true); + set = std::make_shared(size_limits, false, true); { ColumnsWithTypeAndName header; @@ -60,9 +60,12 @@ void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared(header, column_names, set); + auto res = std::make_shared(header, column_names, set); + res->setDescription(this->getStepDescription()); + return res; }); pipeline.resize(num_streams); } @@ -89,7 +92,7 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetPtr set_) +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetWithStatePtr set_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) , column_names(column_names_) , set(set_) @@ -105,7 +108,9 @@ void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared(header, column_names, set); + auto res = std::make_shared(header, column_names, set); + res->setDescription(this->getStepDescription()); + return res; }); } diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 9482428534e..4966173152a 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -2,14 +2,10 @@ #include #include -#include - namespace DB { -using SetPtr = std::shared_ptr; - class CreatingSetOnTheFlyStep : public ITransformingStep { public: @@ -24,13 +20,13 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - SetPtr getSet() const { return set; } + SetWithStatePtr getSet() const { return set; } private: void updateOutputStream() override; Names column_names; - SetPtr set; + SetWithStatePtr set; }; @@ -40,7 +36,7 @@ public: FilterBySetOnTheFlyStep( const DataStream & input_stream_, const Names & column_names_, - SetPtr set_); + SetWithStatePtr set_); String getName() const override { return "FilterBySetOnTheFly"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; @@ -52,7 +48,7 @@ private: void updateOutputStream() override; Names column_names; - SetPtr set; + SetWithStatePtr set; }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 6c6c8954ea4..99312a17ffd 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ #include #include #include +#include namespace DB::ErrorCodes { @@ -134,10 +136,24 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con static size_t tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const Names & allowed_inputs, - bool can_remove_filter = true) + bool can_remove_filter = true, size_t child_idx = 0) { - if (auto split_filter = splitFilter(parent_node, allowed_inputs, 0)) - return tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, 0); + if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx)) + return tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); + return 0; +} + + +/// Push down filter through specified type of step +template +static size_t simplePushDownOverStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, QueryPlanStepPtr & child) +{ + if (typeid_cast(child.get())) + { + Names allowed_inputs = child->getOutputStream().header.getNames(); + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) + return updated_steps; + } return 0; } @@ -234,12 +250,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return updated_steps; } - if (auto * distinct = typeid_cast(child.get())) - { - Names allowed_inputs = distinct->getOutputStream().header.getNames(); - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) - return updated_steps; - } + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; if (auto * join = typeid_cast(child.get())) { @@ -290,9 +302,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes const size_t updated_steps = tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); if (updated_steps > 0) { - LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter to {} side of join", kind); + LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); + return updated_steps; } - return updated_steps; }; if (size_t updated_steps = join_push_down(JoinKind::Left)) @@ -321,12 +333,14 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes // { // } - if (typeid_cast(child.get())) - { - Names allowed_inputs = child->getOutputStream().header.getNames(); - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) - return updated_steps; - } + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; + + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; + + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; if (auto * union_step = typeid_cast(child.get())) { diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index 4a1ea45cdef..d52a669ee75 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -52,8 +52,9 @@ std::string formatBytesHumanReadable(size_t bytes) } -CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) +CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) + , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , set(set_) , log(&Poco::Logger::get(getName())) @@ -77,8 +78,10 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) bool limit_exceeded = !set->insertFromBlock(key_cols); if (limit_exceeded) { - LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {}", formatBytesHumanReadable(set->getTotalByteCount())); + LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {}", + getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); // set->clear(); + set->state = SetWithState::State::Suspended; set.reset(); } } @@ -86,16 +89,18 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (input.isFinished()) { set->finishInsert(); - LOG_DEBUG(log, "Finish building set with {} rows, set size is {}", - set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); /// Release pointer to make it possible destroy it by consumer set.reset(); } } -FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) +FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) + , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , set(set_) , log(&Poco::Logger::get(getName())) @@ -108,38 +113,54 @@ FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_ IProcessor::Status FilterBySetOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); + if (status == Status::Finished) + { + bool has_filter = set && set->state == SetWithState::State::Finished; + if (has_filter) + { + LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {}% filtered", + Poco::toLower(getDescription()), fmt::join(column_names, ", "), + stat.consumed_rows, stat.consumed_rows_before_set, stat.result_rows, + static_cast(100 - 100.0 * stat.result_rows / stat.consumed_rows)); + } + else + { + LOG_DEBUG(log, "Finished {}: bypass {} rows", Poco::toLower(getDescription()), stat.consumed_rows); + } + + /// Release set to free memory + set = nullptr; + } return status; } void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) { + stat.consumed_rows += chunk.getNumRows(); + stat.result_rows += chunk.getNumRows(); + bool can_filter = set && set->state == SetWithState::State::Finished; - if (!set) - return; + if (!can_filter) + stat.consumed_rows_before_set += chunk.getNumRows(); - if (!set->isCreated()) - return; - - if (chunk.getNumRows()) + if (can_filter && chunk.getNumRows()) { auto key_columns = getColumnsByIndices(key_sample_block, chunk, key_column_indices); ColumnPtr mask_col = set->execute(key_columns, false); const auto & mask = assert_cast(mask_col.get())->getData(); + stat.result_rows -= chunk.getNumRows(); + Columns columns = chunk.detachColumns(); - size_t num_rows = 0; + size_t result_num_rows = 0; for (auto & col : columns) { col = col->filter(mask, 0); - num_rows = col->size(); + result_num_rows = col->size(); } - chunk.setColumns(std::move(columns), num_rows); - } + stat.result_rows += result_num_rows; - if (input.isFinished()) - { - /// Release set to free memory - set.reset(); + chunk.setColumns(std::move(columns), result_num_rows); } } diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h index e36e850030f..ea91a96818a 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h @@ -2,13 +2,28 @@ #include #include #include +#include namespace DB { -class Set; -using SetPtr = std::shared_ptr; +class SetWithState : public Set +{ +public: + using Set::Set; + + enum class State + { + Creating, + Finished, + Suspended, + }; + + std::atomic state = State::Creating; +}; + +using SetWithStatePtr = std::shared_ptr; /* * Create a set on the fly. @@ -20,7 +35,7 @@ class CreatingSetsOnTheFlyTransform : public ISimpleTransform { public: explicit CreatingSetsOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetPtr set_); + const Block & header_, const Names & column_names_, SetWithStatePtr set_); String getName() const override { return "CreatingSetsOnTheFlyTransform"; } @@ -29,10 +44,11 @@ public: void transform(Chunk & chunk) override; private: + Names column_names; std::vector key_column_indices; /// Set to fill - SetPtr set; + SetWithStatePtr set; Poco::Logger * log; }; @@ -46,7 +62,7 @@ class FilterBySetOnTheFlyTransform : public ISimpleTransform { public: explicit FilterBySetOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetPtr set_); + const Block & header_, const Names & column_names_, SetWithStatePtr set_); String getName() const override { return "FilterBySetOnTheFlyTransform"; } @@ -58,10 +74,18 @@ private: /// Set::execute requires ColumnsWithTypesAndNames, so we need to convert Chunk to that format Block key_sample_block; + Names column_names; std::vector key_column_indices; /// Filter by this set when it's created - SetPtr set; + SetWithStatePtr set; + + struct Stat + { + size_t consumed_rows = 0; + size_t consumed_rows_before_set = 0; + size_t result_rows = 0; + } stat; Poco::Logger * log; }; diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 52059f4ad19..664ab719b13 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -85,7 +85,7 @@ public: /// Add chain to every output port. void addChains(std::vector chains); - /// Changes the number of output ports if needed. Adds ResizeTransform. + /// Changes the number of output ports if needed. Adds (Strict)ResizeProcessor. void resize(size_t num_streams, bool force = false, bool strict = false); using Transformer = std::function; From 71708d595f9a73b1ba12bd2636064b8e900461f3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 Aug 2022 13:20:12 +0000 Subject: [PATCH 161/582] Create sets for joins: wip --- src/Interpreters/InterpreterSelectQuery.cpp | 9 +- src/Processors/DelayedPortsProcessor.cpp | 128 ++++++++++++++++++ src/Processors/DelayedPortsProcessor.h | 74 ++++++++++ src/Processors/Port.cpp | 2 +- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 31 ++++- .../QueryPlan/CreatingSetOnTheFlyStep.h | 34 ++++- src/Processors/ResizeProcessor.h | 7 + src/QueryPipeline/Pipe.cpp | 5 +- src/QueryPipeline/Pipe.h | 2 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 2 +- 11 files changed, 286 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3229313941f..74e25562d29 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1456,9 +1456,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(plan.getCurrentDataStream(), key_names, set); + auto filter_by_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, set, ports_state); filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); plan.addStep(std::move(filter_by_set_step)); }; @@ -1472,8 +1472,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(); + add_filter_by_set(query_plan, join_clause.key_names_left, right_set, ports_state, false); + add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set, ports_state, true); } add_sorting(query_plan, join_clause.key_names_left, false); diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index 24023529bca..48930ca9e51 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -170,4 +170,132 @@ IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_in return Status::PortFull; } + +NotifyProcessor::NotifyProcessor(const Block & header, size_t num_ports) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)) + , aux_in_port(Block(), this) + , aux_out_port(Block(), this) +{ + port_pairs.resize(num_ports); + + auto input_it = inputs.begin(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + port_pairs[i].input_port = &*input_it; + ++input_it; + + port_pairs[i].output_port = &*output_it; + ++output_it; + } + +} + +void NotifyProcessor::finishPair(PortsPair & pair) +{ + if (!pair.is_finished) + { + pair.output_port->finish(); + pair.input_port->close(); + + pair.is_finished = true; + ++num_finished_pairs; + } +} + +bool NotifyProcessor::processPair(PortsPair & pair) +{ + if (pair.output_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (pair.input_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (!pair.output_port->canPush()) + { + pair.input_port->setNotNeeded(); + return false; + } + + pair.input_port->setNeeded(); + if (pair.input_port->hasData()) + { + Chunk chunk = pair.input_port->pull(true); + dataCallback(chunk); + pair.output_port->push(std::move(chunk)); + } + + return true; +} + +IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + bool need_data = false; + + for (const auto & output_number : updated_outputs) + need_data = processPair(port_pairs[output_number]) || need_data; + for (const auto & input_number : updated_inputs) + need_data = processPair(port_pairs[input_number]) || need_data; + + if (num_finished_pairs == port_pairs.size()) + return Status::Finished; + + if (need_data) + return Status::NeedData; + + return Status::PortFull; +} + +IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + auto status = processRegularPorts(updated_inputs, updated_outputs); + if (status != Status::Ready) + return status; + + if (ready == AuxPortState::NotInitialized && isReady()) + ready = AuxPortState::Triggered; + + if (ready == AuxPortState::Triggered) + { + if (aux_out_port.canPush()) + { + aux_out_port.push({}); + ready = AuxPortState::Finished; + return Status::Ready; + } + return Status::PortFull; + } + + if (waiting == AuxPortState::NotInitialized && isWaiting()) + { + aux_in_port.setNeeded(); + waiting = AuxPortState::Triggered; + } + + if (waiting == AuxPortState::Triggered) + { + if (aux_in_port.hasData()) + { + aux_in_port.pull(true); + waiting = AuxPortState::Finished; + return Status::Ready; + } + return Status::PortFull; + } + + return Status::Ready; +} + +std::pair NotifyProcessor::getAuxPorts() +{ + return std::make_pair(&aux_in_port, &aux_out_port); +} + + } diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index a6a9590e0c8..a5314ca36a1 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -1,5 +1,7 @@ #pragma once #include +#include +#include namespace DB { @@ -39,4 +41,76 @@ private: void finishPair(PortsPair & pair); }; + +class NotifyProcessor : public IProcessor +{ +public: + NotifyProcessor(const Block & header, size_t num_ports); + + String getName() const override { return "NotifyProcessor"; } + + Status prepare(const PortNumbers &, const PortNumbers &) override; + + std::pair getAuxPorts(); + + virtual bool isReady() const { return true; } + virtual bool isWaiting() const { return false; } + + virtual void dataCallback(const Chunk & chunk) { UNUSED(chunk); } + +private: + + enum class AuxPortState + { + NotInitialized, + Triggered, + Finished, + }; + + struct PortsPair + { + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_finished = false; + }; + + bool processPair(PortsPair & pair); + void finishPair(PortsPair & pair); + Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); + + std::vector port_pairs; + size_t num_finished_pairs = 0; + + InputPort aux_in_port; + OutputPort aux_out_port; + + AuxPortState ready = AuxPortState::NotInitialized; + AuxPortState waiting = AuxPortState::NotInitialized; +}; + + +class NotifyProcessor2 : public NotifyProcessor +{ +public: + using NotifyProcessor::NotifyProcessor; + + bool isReady() const override + { + return data_consumed > 10_MiB; + } + + bool isWaiting() const override + { + return data_consumed < 10_MiB; + } + + void dataCallback(const Chunk & chunk) override + { + data_consumed += chunk.allocatedBytes(); + } + +private: + size_t data_consumed = 0; +}; + } diff --git a/src/Processors/Port.cpp b/src/Processors/Port.cpp index 86431dbc6e6..a856dbcea3b 100644 --- a/src/Processors/Port.cpp +++ b/src/Processors/Port.cpp @@ -19,7 +19,7 @@ void connect(OutputPort & output, InputPort & input) auto out_name = output.getProcessor().getName(); auto in_name = input.getProcessor().getName(); - assertCompatibleHeader(output.getHeader(), input.getHeader(), fmt::format(" function connect between {} and {}", out_name, in_name)); + assertCompatibleHeader(output.getHeader(), input.getHeader(), fmt::format("function connect between {} and {}", out_name, in_name)); input.output_port = &output; output.input_port = &input; diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index 8eaeecb597f..0bb363fc17d 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -92,18 +93,46 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetWithStatePtr set_) +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, + SetWithStatePtr set_, PortsStatePtr ports_state_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) , column_names(column_names_) , set(set_) + , ports_state(ports_state_) { if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); } + +static void connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs) +{ + auto input_it = inputs.begin(); + for (auto & port : ports) + { + connect(*port, *input_it); + input_it++; + } + assert(input_it == inputs.end()); +} + void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { UNUSED(settings); + + Block input_header = pipeline.getHeader(); + pipeline.transform([&input_header, this](OutputPortRawPtrs ports) + { + size_t num_streams = ports.size(); + + auto notifier = std::make_shared(input_header, num_streams); + + connectAllInputs(ports, notifier->getInputs()); + ports_state->tryConnectPorts(notifier->getAuxPorts()); + + return Processors{notifier}; + }, /* check_ports= */ false); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 4966173152a..2a85d26836d 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -33,10 +33,40 @@ private: class FilterBySetOnTheFlyStep : public ITransformingStep { public: + + class PortsState : public boost::noncopyable + { + public: + using PortPair = std::pair; + + /// Remember ports passed on the first call and connect with ones from second call. + bool tryConnectPorts(PortPair rhs_ports) + { + std::lock_guard lock(mux); + if (input_port || output_port) + { + assert(input_port && output_port); + connect(*rhs_ports.second, *input_port); + connect(*output_port, *rhs_ports.first); + return true; + } + std::tie(input_port, output_port) = rhs_ports; + return false; + } + + private: + std::mutex mux; + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + }; + + using PortsStatePtr = std::shared_ptr; + FilterBySetOnTheFlyStep( const DataStream & input_stream_, const Names & column_names_, - SetWithStatePtr set_); + SetWithStatePtr set_, + PortsStatePtr ports_state_); String getName() const override { return "FilterBySetOnTheFly"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; @@ -48,7 +78,9 @@ private: void updateOutputStream() override; Names column_names; + SetWithStatePtr set; + PortsStatePtr ports_state; }; } diff --git a/src/Processors/ResizeProcessor.h b/src/Processors/ResizeProcessor.h index f9c188e041a..364d1b4c883 100644 --- a/src/Processors/ResizeProcessor.h +++ b/src/Processors/ResizeProcessor.h @@ -85,6 +85,13 @@ public: { } + StrictResizeProcessor(InputPorts inputs_, OutputPorts outputs_) + : IProcessor(inputs_, outputs_) + , current_input(inputs.begin()) + , current_output(outputs.begin()) + { + } + String getName() const override { return "StrictResize"; } Status prepare(const PortNumbers &, const PortNumbers &) override; diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index ae342abeea5..224dd9e5145 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -770,7 +770,7 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) header.clear(); } -void Pipe::transform(const Transformer & transformer) +void Pipe::transform(const Transformer & transformer, bool check_ports) { if (output_ports.empty()) throw Exception("Cannot transform empty Pipe", ErrorCodes::LOGICAL_ERROR); @@ -799,6 +799,9 @@ void Pipe::transform(const Transformer & transformer) { for (const auto & port : processor->getInputs()) { + if (!check_ports) + break; + if (!port.isConnected()) throw Exception( ErrorCodes::LOGICAL_ERROR, diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 664ab719b13..79d19a18193 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -91,7 +91,7 @@ public: using Transformer = std::function; /// Transform Pipe in general way. - void transform(const Transformer & transformer); + void transform(const Transformer & transformer, bool check_ports = true); /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a7db7442cba..877c19a493e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -159,10 +159,10 @@ void QueryPipelineBuilder::addChain(Chain chain) pipe.addChains(std::move(chains)); } -void QueryPipelineBuilder::transform(const Transformer & transformer) +void QueryPipelineBuilder::transform(const Transformer & transformer, bool check_ports) { checkInitializedAndNotCompleted(); - pipe.transform(transformer); + pipe.transform(transformer, check_ports); } void QueryPipelineBuilder::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 100a2e07341..4edae83fe86 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -69,7 +69,7 @@ public: using Transformer = std::function; /// Transform pipeline in general way. - void transform(const Transformer & transformer); + void transform(const Transformer & transformer, bool check_ports = true); /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); From 31a167848df68b6a84bf1e6e5c150a6543b51b28 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 Aug 2022 14:03:57 +0000 Subject: [PATCH 162/582] Fix set finish condition in CreatingSetsOnTheFlyTransform --- .../CreatingSetsOnTheFlyTransform.cpp | 40 ++++++++++++------- 1 file changed, 26 insertions(+), 14 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index d52a669ee75..b3a3410dd9e 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -64,6 +64,29 @@ CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & heade IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); + + if (status == Status::Finished && set) + { + if (input.isFinished()) + { + set->finishInsert(); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + } + else + { + /// Should not happen because processor places before join that reads all the data + /// But let's hanlde this case just for safety. + set->state = SetWithState::State::Suspended; + LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", + getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); + } + + /// Release pointer to make it possible destroy it by consumer + set.reset(); + } + return status; } @@ -74,28 +97,17 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { - Columns key_cols = getColumnsByIndices(chunk, key_column_indices); - bool limit_exceeded = !set->insertFromBlock(key_cols); + Columns key_columns = getColumnsByIndices(chunk, key_column_indices); + bool limit_exceeded = !set->insertFromBlock(key_columns); if (limit_exceeded) { LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); - // set->clear(); + // TODO(@vdimir): set->clear() ? set->state = SetWithState::State::Suspended; set.reset(); } } - - if (input.isFinished()) - { - set->finishInsert(); - set->state = SetWithState::State::Finished; - LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); - - /// Release pointer to make it possible destroy it by consumer - set.reset(); - } } FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) From c778bba13f18d555b417afd250841f30a3ed9cd2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 2 Aug 2022 13:46:02 +0000 Subject: [PATCH 163/582] Create sets for joins: wip --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 13 +- src/Processors/DelayedPortsProcessor.cpp | 166 ++++++++++++++---- src/Processors/DelayedPortsProcessor.h | 67 ++++--- src/Processors/Port.cpp | 10 +- src/Processors/Port.h | 8 +- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 58 ++++-- .../QueryPlan/CreatingSetOnTheFlyStep.h | 30 +++- .../CreatingSetsOnTheFlyTransform.cpp | 41 +++-- .../CreatingSetsOnTheFlyTransform.h | 9 +- src/QueryPipeline/Pipe.cpp | 7 +- 11 files changed, 301 insertions(+), 110 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 60f588f1618..8a81a8b0b29 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(UInt64, max_bytes_in_set_to_optimize_join, 1_GiB, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ + M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining (-1 to disable).", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 74e25562d29..f16c81e93b1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1448,7 +1448,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(plan.getCurrentDataStream(), key_names, size_limits); creating_set_step->setStepDescription(fmt::format("Create set for {} stream", is_right ? "right" : "left")); auto set = creating_set_step->getSet(); @@ -1456,9 +1456,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(plan.getCurrentDataStream(), key_names, set, ports_state); + auto filter_by_set_step = std::make_unique( + plan.getCurrentDataStream(), rhs_header, key_names, settings.max_rows_in_set_to_optimize_join, set, ports_state); filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); plan.addStep(std::move(filter_by_set_step)); }; @@ -1467,14 +1468,14 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getTableJoin().getOnlyClause(); - if (settings.max_bytes_in_set_to_optimize_join > 0) + if (settings.max_rows_in_set_to_optimize_join > 0) { auto left_set = add_create_set(query_plan, join_clause.key_names_left, false); auto right_set = add_create_set(*joined_plan, join_clause.key_names_right, true); auto ports_state = std::make_shared(); - add_filter_by_set(query_plan, join_clause.key_names_left, right_set, ports_state, false); - add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set, ports_state, true); + add_filter_by_set(query_plan, joined_plan->getCurrentDataStream().header, join_clause.key_names_left, right_set, ports_state, false); + add_filter_by_set(*joined_plan, query_plan.getCurrentDataStream().header, join_clause.key_names_right, left_set, ports_state, true); } add_sorting(query_plan, join_clause.key_names_left, false); diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index 48930ca9e51..9a7855a3635 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include "Processors/Port.h" namespace DB @@ -170,12 +172,21 @@ IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_in return Status::PortFull; } - -NotifyProcessor::NotifyProcessor(const Block & header, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)) - , aux_in_port(Block(), this) - , aux_out_port(Block(), this) +static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) { + InputPorts res(num_ports, header); + res.emplace_back(last_header); + return res; +} + +NotifyProcessor::NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_) + : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) + , aux_in_port(inputs.back()) + , aux_out_port(outputs.back()) + , sync_state(sync_state_) + , idx(sync_state->idx++) +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: idx {}", __FILE__, __LINE__, getDescription(), idx); port_pairs.resize(num_ports); auto input_it = inputs.begin(); @@ -188,7 +199,6 @@ NotifyProcessor::NotifyProcessor(const Block & header, size_t num_ports) port_pairs[i].output_port = &*output_it; ++output_it; } - } void NotifyProcessor::finishPair(PortsPair & pair) @@ -234,16 +244,38 @@ bool NotifyProcessor::processPair(PortsPair & pair) return true; } +bool NotifyProcessor::isPairsFinished() const +{ + return num_finished_pairs == port_pairs.size(); +} + IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { + if (isPairsFinished()) + return Status::Finished; + bool need_data = false; - for (const auto & output_number : updated_outputs) - need_data = processPair(port_pairs[output_number]) || need_data; - for (const auto & input_number : updated_inputs) - need_data = processPair(port_pairs[input_number]) || need_data; + UNUSED(updated_inputs); + UNUSED(updated_outputs); - if (num_finished_pairs == port_pairs.size()) + // for (const auto & output_number : updated_outputs) + for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) + { + if (output_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[output_number]) || need_data; + } + + // for (const auto & input_number : updated_inputs) + for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) + { + if (input_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[input_number]) || need_data; + } + + if (isPairsFinished()) return Status::Finished; if (need_data) @@ -252,44 +284,102 @@ IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & upda return Status::PortFull; } +void NotifyProcessor::work() +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} work {}", __FILE__, __LINE__, getDescription()); +} + +bool NotifyProcessor::sendPing() +{ + if (aux_out_port.canPush()) + { + Chunk chunk(aux_out_port.getHeader().cloneEmpty().getColumns(), 0); + aux_out_port.push(std::move(chunk)); + is_send = true; + aux_out_port.finish(); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing OK {} ({})", __FILE__, __LINE__, idx, log()); + return true; + } + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing NA {} ({})", __FILE__, __LINE__, idx, log()); + return false; +} + +bool NotifyProcessor::recievePing() +{ + if (aux_in_port.hasData()) + { + aux_in_port.pull(); + is_recieved = true; + aux_in_port.close(); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing OK {} ({})", __FILE__, __LINE__, idx, log()); + return true; + } + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing NA {} ({})", __FILE__, __LINE__, idx, log()); + return false; +} + + IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { - auto status = processRegularPorts(updated_inputs, updated_outputs); - if (status != Status::Ready) - return status; - - if (ready == AuxPortState::NotInitialized && isReady()) - ready = AuxPortState::Triggered; - - if (ready == AuxPortState::Triggered) - { - if (aux_out_port.canPush()) - { - aux_out_port.push({}); - ready = AuxPortState::Finished; - return Status::Ready; - } - return Status::PortFull; - } - - if (waiting == AuxPortState::NotInitialized && isWaiting()) + if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} set_needed_once {}: {}", __FILE__, __LINE__, getDescription(), idx); + set_needed_once = true; aux_in_port.setNeeded(); - waiting = AuxPortState::Triggered; } - if (waiting == AuxPortState::Triggered) + if (idx == 0 || is_send) { - if (aux_in_port.hasData()) + if (!is_recieved) { - aux_in_port.pull(true); - waiting = AuxPortState::Finished; - return Status::Ready; + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } } - return Status::PortFull; } - return Status::Ready; + if (idx == 1 || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + + auto status = processRegularPorts(updated_inputs, updated_outputs); + if (status == Status::Finished) + { + if (idx == 0 || is_send) + { + if (!is_recieved) + { + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } + } + } + + if (idx == 1 || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + } + if (status == Status::PortFull) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); + } + return status; } std::pair NotifyProcessor::getAuxPorts() diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index a5314ca36a1..8b9c44bad29 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -2,6 +2,8 @@ #include #include #include +#include + namespace DB { @@ -45,28 +47,29 @@ private: class NotifyProcessor : public IProcessor { public: - NotifyProcessor(const Block & header, size_t num_ports); + struct State + { + std::atomic_bool waiting = false; + std::atomic_bool can_push = false; + std::atomic_size_t idx = 0; + }; + using StatePtr = std::shared_ptr; + + NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_); String getName() const override { return "NotifyProcessor"; } Status prepare(const PortNumbers &, const PortNumbers &) override; + void work() override; std::pair getAuxPorts(); - virtual bool isReady() const { return true; } - virtual bool isWaiting() const { return false; } + + virtual bool canSend() const = 0; virtual void dataCallback(const Chunk & chunk) { UNUSED(chunk); } -private: - - enum class AuxPortState - { - NotInitialized, - Triggered, - Finished, - }; - +protected: struct PortsPair { InputPort * input_port = nullptr; @@ -74,6 +77,11 @@ private: bool is_finished = false; }; + bool sendPing(); + bool recievePing(); + virtual String log() = 0; + + bool isPairsFinished() const; bool processPair(PortsPair & pair); void finishPair(PortsPair & pair); Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); @@ -81,36 +89,49 @@ private: std::vector port_pairs; size_t num_finished_pairs = 0; - InputPort aux_in_port; - OutputPort aux_out_port; + InputPort & aux_in_port; + OutputPort & aux_out_port; - AuxPortState ready = AuxPortState::NotInitialized; - AuxPortState waiting = AuxPortState::NotInitialized; + bool is_send = false; + bool is_recieved = false; + + bool set_needed_once = false; + StatePtr sync_state; + + size_t idx; }; class NotifyProcessor2 : public NotifyProcessor { public: - using NotifyProcessor::NotifyProcessor; - - bool isReady() const override + NotifyProcessor2(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, NotifyProcessor::StatePtr sync_state_) + : NotifyProcessor(header, aux_header, num_ports, sync_state_) + , size(size_) { - return data_consumed > 10_MiB; } - bool isWaiting() const override + bool canSend() const override { - return data_consumed < 10_MiB; + return isPairsFinished() || data_consumed > size; } + void dataCallback(const Chunk & chunk) override { - data_consumed += chunk.allocatedBytes(); + data_consumed += chunk.getNumRows(); + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: data_consumed {}", __FILE__, __LINE__, getDescription(), data_consumed); + } + + String log() override + { + return fmt::format("data {} / {} = {:.2f}", data_consumed, size, data_consumed / float(size)); } private: size_t data_consumed = 0; + + size_t size; }; } diff --git a/src/Processors/Port.cpp b/src/Processors/Port.cpp index a856dbcea3b..79532dd4d6c 100644 --- a/src/Processors/Port.cpp +++ b/src/Processors/Port.cpp @@ -8,16 +8,16 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void connect(OutputPort & output, InputPort & input) +void connect(OutputPort & output, InputPort & input, bool reconnect) { - if (input.state) + if (!reconnect && input.state) throw Exception(ErrorCodes::LOGICAL_ERROR, "Port is already connected, (header: [{}])", input.header.dumpStructure()); - if (output.state) + if (!reconnect && output.state) throw Exception(ErrorCodes::LOGICAL_ERROR, "Port is already connected, (header: [{}])", output.header.dumpStructure()); - auto out_name = output.getProcessor().getName(); - auto in_name = input.getProcessor().getName(); + auto out_name = output.processor ? output.getProcessor().getName() : "null"; + auto in_name = input.processor ? input.getProcessor().getName() : "null"; assertCompatibleHeader(output.getHeader(), input.getHeader(), fmt::format("function connect between {} and {}", out_name, in_name)); diff --git a/src/Processors/Port.h b/src/Processors/Port.h index e3fb0e3e342..9163402f600 100644 --- a/src/Processors/Port.h +++ b/src/Processors/Port.h @@ -25,7 +25,7 @@ namespace ErrorCodes class Port { - friend void connect(OutputPort &, InputPort &); + friend void connect(OutputPort &, InputPort &, bool); friend class IProcessor; public: @@ -267,7 +267,7 @@ protected: /// * You can pull only if port hasData(). class InputPort : public Port { - friend void connect(OutputPort &, InputPort &); + friend void connect(OutputPort &, InputPort &, bool); private: OutputPort * output_port = nullptr; @@ -390,7 +390,7 @@ public: /// * You can push only if port doesn't hasData(). class OutputPort : public Port { - friend void connect(OutputPort &, InputPort &); + friend void connect(OutputPort &, InputPort &, bool); private: InputPort * input_port = nullptr; @@ -483,6 +483,6 @@ using InputPorts = std::list; using OutputPorts = std::list; -void connect(OutputPort & output, InputPort & input); +void connect(OutputPort & output, InputPort & input, bool reconnect = false); } diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index 0bb363fc17d..d7d25071d1d 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -57,18 +57,17 @@ void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, { UNUSED(settings); size_t num_streams = pipeline.getNumStreams(); + // pipeline.resize(1); - pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - auto res = std::make_shared(header, column_names, set); + auto res = std::make_shared(header, column_names, num_streams, set); res->setDescription(this->getStepDescription()); return res; }); - pipeline.resize(num_streams); + // pipeline.resize(num_streams); } void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const @@ -93,10 +92,13 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Block & rhs_input_stream_header_, + const Names & column_names_, size_t buffer_size_, SetWithStatePtr set_, PortsStatePtr ports_state_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) , column_names(column_names_) + , buffer_size(buffer_size_) + , rhs_input_stream_header(rhs_input_stream_header_.cloneEmpty()) , set(set_) , ports_state(ports_state_) { @@ -105,34 +107,61 @@ FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream } -static void connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs) +static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) { auto input_it = inputs.begin(); - for (auto & port : ports) + for (size_t i = 0; i < num_ports; ++i) { - connect(*port, *input_it); + connect(*ports[i], *input_it); input_it++; } - assert(input_it == inputs.end()); + return input_it; } void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { UNUSED(settings); + UNUSED(connectAllInputs); + + UNUSED(buffer_size); Block input_header = pipeline.getHeader(); pipeline.transform([&input_header, this](OutputPortRawPtrs ports) { - size_t num_streams = ports.size(); + Processors transforms; - auto notifier = std::make_shared(input_header, num_streams); + size_t num_ports = ports.size(); - connectAllInputs(ports, notifier->getInputs()); - ports_state->tryConnectPorts(notifier->getAuxPorts()); + auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, buffer_size, ports_state->sync_state); + notifier->setDescription(getStepDescription()); - return Processors{notifier}; + auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); + assert(&*input_it == notifier->getAuxPorts().first); + input_it++; + assert(input_it == notifier->getInputs().end()); + + ports_state->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {}", __FILE__, __LINE__, + notifier->getAuxPorts().first->isConnected(), notifier->getAuxPorts().second->isConnected()); + + auto & outputs = notifier->getOutputs(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < outputs.size() - 1; ++i) + { + auto & port = *output_it++; + auto transform = std::make_shared(port.getHeader(), column_names, set); + transform->setDescription(this->getStepDescription()); + connect(port, transform->getInputPort()); + transforms.emplace_back(std::move(transform)); + } + output_it++; + assert(output_it == outputs.end()); + transforms.emplace_back(std::move(notifier)); + + return transforms; }, /* check_ports= */ false); + /* pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) @@ -141,6 +170,7 @@ void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, res->setDescription(this->getStepDescription()); return res; }); + */ } void FilterBySetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 2a85d26836d..787e14c0e0e 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB @@ -37,34 +38,53 @@ public: class PortsState : public boost::noncopyable { public: + std::shared_ptr sync_state; + + explicit PortsState() + : sync_state(std::make_shared()) + { + } + using PortPair = std::pair; /// Remember ports passed on the first call and connect with ones from second call. - bool tryConnectPorts(PortPair rhs_ports) + bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} {} / {} {}", __FILE__, __LINE__, + bool(input_port), input_port ? input_port->isConnected() : false, + bool(output_port), output_port ? output_port->isConnected() : false); std::lock_guard lock(mux); if (input_port || output_port) { assert(input_port && output_port); + assert(!input_port->isConnected()); connect(*rhs_ports.second, *input_port); - connect(*output_port, *rhs_ports.first); + connect(*output_port, *rhs_ports.first, /* reconnect= */ true); return true; } std::tie(input_port, output_port) = rhs_ports; + assert(input_port && output_port); + assert(!input_port->isConnected() && !output_port->isConnected()); + + dummy_input_port = std::make_unique(output_port->getHeader(), proc); + connect(*output_port, *dummy_input_port); return false; } - private: std::mutex mux; InputPort * input_port = nullptr; OutputPort * output_port = nullptr; + + std::unique_ptr dummy_input_port; }; using PortsStatePtr = std::shared_ptr; FilterBySetOnTheFlyStep( const DataStream & input_stream_, + const Block & rhs_input_stream_header_, const Names & column_names_, + size_t buffer_size_, SetWithStatePtr set_, PortsStatePtr ports_state_); @@ -79,6 +99,10 @@ private: Names column_names; + size_t buffer_size; + + Block rhs_input_stream_header; + SetWithStatePtr set; PortsStatePtr ports_state; }; diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index b3a3410dd9e..1f9282d8baa 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -52,10 +53,12 @@ std::string formatBytesHumanReadable(size_t bytes) } -CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) +CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( + const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) + , num_streams(num_streams_) , set(set_) , log(&Poco::Logger::get(getName())) { @@ -65,14 +68,18 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); - if (status == Status::Finished && set) + if (status == Status::Finished && set && set->state == SetWithState::State::Creating) { if (input.isFinished()) { - set->finishInsert(); - set->state = SetWithState::State::Finished; - LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + set->finished_count++; + if (set->finished_count == num_streams) + { + set->finishInsert(); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + } } else { @@ -82,29 +89,38 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); } + } + if (status == Status::Finished && set && set->state != SetWithState::State::Creating) /// Release pointer to make it possible destroy it by consumer set.reset(); - } return status; } void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) { - if (!set) + if (!set || set->state != SetWithState::State::Creating) + { + if (set) + set.reset(); return; + } if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); + size_t prev_size = set->getTotalByteCount(); bool limit_exceeded = !set->insertFromBlock(key_columns); if (limit_exceeded) { - LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {}", - getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); + auto prev_state = set->state.exchange(SetWithState::State::Suspended); + if (prev_state == SetWithState::State::Creating) + { + LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {} ({} -> {} bytes)", + getDescription(), formatBytesHumanReadable(set->getTotalByteCount()), prev_size, set->getTotalByteCount()); + } // TODO(@vdimir): set->clear() ? - set->state = SetWithState::State::Suspended; set.reset(); } } @@ -150,8 +166,9 @@ void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) { stat.consumed_rows += chunk.getNumRows(); stat.result_rows += chunk.getNumRows(); - bool can_filter = set && set->state == SetWithState::State::Finished; + bool can_filter = set && set->state == SetWithState::State::Finished; + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {} / {}", __FILE__, __LINE__, set->finished_count.load(), set->state.load(), set->getTotalRowCount()); if (!can_filter) stat.consumed_rows_before_set += chunk.getNumRows(); diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h index ea91a96818a..072095cc3db 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h @@ -1,4 +1,6 @@ #pragma once +#include +#include #include #include #include @@ -21,6 +23,7 @@ public: }; std::atomic state = State::Creating; + std::atomic_size_t finished_count = 0; }; using SetWithStatePtr = std::shared_ptr; @@ -35,7 +38,7 @@ class CreatingSetsOnTheFlyTransform : public ISimpleTransform { public: explicit CreatingSetsOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetWithStatePtr set_); + const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_); String getName() const override { return "CreatingSetsOnTheFlyTransform"; } @@ -47,6 +50,8 @@ private: Names column_names; std::vector key_column_indices; + size_t num_streams; + /// Set to fill SetWithStatePtr set; @@ -56,7 +61,7 @@ private: /* * Filter the input chunk by the set. - * When set building is not comleted, just return the source data. + * When set building is not completed, just return the source data. */ class FilterBySetOnTheFlyTransform : public ISimpleTransform { diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 224dd9e5145..291739079a2 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -784,6 +784,9 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) for (const auto & port : output_ports) { + if (!check_ports) + break; + if (!port->isConnected()) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -809,7 +812,7 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) processor->getName()); const auto * connected_processor = &port.getOutputPort().getProcessor(); - if (!set.contains(connected_processor)) + if (check_ports && !set.contains(connected_processor)) throw Exception( ErrorCodes::LOGICAL_ERROR, "Transformation of Pipe is not valid because processor {} has input port which is connected with unknown processor {}", @@ -826,7 +829,7 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) } const auto * connected_processor = &port.getInputPort().getProcessor(); - if (!set.contains(connected_processor)) + if (check_ports && !set.contains(connected_processor)) throw Exception( ErrorCodes::LOGICAL_ERROR, "Transformation of Pipe is not valid because processor {} has output port which is connected with unknown processor {}", From 51a51694d6030fc2ac289be992638133fc89d8ba Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:08:25 +0000 Subject: [PATCH 164/582] Create sets for joins: better code --- src/Interpreters/InterpreterSelectQuery.cpp | 44 ++-- src/Processors/DelayedPortsProcessor.cpp | 49 ++-- src/Processors/DelayedPortsProcessor.h | 75 +++--- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 243 +++++++++--------- .../QueryPlan/CreatingSetOnTheFlyStep.h | 98 ++----- .../Optimizations/filterPushDown.cpp | 3 - 6 files changed, 223 insertions(+), 289 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f16c81e93b1..212d112d503 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1423,7 +1423,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data, settings.optimize_sorting_by_input_stream_properties); - sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", is_right ? "right" : "left")); + sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", join_pos)); plan.addStep(std::move(sorting_step)); }; - auto add_create_set = [&settings](QueryPlan & plan, const Names & key_names, bool is_right) + auto crosswise_connection = CreatingSetOnTheFlyStep::createCrossConnection(); + auto add_create_set = [&settings, crosswise_connection](QueryPlan & plan, const DataStream & rhs_data_stream, const Names & key_names, JoinTableSide join_pos) { - SizeLimits size_limits(settings.max_rows_in_set_to_optimize_join, 0, OverflowMode::BREAK); - auto creating_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, size_limits); - creating_set_step->setStepDescription(fmt::format("Create set for {} stream", is_right ? "right" : "left")); - auto set = creating_set_step->getSet(); - plan.addStep(std::move(creating_set_step)); - return set; - }; + size_t max_rows = std::max(100, settings.max_rows_in_set_to_optimize_join); - auto add_filter_by_set = [&settings](QueryPlan & plan, const Block & rhs_header, const Names & key_names, auto set, auto ports_state, bool is_right) - { - auto filter_by_set_step = std::make_unique( - plan.getCurrentDataStream(), rhs_header, key_names, settings.max_rows_in_set_to_optimize_join, set, ports_state); - filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); - plan.addStep(std::move(filter_by_set_step)); + auto creating_set_step = std::make_unique( + plan.getCurrentDataStream(), rhs_data_stream, key_names, max_rows, crosswise_connection, join_pos); + creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); + + auto * step_raw_ptr = creating_set_step.get(); + plan.addStep(std::move(creating_set_step)); + return step_raw_ptr; }; if (expressions.join->pipelineType() == JoinPipelineType::YShaped) { const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); - if (settings.max_rows_in_set_to_optimize_join > 0) + if (settings.max_rows_in_set_to_optimize_join >= 0) { - auto left_set = add_create_set(query_plan, join_clause.key_names_left, false); - auto right_set = add_create_set(*joined_plan, join_clause.key_names_right, true); - - auto ports_state = std::make_shared(); - add_filter_by_set(query_plan, joined_plan->getCurrentDataStream().header, join_clause.key_names_left, right_set, ports_state, false); - add_filter_by_set(*joined_plan, query_plan.getCurrentDataStream().header, join_clause.key_names_right, left_set, ports_state, true); + auto * left_set = add_create_set(query_plan, joined_plan->getCurrentDataStream(), join_clause.key_names_left, JoinTableSide::Left); + auto * right_set = add_create_set(*joined_plan, query_plan.getCurrentDataStream(), join_clause.key_names_right, JoinTableSide::Right); + left_set->setFiltering(right_set->getSet()); + right_set->setFiltering(left_set->getSet()); } - add_sorting(query_plan, join_clause.key_names_left, false); - add_sorting(*joined_plan, join_clause.key_names_right, true); + add_sorting(query_plan, join_clause.key_names_left, JoinTableSide::Left); + add_sorting(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); } QueryPlanStepPtr join_step = std::make_unique( diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index 9a7855a3635..a4b903ec62b 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -179,14 +179,14 @@ static InputPorts createPortsList(const Block & header, const Block & last_heade return res; } -NotifyProcessor::NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_) +PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) , aux_in_port(inputs.back()) , aux_out_port(outputs.back()) - , sync_state(sync_state_) - , idx(sync_state->idx++) + , order(order_) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: idx {}", __FILE__, __LINE__, getDescription(), idx); + assert(order == First || order == Second); + port_pairs.resize(num_ports); auto input_it = inputs.begin(); @@ -201,7 +201,7 @@ NotifyProcessor::NotifyProcessor(const Block & header, const Block & aux_header, } } -void NotifyProcessor::finishPair(PortsPair & pair) +void PingPongProcessor::finishPair(PortsPair & pair) { if (!pair.is_finished) { @@ -213,7 +213,7 @@ void NotifyProcessor::finishPair(PortsPair & pair) } } -bool NotifyProcessor::processPair(PortsPair & pair) +bool PingPongProcessor::processPair(PortsPair & pair) { if (pair.output_port->isFinished()) { @@ -237,19 +237,19 @@ bool NotifyProcessor::processPair(PortsPair & pair) if (pair.input_port->hasData()) { Chunk chunk = pair.input_port->pull(true); - dataCallback(chunk); + ready_to_send = isReady(chunk) || ready_to_send; pair.output_port->push(std::move(chunk)); } return true; } -bool NotifyProcessor::isPairsFinished() const +bool PingPongProcessor::isPairsFinished() const { return num_finished_pairs == port_pairs.size(); } -IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { if (isPairsFinished()) return Status::Finished; @@ -284,12 +284,7 @@ IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & upda return Status::PortFull; } -void NotifyProcessor::work() -{ - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} work {}", __FILE__, __LINE__, getDescription()); -} - -bool NotifyProcessor::sendPing() +bool PingPongProcessor::sendPing() { if (aux_out_port.canPush()) { @@ -297,38 +292,37 @@ bool NotifyProcessor::sendPing() aux_out_port.push(std::move(chunk)); is_send = true; aux_out_port.finish(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing OK {} ({})", __FILE__, __LINE__, idx, log()); return true; } - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing NA {} ({})", __FILE__, __LINE__, idx, log()); return false; } -bool NotifyProcessor::recievePing() +bool PingPongProcessor::recievePing() { if (aux_in_port.hasData()) { aux_in_port.pull(); is_recieved = true; aux_in_port.close(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing OK {} ({})", __FILE__, __LINE__, idx, log()); return true; } - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing NA {} ({})", __FILE__, __LINE__, idx, log()); return false; } +bool PingPongProcessor::canSend() const +{ + return !is_send && (ready_to_send || isPairsFinished()); +} -IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} set_needed_once {}: {}", __FILE__, __LINE__, getDescription(), idx); set_needed_once = true; aux_in_port.setNeeded(); } - if (idx == 0 || is_send) + if (order == First || is_send) { if (!is_recieved) { @@ -340,7 +334,7 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, } } - if (idx == 1 || is_recieved) + if (order == Second || is_recieved) { if (!is_send && canSend()) { @@ -353,7 +347,7 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, auto status = processRegularPorts(updated_inputs, updated_outputs); if (status == Status::Finished) { - if (idx == 0 || is_send) + if (order == First || is_send) { if (!is_recieved) { @@ -365,7 +359,7 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, } } - if (idx == 1 || is_recieved) + if (order == Second || is_recieved) { if (!is_send && canSend()) { @@ -382,10 +376,9 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, return status; } -std::pair NotifyProcessor::getAuxPorts() +std::pair PingPongProcessor::getAuxPorts() { return std::make_pair(&aux_in_port, &aux_out_port); } - } diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index 8b9c44bad29..b36fa7c3877 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -44,30 +44,47 @@ private: }; -class NotifyProcessor : public IProcessor +/* + * Processor with N inputs and N outputs. Moves data from i-th input to i-th output as is. + * It has a pair of auxiliary ports to notify another instance by sending empty chunk after some condition holds. + * You should use this processor in pair of instances and connect auxiliary ports crosswise. + * + * ╭─┴───┴───┴───┴───┴─╮ ╭─┴───┴───┴───┴───┴─╮ + * │ ├─ aux ⟶│ │ + * │ PingPongProcessor │ │ PingPongProcessor │ + * │ │⟵ aux ─┤ │ + * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ + * + * One of the processors starts processing data, and another waits for notification. + * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. + * After that, the second one also processes data until `isReady`, then send a notification back to the first one. + * After this roundtrip, processors bypass data from regular inputs to outputs. + */ +class PingPongProcessor : public IProcessor { public: - struct State + enum class Order : uint8_t { - std::atomic_bool waiting = false; - std::atomic_bool can_push = false; - std::atomic_size_t idx = 0; + /// Processor that starts processing data. + First, + /// Processor that waits for notification. + Second, }; - using StatePtr = std::shared_ptr; - NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_); + using enum Order; - String getName() const override { return "NotifyProcessor"; } + /// The `aux_header` is a header from another instance of procssor. + /// It's required because all outputs should have the same structure. + /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. + PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); + + String getName() const override { return "PingPongProcessor"; } Status prepare(const PortNumbers &, const PortNumbers &) override; - void work() override; std::pair getAuxPorts(); - - virtual bool canSend() const = 0; - - virtual void dataCallback(const Chunk & chunk) { UNUSED(chunk); } + virtual bool isReady(const Chunk & chunk) = 0; protected: struct PortsPair @@ -79,7 +96,7 @@ protected: bool sendPing(); bool recievePing(); - virtual String log() = 0; + bool canSend() const; bool isPairsFinished() const; bool processPair(PortsPair & pair); @@ -95,37 +112,27 @@ protected: bool is_send = false; bool is_recieved = false; - bool set_needed_once = false; - StatePtr sync_state; + bool ready_to_send = false; - size_t idx; + bool set_needed_once = false; + + Order order; }; - -class NotifyProcessor2 : public NotifyProcessor +/// Reads first N rows from two streams evenly. +class ReadHeadBalancedProceesor : public PingPongProcessor { public: - NotifyProcessor2(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, NotifyProcessor::StatePtr sync_state_) - : NotifyProcessor(header, aux_header, num_ports, sync_state_) + ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) + : PingPongProcessor(header, aux_header, num_ports, order_) , size(size_) { } - bool canSend() const override - { - return isPairsFinished() || data_consumed > size; - } - - - void dataCallback(const Chunk & chunk) override + bool isReady(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: data_consumed {}", __FILE__, __LINE__, getDescription(), data_consumed); - } - - String log() override - { - return fmt::format("data {} / {} = {:.2f}", data_consumed, size, data_consumed / float(size)); + return data_consumed > size; } private: diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index d7d25071d1d..a56b53a6559 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -17,6 +17,23 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +namespace +{ + +InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) +{ + auto input_it = inputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + connect(*ports[i], *input_it); + input_it++; + } + return input_it; +} + +} + static ITransformingStep::Traits getTraits(bool is_filter) { return ITransformingStep::Traits @@ -33,41 +50,126 @@ static ITransformingStep::Traits getTraits(bool is_filter) }; } -CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, const SizeLimits & size_limits) + +class CreatingSetOnTheFlyStep::CrosswiseConnection : public boost::noncopyable +{ +public: + using PortPair = std::pair; + + /// Remember ports passed on the first call and connect with ones from second call. + bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) + { + std::lock_guard lock(mux); + if (input_port || output_port) + { + assert(input_port && output_port); + assert(!input_port->isConnected()); + connect(*rhs_ports.second, *input_port); + connect(*output_port, *rhs_ports.first, /* reconnect= */ true); + return true; + } + std::tie(input_port, output_port) = rhs_ports; + assert(input_port && output_port); + assert(!input_port->isConnected() && !output_port->isConnected()); + + dummy_input_port = std::make_unique(output_port->getHeader(), proc); + connect(*output_port, *dummy_input_port); + return false; + } + +private: + std::mutex mux; + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + + std::unique_ptr dummy_input_port; +}; + +CreatingSetOnTheFlyStep::CrosswiseConnectionPtr CreatingSetOnTheFlyStep::createCrossConnection() +{ + return std::make_shared(); +} + +CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep( + const DataStream & input_stream_, + const DataStream & rhs_input_stream_, + const Names & column_names_, + size_t max_rows_, + CrosswiseConnectionPtr crosswise_connection_, + JoinTableSide position_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) , column_names(column_names_) + , max_rows(max_rows_) + , rhs_input_stream_header(rhs_input_stream_.header) + , own_set(std::make_shared(SizeLimits(max_rows, 0, OverflowMode::BREAK), false, true)) + , filtering_set(nullptr) + , crosswise_connection(crosswise_connection_) + , position(position_) { + if (crosswise_connection == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Crosswise connection is not initialized"); + if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); - set = std::make_shared(size_limits, false, true); - - { - ColumnsWithTypeAndName header; - for (const auto & name : column_names) - { - ColumnWithTypeAndName column = input_streams[0].header.getByName(name); - header.emplace_back(column); - } - set->setHeader(header); - } + ColumnsWithTypeAndName header; + for (const auto & name : column_names) + header.emplace_back(input_streams[0].header.getByName(name)); + own_set->setHeader(header); } -void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) +void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - UNUSED(settings); size_t num_streams = pipeline.getNumStreams(); - // pipeline.resize(1); - - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([this, num_streams](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - auto res = std::make_shared(header, column_names, num_streams, set); + auto res = std::make_shared(header, column_names, num_streams, own_set); res->setDescription(this->getStepDescription()); return res; }); - // pipeline.resize(num_streams); + + if (!filtering_set) + { + LOG_DEBUG(log, "filtering_set is null"); + return; + } + + Block input_header = pipeline.getHeader(); + pipeline.transform([&input_header, this](OutputPortRawPtrs ports) + { + Processors transforms; + + size_t num_ports = ports.size(); + + auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; + auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows, idx); + notifier->setDescription(getStepDescription()); + + auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); + assert(&*input_it == notifier->getAuxPorts().first); + input_it++; + assert(input_it == notifier->getInputs().end()); + + crosswise_connection->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); + + auto & outputs = notifier->getOutputs(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < outputs.size() - 1; ++i) + { + auto & port = *output_it++; + auto transform = std::make_shared(port.getHeader(), column_names, filtering_set); + transform->setDescription(this->getStepDescription()); + connect(port, transform->getInputPort()); + transforms.emplace_back(std::move(transform)); + } + output_it++; + assert(output_it == outputs.end()); + transforms.emplace_back(std::move(notifier)); + + return transforms; + }, /* check_ports= */ false); } void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const @@ -92,107 +194,4 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Block & rhs_input_stream_header_, - const Names & column_names_, size_t buffer_size_, - SetWithStatePtr set_, PortsStatePtr ports_state_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) - , column_names(column_names_) - , buffer_size(buffer_size_) - , rhs_input_stream_header(rhs_input_stream_header_.cloneEmpty()) - , set(set_) - , ports_state(ports_state_) -{ - if (input_streams.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); -} - - -static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) -{ - auto input_it = inputs.begin(); - for (size_t i = 0; i < num_ports; ++i) - { - connect(*ports[i], *input_it); - input_it++; - } - return input_it; -} - -void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) -{ - UNUSED(settings); - UNUSED(connectAllInputs); - - UNUSED(buffer_size); - - Block input_header = pipeline.getHeader(); - pipeline.transform([&input_header, this](OutputPortRawPtrs ports) - { - Processors transforms; - - size_t num_ports = ports.size(); - - auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, buffer_size, ports_state->sync_state); - notifier->setDescription(getStepDescription()); - - auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); - assert(&*input_it == notifier->getAuxPorts().first); - input_it++; - assert(input_it == notifier->getInputs().end()); - - ports_state->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {}", __FILE__, __LINE__, - notifier->getAuxPorts().first->isConnected(), notifier->getAuxPorts().second->isConnected()); - - auto & outputs = notifier->getOutputs(); - auto output_it = outputs.begin(); - for (size_t i = 0; i < outputs.size() - 1; ++i) - { - auto & port = *output_it++; - auto transform = std::make_shared(port.getHeader(), column_names, set); - transform->setDescription(this->getStepDescription()); - connect(port, transform->getInputPort()); - transforms.emplace_back(std::move(transform)); - } - output_it++; - assert(output_it == outputs.end()); - transforms.emplace_back(std::move(notifier)); - - return transforms; - }, /* check_ports= */ false); - - /* - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - auto res = std::make_shared(header, column_names, set); - res->setDescription(this->getStepDescription()); - return res; - }); - */ -} - -void FilterBySetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add(getName(), true); -} - -void FilterBySetOnTheFlyStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << getName(); - - settings.out << '\n'; -} - -void FilterBySetOnTheFlyStep::updateOutputStream() -{ - if (input_streams.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); - - output_stream = input_streams[0]; -} - - } diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 787e14c0e0e..67f31a97009 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -10,10 +10,18 @@ namespace DB class CreatingSetOnTheFlyStep : public ITransformingStep { public: - explicit CreatingSetOnTheFlyStep( + /// Two instances of step need some shared state to connect processors crosswise + class CrosswiseConnection; + using CrosswiseConnectionPtr = std::shared_ptr; + static CrosswiseConnectionPtr createCrossConnection(); + + CreatingSetOnTheFlyStep( const DataStream & input_stream_, + const DataStream & rhs_input_stream_, const Names & column_names_, - const SizeLimits & size_limits = {}); + size_t max_rows_, + CrosswiseConnectionPtr crosswise_connection_, + JoinTableSide position_); String getName() const override { return "CreatingSetsOnTheFly"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; @@ -21,90 +29,26 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - SetWithStatePtr getSet() const { return set; } - -private: - void updateOutputStream() override; - - Names column_names; - SetWithStatePtr set; -}; - - -class FilterBySetOnTheFlyStep : public ITransformingStep -{ -public: - - class PortsState : public boost::noncopyable - { - public: - std::shared_ptr sync_state; - - explicit PortsState() - : sync_state(std::make_shared()) - { - } - - using PortPair = std::pair; - - /// Remember ports passed on the first call and connect with ones from second call. - bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} {} / {} {}", __FILE__, __LINE__, - bool(input_port), input_port ? input_port->isConnected() : false, - bool(output_port), output_port ? output_port->isConnected() : false); - std::lock_guard lock(mux); - if (input_port || output_port) - { - assert(input_port && output_port); - assert(!input_port->isConnected()); - connect(*rhs_ports.second, *input_port); - connect(*output_port, *rhs_ports.first, /* reconnect= */ true); - return true; - } - std::tie(input_port, output_port) = rhs_ports; - assert(input_port && output_port); - assert(!input_port->isConnected() && !output_port->isConnected()); - - dummy_input_port = std::make_unique(output_port->getHeader(), proc); - connect(*output_port, *dummy_input_port); - return false; - } - private: - std::mutex mux; - InputPort * input_port = nullptr; - OutputPort * output_port = nullptr; - - std::unique_ptr dummy_input_port; - }; - - using PortsStatePtr = std::shared_ptr; - - FilterBySetOnTheFlyStep( - const DataStream & input_stream_, - const Block & rhs_input_stream_header_, - const Names & column_names_, - size_t buffer_size_, - SetWithStatePtr set_, - PortsStatePtr ports_state_); - - String getName() const override { return "FilterBySetOnTheFly"; } - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; + SetWithStatePtr getSet() const { return own_set; } + void setFiltering(SetWithStatePtr filtering_set_) { filtering_set = filtering_set_; } private: void updateOutputStream() override; Names column_names; - size_t buffer_size; + size_t max_rows; Block rhs_input_stream_header; - SetWithStatePtr set; - PortsStatePtr ports_state; + SetWithStatePtr own_set; + SetWithStatePtr filtering_set; + + CrosswiseConnectionPtr crosswise_connection; + + JoinTableSide position; + + Poco::Logger * log = &Poco::Logger::get("CreatingSetOnTheFlyStep"); }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 99312a17ffd..f60f8192938 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -336,9 +336,6 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; - if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) - return updated_steps; - if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; From 6ae15db3ad445dd6ea379948106f1b43c4472141 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:15:04 +0000 Subject: [PATCH 165/582] check join kind for before applying max_rows_in_set_to_optimize_join --- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 212d112d503..15ed0e7d493 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1449,6 +1449,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); auto creating_set_step = std::make_unique( @@ -1462,14 +1463,21 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

pipelineType() == JoinPipelineType::YShaped) { - const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); + const auto & table_join = expressions.join->getTableJoin(); + const auto & join_clause = table_join.getOnlyClause(); - if (settings.max_rows_in_set_to_optimize_join >= 0) + auto join_kind = table_join.kind(); + bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); + if (settings.max_rows_in_set_to_optimize_join >= 0 && kind_allows_filtering) { auto * left_set = add_create_set(query_plan, joined_plan->getCurrentDataStream(), join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(*joined_plan, query_plan.getCurrentDataStream(), join_clause.key_names_right, JoinTableSide::Right); - left_set->setFiltering(right_set->getSet()); - right_set->setFiltering(left_set->getSet()); + + if (isInnerOrLeft(join_kind)) + right_set->setFiltering(left_set->getSet()); + + if (isInnerOrRight(join_kind)) + left_set->setFiltering(right_set->getSet()); } add_sorting(query_plan, join_clause.key_names_left, JoinTableSide::Left); From e472e13c709ee2c72b54c3a4162aa8364ae371f9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:20:51 +0000 Subject: [PATCH 166/582] move PingPongProcessor/ReadHeadBalancedProceesor into separate file --- src/Processors/DelayedPortsProcessor.cpp | 211 ----------------- src/Processors/DelayedPortsProcessor.h | 102 --------- src/Processors/PingPongProcessor.cpp | 215 ++++++++++++++++++ src/Processors/PingPongProcessor.h | 108 +++++++++ .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 2 +- 5 files changed, 324 insertions(+), 314 deletions(-) create mode 100644 src/Processors/PingPongProcessor.cpp create mode 100644 src/Processors/PingPongProcessor.h diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index a4b903ec62b..24023529bca 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -1,8 +1,6 @@ #include #include -#include -#include "Processors/Port.h" namespace DB @@ -172,213 +170,4 @@ IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_in return Status::PortFull; } -static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) -{ - InputPorts res(num_ports, header); - res.emplace_back(last_header); - return res; -} - -PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) - : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) - , aux_in_port(inputs.back()) - , aux_out_port(outputs.back()) - , order(order_) -{ - assert(order == First || order == Second); - - port_pairs.resize(num_ports); - - auto input_it = inputs.begin(); - auto output_it = outputs.begin(); - for (size_t i = 0; i < num_ports; ++i) - { - port_pairs[i].input_port = &*input_it; - ++input_it; - - port_pairs[i].output_port = &*output_it; - ++output_it; - } -} - -void PingPongProcessor::finishPair(PortsPair & pair) -{ - if (!pair.is_finished) - { - pair.output_port->finish(); - pair.input_port->close(); - - pair.is_finished = true; - ++num_finished_pairs; - } -} - -bool PingPongProcessor::processPair(PortsPair & pair) -{ - if (pair.output_port->isFinished()) - { - finishPair(pair); - return false; - } - - if (pair.input_port->isFinished()) - { - finishPair(pair); - return false; - } - - if (!pair.output_port->canPush()) - { - pair.input_port->setNotNeeded(); - return false; - } - - pair.input_port->setNeeded(); - if (pair.input_port->hasData()) - { - Chunk chunk = pair.input_port->pull(true); - ready_to_send = isReady(chunk) || ready_to_send; - pair.output_port->push(std::move(chunk)); - } - - return true; -} - -bool PingPongProcessor::isPairsFinished() const -{ - return num_finished_pairs == port_pairs.size(); -} - -IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) -{ - if (isPairsFinished()) - return Status::Finished; - - bool need_data = false; - - UNUSED(updated_inputs); - UNUSED(updated_outputs); - - // for (const auto & output_number : updated_outputs) - for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) - { - if (output_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[output_number]) || need_data; - } - - // for (const auto & input_number : updated_inputs) - for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) - { - if (input_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[input_number]) || need_data; - } - - if (isPairsFinished()) - return Status::Finished; - - if (need_data) - return Status::NeedData; - - return Status::PortFull; -} - -bool PingPongProcessor::sendPing() -{ - if (aux_out_port.canPush()) - { - Chunk chunk(aux_out_port.getHeader().cloneEmpty().getColumns(), 0); - aux_out_port.push(std::move(chunk)); - is_send = true; - aux_out_port.finish(); - return true; - } - return false; -} - -bool PingPongProcessor::recievePing() -{ - if (aux_in_port.hasData()) - { - aux_in_port.pull(); - is_recieved = true; - aux_in_port.close(); - return true; - } - return false; -} - -bool PingPongProcessor::canSend() const -{ - return !is_send && (ready_to_send || isPairsFinished()); -} - -IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) -{ - if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) - { - set_needed_once = true; - aux_in_port.setNeeded(); - } - - if (order == First || is_send) - { - if (!is_recieved) - { - bool recieved = recievePing(); - if (!recieved) - { - return Status::NeedData; - } - } - } - - if (order == Second || is_recieved) - { - if (!is_send && canSend()) - { - bool sent = sendPing(); - if (!sent) - return Status::PortFull; - } - } - - auto status = processRegularPorts(updated_inputs, updated_outputs); - if (status == Status::Finished) - { - if (order == First || is_send) - { - if (!is_recieved) - { - bool recieved = recievePing(); - if (!recieved) - { - return Status::NeedData; - } - } - } - - if (order == Second || is_recieved) - { - if (!is_send && canSend()) - { - bool sent = sendPing(); - if (!sent) - return Status::PortFull; - } - } - } - if (status == Status::PortFull) - { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); - } - return status; -} - -std::pair PingPongProcessor::getAuxPorts() -{ - return std::make_pair(&aux_in_port, &aux_out_port); -} - } diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index b36fa7c3877..a6a9590e0c8 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -1,9 +1,5 @@ #pragma once #include -#include -#include -#include - namespace DB { @@ -43,102 +39,4 @@ private: void finishPair(PortsPair & pair); }; - -/* - * Processor with N inputs and N outputs. Moves data from i-th input to i-th output as is. - * It has a pair of auxiliary ports to notify another instance by sending empty chunk after some condition holds. - * You should use this processor in pair of instances and connect auxiliary ports crosswise. - * - * ╭─┴───┴───┴───┴───┴─╮ ╭─┴───┴───┴───┴───┴─╮ - * │ ├─ aux ⟶│ │ - * │ PingPongProcessor │ │ PingPongProcessor │ - * │ │⟵ aux ─┤ │ - * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ - * - * One of the processors starts processing data, and another waits for notification. - * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. - * After that, the second one also processes data until `isReady`, then send a notification back to the first one. - * After this roundtrip, processors bypass data from regular inputs to outputs. - */ -class PingPongProcessor : public IProcessor -{ -public: - enum class Order : uint8_t - { - /// Processor that starts processing data. - First, - /// Processor that waits for notification. - Second, - }; - - using enum Order; - - /// The `aux_header` is a header from another instance of procssor. - /// It's required because all outputs should have the same structure. - /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. - PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); - - String getName() const override { return "PingPongProcessor"; } - - Status prepare(const PortNumbers &, const PortNumbers &) override; - - std::pair getAuxPorts(); - - virtual bool isReady(const Chunk & chunk) = 0; - -protected: - struct PortsPair - { - InputPort * input_port = nullptr; - OutputPort * output_port = nullptr; - bool is_finished = false; - }; - - bool sendPing(); - bool recievePing(); - bool canSend() const; - - bool isPairsFinished() const; - bool processPair(PortsPair & pair); - void finishPair(PortsPair & pair); - Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); - - std::vector port_pairs; - size_t num_finished_pairs = 0; - - InputPort & aux_in_port; - OutputPort & aux_out_port; - - bool is_send = false; - bool is_recieved = false; - - bool ready_to_send = false; - - bool set_needed_once = false; - - Order order; -}; - -/// Reads first N rows from two streams evenly. -class ReadHeadBalancedProceesor : public PingPongProcessor -{ -public: - ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) - : PingPongProcessor(header, aux_header, num_ports, order_) - , size(size_) - { - } - - bool isReady(const Chunk & chunk) override - { - data_consumed += chunk.getNumRows(); - return data_consumed > size; - } - -private: - size_t data_consumed = 0; - - size_t size; -}; - } diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp new file mode 100644 index 00000000000..777d768f9a4 --- /dev/null +++ b/src/Processors/PingPongProcessor.cpp @@ -0,0 +1,215 @@ +#include + +namespace DB +{ + +static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) +{ + InputPorts res(num_ports, header); + res.emplace_back(last_header); + return res; +} + +PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) + : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) + , aux_in_port(inputs.back()) + , aux_out_port(outputs.back()) + , order(order_) +{ + assert(order == First || order == Second); + + port_pairs.resize(num_ports); + + auto input_it = inputs.begin(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + port_pairs[i].input_port = &*input_it; + ++input_it; + + port_pairs[i].output_port = &*output_it; + ++output_it; + } +} + +void PingPongProcessor::finishPair(PortsPair & pair) +{ + if (!pair.is_finished) + { + pair.output_port->finish(); + pair.input_port->close(); + + pair.is_finished = true; + ++num_finished_pairs; + } +} + +bool PingPongProcessor::processPair(PortsPair & pair) +{ + if (pair.output_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (pair.input_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (!pair.output_port->canPush()) + { + pair.input_port->setNotNeeded(); + return false; + } + + pair.input_port->setNeeded(); + if (pair.input_port->hasData()) + { + Chunk chunk = pair.input_port->pull(true); + ready_to_send = isReady(chunk) || ready_to_send; + pair.output_port->push(std::move(chunk)); + } + + return true; +} + +bool PingPongProcessor::isPairsFinished() const +{ + return num_finished_pairs == port_pairs.size(); +} + +IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + if (isPairsFinished()) + return Status::Finished; + + bool need_data = false; + + UNUSED(updated_inputs); + UNUSED(updated_outputs); + + // for (const auto & output_number : updated_outputs) + for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) + { + if (output_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[output_number]) || need_data; + } + + // for (const auto & input_number : updated_inputs) + for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) + { + if (input_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[input_number]) || need_data; + } + + if (isPairsFinished()) + return Status::Finished; + + if (need_data) + return Status::NeedData; + + return Status::PortFull; +} + +bool PingPongProcessor::sendPing() +{ + if (aux_out_port.canPush()) + { + Chunk chunk(aux_out_port.getHeader().cloneEmpty().getColumns(), 0); + aux_out_port.push(std::move(chunk)); + is_send = true; + aux_out_port.finish(); + return true; + } + return false; +} + +bool PingPongProcessor::recievePing() +{ + if (aux_in_port.hasData()) + { + aux_in_port.pull(); + is_recieved = true; + aux_in_port.close(); + return true; + } + return false; +} + +bool PingPongProcessor::canSend() const +{ + return !is_send && (ready_to_send || isPairsFinished()); +} + +IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) + { + set_needed_once = true; + aux_in_port.setNeeded(); + } + + if (order == First || is_send) + { + if (!is_recieved) + { + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } + } + } + + if (order == Second || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + + auto status = processRegularPorts(updated_inputs, updated_outputs); + if (status == Status::Finished) + { + if (order == First || is_send) + { + if (!is_recieved) + { + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } + } + } + + if (order == Second || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + } + if (status == Status::PortFull) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); + } + return status; +} + +std::pair PingPongProcessor::getAuxPorts() +{ + return std::make_pair(&aux_in_port, &aux_out_port); +} + +} diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h new file mode 100644 index 00000000000..09cdc6c0ec5 --- /dev/null +++ b/src/Processors/PingPongProcessor.h @@ -0,0 +1,108 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +/* + * Processor with N inputs and N outputs. Moves data from i-th input to i-th output as is. + * It has a pair of auxiliary ports to notify another instance by sending empty chunk after some condition holds. + * You should use this processor in pair of instances and connect auxiliary ports crosswise. + * + * ╭─┴───┴───┴───┴───┴─╮ ╭─┴───┴───┴───┴───┴─╮ + * │ ├─ aux ⟶│ │ + * │ PingPongProcessor │ │ PingPongProcessor │ + * │ │⟵ aux ─┤ │ + * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ + * + * One of the processors starts processing data, and another waits for notification. + * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. + * After that, the second one also processes data until `isReady`, then send a notification back to the first one. + * After this roundtrip, processors bypass data from regular inputs to outputs. + */ +class PingPongProcessor : public IProcessor +{ +public: + enum class Order : uint8_t + { + /// Processor that starts processing data. + First, + /// Processor that waits for notification. + Second, + }; + + using enum Order; + + /// The `aux_header` is a header from another instance of procssor. + /// It's required because all outputs should have the same structure. + /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. + PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); + + String getName() const override { return "PingPongProcessor"; } + + Status prepare(const PortNumbers &, const PortNumbers &) override; + + std::pair getAuxPorts(); + + virtual bool isReady(const Chunk & chunk) = 0; + +protected: + struct PortsPair + { + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_finished = false; + }; + + bool sendPing(); + bool recievePing(); + bool canSend() const; + + bool isPairsFinished() const; + bool processPair(PortsPair & pair); + void finishPair(PortsPair & pair); + Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); + + std::vector port_pairs; + size_t num_finished_pairs = 0; + + InputPort & aux_in_port; + OutputPort & aux_out_port; + + bool is_send = false; + bool is_recieved = false; + + bool ready_to_send = false; + + bool set_needed_once = false; + + Order order; +}; + +/// Reads first N rows from two streams evenly. +class ReadHeadBalancedProceesor : public PingPongProcessor +{ +public: + ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) + : PingPongProcessor(header, aux_header, num_ports, order_) + , size(size_) + { + } + + bool isReady(const Chunk & chunk) override + { + data_consumed += chunk.getNumRows(); + return data_consumed > size; + } + +private: + size_t data_consumed = 0; + + size_t size; +}; + +} diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index a56b53a6559..df0b399e37c 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { From d82a75ae752cf5ec9120c80d0a909ab911ef2386 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:26:47 +0000 Subject: [PATCH 167/582] cleanup PingPongProcessor --- src/Processors/PingPongProcessor.cpp | 30 +++++----------------------- src/Processors/PingPongProcessor.h | 20 +++++++++---------- 2 files changed, 14 insertions(+), 36 deletions(-) diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index 777d768f9a4..e34de025bb1 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -80,31 +80,15 @@ bool PingPongProcessor::isPairsFinished() const return num_finished_pairs == port_pairs.size(); } -IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::processRegularPorts() { if (isPairsFinished()) return Status::Finished; bool need_data = false; - UNUSED(updated_inputs); - UNUSED(updated_outputs); - - // for (const auto & output_number : updated_outputs) - for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) - { - if (output_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[output_number]) || need_data; - } - - // for (const auto & input_number : updated_inputs) - for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) - { - if (input_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[input_number]) || need_data; - } + for (auto & pair : port_pairs) + need_data = processPair(pair) || need_data; if (isPairsFinished()) return Status::Finished; @@ -145,7 +129,7 @@ bool PingPongProcessor::canSend() const return !is_send && (ready_to_send || isPairsFinished()); } -IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::prepare() { if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) { @@ -175,7 +159,7 @@ IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs } } - auto status = processRegularPorts(updated_inputs, updated_outputs); + auto status = processRegularPorts(); if (status == Status::Finished) { if (order == First || is_send) @@ -200,10 +184,6 @@ IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs } } } - if (status == Status::PortFull) - { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); - } return status; } diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 09cdc6c0ec5..676ad44ea96 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -44,7 +44,7 @@ public: String getName() const override { return "PingPongProcessor"; } - Status prepare(const PortNumbers &, const PortNumbers &) override; + Status prepare() override; std::pair getAuxPorts(); @@ -65,7 +65,7 @@ protected: bool isPairsFinished() const; bool processPair(PortsPair & pair); void finishPair(PortsPair & pair); - Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); + Status processRegularPorts(); std::vector port_pairs; size_t num_finished_pairs = 0; @@ -78,6 +78,7 @@ protected: bool ready_to_send = false; + /// Used to set 'needed' flag once for auxiliary input at first `prepare` call. bool set_needed_once = false; Order order; @@ -87,22 +88,19 @@ protected: class ReadHeadBalancedProceesor : public PingPongProcessor { public: - ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) - : PingPongProcessor(header, aux_header, num_ports, order_) - , size(size_) - { - } + ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) + : PingPongProcessor(header, aux_header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) + {} bool isReady(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); - return data_consumed > size; + return data_consumed > size_to_wait; } private: - size_t data_consumed = 0; - - size_t size; + size_t data_consumed; + size_t size_to_wait; }; } From 67a9acc8db9fd25635048e1e21b4b37d67606a90 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:28:28 +0000 Subject: [PATCH 168/582] rename CreatingSetOnTheFlyStep -> CreateSetAndFilterOnTheFlyStep --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++--- ...cpp => CreateSetAndFilterOnTheFlyStep.cpp} | 26 +++++++------------ ...tep.h => CreateSetAndFilterOnTheFlyStep.h} | 6 ++--- .../Optimizations/filterPushDown.cpp | 4 +-- 4 files changed, 18 insertions(+), 24 deletions(-) rename src/Processors/QueryPlan/{CreatingSetOnTheFlyStep.cpp => CreateSetAndFilterOnTheFlyStep.cpp} (86%) rename src/Processors/QueryPlan/{CreatingSetOnTheFlyStep.h => CreateSetAndFilterOnTheFlyStep.h} (88%) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 15ed0e7d493..72fe8190969 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,7 +39,7 @@ #include #include #include -#include +#include #include #include #include @@ -1446,13 +1446,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); - auto creating_set_step = std::make_unique( + auto creating_set_step = std::make_unique( plan.getCurrentDataStream(), rhs_data_stream, key_names, max_rows, crosswise_connection, join_pos); creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp similarity index 86% rename from src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp rename to src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index df0b399e37c..9a49cf21216 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -17,11 +17,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -namespace -{ - -InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) +static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) { auto input_it = inputs.begin(); for (size_t i = 0; i < num_ports; ++i) @@ -32,8 +28,6 @@ InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inpu return input_it; } -} - static ITransformingStep::Traits getTraits(bool is_filter) { return ITransformingStep::Traits @@ -51,7 +45,7 @@ static ITransformingStep::Traits getTraits(bool is_filter) } -class CreatingSetOnTheFlyStep::CrosswiseConnection : public boost::noncopyable +class CreateSetAndFilterOnTheFlyStep::CrosswiseConnection : public boost::noncopyable { public: using PortPair = std::pair; @@ -85,12 +79,12 @@ private: std::unique_ptr dummy_input_port; }; -CreatingSetOnTheFlyStep::CrosswiseConnectionPtr CreatingSetOnTheFlyStep::createCrossConnection() +CreateSetAndFilterOnTheFlyStep::CrosswiseConnectionPtr CreateSetAndFilterOnTheFlyStep::createCrossConnection() { - return std::make_shared(); + return std::make_shared(); } -CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep( +CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, @@ -118,7 +112,7 @@ CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep( own_set->setHeader(header); } -void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { size_t num_streams = pipeline.getNumStreams(); pipeline.addSimpleTransform([this, num_streams](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -172,12 +166,12 @@ void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, }, /* check_ports= */ false); } -void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const +void CreateSetAndFilterOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const { map.add(getName(), true); } -void CreatingSetOnTheFlyStep::describeActions(FormatSettings & settings) const +void CreateSetAndFilterOnTheFlyStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, ' '); settings.out << prefix << getName(); @@ -185,7 +179,7 @@ void CreatingSetOnTheFlyStep::describeActions(FormatSettings & settings) const settings.out << '\n'; } -void CreatingSetOnTheFlyStep::updateOutputStream() +void CreateSetAndFilterOnTheFlyStep::updateOutputStream() { if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h similarity index 88% rename from src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h rename to src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 67f31a97009..c489ff7356a 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -7,7 +7,7 @@ namespace DB { -class CreatingSetOnTheFlyStep : public ITransformingStep +class CreateSetAndFilterOnTheFlyStep : public ITransformingStep { public: /// Two instances of step need some shared state to connect processors crosswise @@ -15,7 +15,7 @@ public: using CrosswiseConnectionPtr = std::shared_ptr; static CrosswiseConnectionPtr createCrossConnection(); - CreatingSetOnTheFlyStep( + CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, @@ -48,7 +48,7 @@ private: JoinTableSide position; - Poco::Logger * log = &Poco::Logger::get("CreatingSetOnTheFlyStep"); + Poco::Logger * log = &Poco::Logger::get("CreateSetAndFilterOnTheFlyStep"); }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index f60f8192938..7c0402bb4b9 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -336,7 +336,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; - if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; if (auto * union_step = typeid_cast(child.get())) From 7228091ff1ef7252289a9d113eb7803fc7f9dfb4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:44:57 +0000 Subject: [PATCH 169/582] rename CreateSetAndFilterOnTheFlyTransform --- src/Processors/PingPongProcessor.h | 4 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 45 +++++++++++-------- .../CreateSetAndFilterOnTheFlyStep.h | 13 ++++-- ...> CreateSetAndFilterOnTheFlyTransform.cpp} | 5 ++- ... => CreateSetAndFilterOnTheFlyTransform.h} | 0 5 files changed, 42 insertions(+), 25 deletions(-) rename src/Processors/Transforms/{CreatingSetsOnTheFlyTransform.cpp => CreateSetAndFilterOnTheFlyTransform.cpp} (98%) rename src/Processors/Transforms/{CreatingSetsOnTheFlyTransform.h => CreateSetAndFilterOnTheFlyTransform.h} (100%) diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 676ad44ea96..3471ea2d772 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -85,10 +85,10 @@ protected: }; /// Reads first N rows from two streams evenly. -class ReadHeadBalancedProceesor : public PingPongProcessor +class ReadHeadBalancedProcessor : public PingPongProcessor { public: - ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) + ReadHeadBalancedProcessor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) : PingPongProcessor(header, aux_header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) {} diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 9a49cf21216..5ec160f6251 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include @@ -51,6 +51,7 @@ public: using PortPair = std::pair; /// Remember ports passed on the first call and connect with ones from second call. + /// Thread-safe. bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) { std::lock_guard lock(mux); @@ -88,14 +89,14 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, - size_t max_rows_, + size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) , column_names(column_names_) - , max_rows(max_rows_) + , max_rows_in_set(max_rows_in_set_) , rhs_input_stream_header(rhs_input_stream_.header) - , own_set(std::make_shared(SizeLimits(max_rows, 0, OverflowMode::BREAK), false, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) @@ -126,29 +127,33 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi if (!filtering_set) { - LOG_DEBUG(log, "filtering_set is null"); + LOG_DEBUG(log, "Skip filtering {} stream", position); return; } Block input_header = pipeline.getHeader(); - pipeline.transform([&input_header, this](OutputPortRawPtrs ports) + auto pipeline_transform = [&input_header, this](OutputPortRawPtrs ports) { - Processors transforms; + Processors result_transforms; size_t num_ports = ports.size(); + /// Add balancing transform auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; - auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows, idx); - notifier->setDescription(getStepDescription()); + auto stream_balancer = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows_in_set, idx); + stream_balancer->setDescription(getStepDescription()); - auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); - assert(&*input_it == notifier->getAuxPorts().first); + /// Regular inputs just bypass data for respective ports + auto input_it = connectAllInputs(ports, stream_balancer->getInputs(), num_ports); + assert(&*input_it == stream_balancer->getAuxPorts().first); input_it++; - assert(input_it == notifier->getInputs().end()); + assert(input_it == stream_balancer->getInputs().end()); - crosswise_connection->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); + /// Connect auxilary ports + crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); - auto & outputs = notifier->getOutputs(); + /// Add filtering transform, ports just connected respectively + auto & outputs = stream_balancer->getOutputs(); auto output_it = outputs.begin(); for (size_t i = 0; i < outputs.size() - 1; ++i) { @@ -156,14 +161,18 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi auto transform = std::make_shared(port.getHeader(), column_names, filtering_set); transform->setDescription(this->getStepDescription()); connect(port, transform->getInputPort()); - transforms.emplace_back(std::move(transform)); + result_transforms.emplace_back(std::move(transform)); } output_it++; assert(output_it == outputs.end()); - transforms.emplace_back(std::move(notifier)); + result_transforms.emplace_back(std::move(stream_balancer)); - return transforms; - }, /* check_ports= */ false); + return result_transforms; + }; + + /// Auxilary port stream_balancer can be connected later (by crosswise_connection). + /// So, use unsafe `transform` with `check_ports = false` to avoid assertions + pipeline.transform(std::move(pipeline_transform), /* check_ports= */ false); } void CreateSetAndFilterOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index c489ff7356a..bcaa87ed551 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -1,12 +1,17 @@ #pragma once #include -#include #include namespace DB { +/* + * Used to optimize JOIN when joining a small table over a large table. + * Currently applied only for the full sorting join. + * It tries to build a set for each stream. + * Once one stream is finished, it starts to filter another stream with this set. + */ class CreateSetAndFilterOnTheFlyStep : public ITransformingStep { public: @@ -19,7 +24,7 @@ public: const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, - size_t max_rows_, + size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_); @@ -30,6 +35,8 @@ public: void describeActions(FormatSettings & settings) const override; SetWithStatePtr getSet() const { return own_set; } + + /// Set for another stream. void setFiltering(SetWithStatePtr filtering_set_) { filtering_set = filtering_set_; } private: @@ -37,7 +44,7 @@ private: Names column_names; - size_t max_rows; + size_t max_rows_in_set; Block rhs_input_stream_header; diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp similarity index 98% rename from src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp rename to src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 1f9282d8baa..27d033a4580 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -1,8 +1,9 @@ +#include + #include #include -#include -#include +#include #include #include #include diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h similarity index 100% rename from src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h rename to src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h From 8e1632f8242ed3eaf8a62bf9faf06d33e74edfb3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:21:34 +0000 Subject: [PATCH 170/582] Create sets for joins: better code --- .../CreateSetAndFilterOnTheFlyStep.h | 1 + .../CreateSetAndFilterOnTheFlyTransform.cpp | 54 +++++++++++-------- .../CreateSetAndFilterOnTheFlyTransform.h | 38 ++++++++----- 3 files changed, 58 insertions(+), 35 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index bcaa87ed551..6e38fd31349 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 27d033a4580..8360eae9a8e 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -53,7 +53,6 @@ std::string formatBytesHumanReadable(size_t bytes) } - CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) @@ -67,35 +66,39 @@ CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() { - auto status = ISimpleTransform::prepare(); + IProcessor::Status status = ISimpleTransform::prepare(); - if (status == Status::Finished && set && set->state == SetWithState::State::Creating) + if (!set || status != Status::Finished) + /// Nothing to handle with set + return status; + + /// Finalize set + if (set->state == SetWithState::State::Creating) { if (input.isFinished()) { set->finished_count++; - if (set->finished_count == num_streams) - { - set->finishInsert(); - set->state = SetWithState::State::Finished; - LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); - } + if (set->finished_count != num_streams) + /// Not all instances of processor are finished + return status; + + set->finishInsert(); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + set.reset(); } else { - /// Should not happen because processor places before join that reads all the data + /// Should not happen because processor inserted before join that reads all the data /// But let's hanlde this case just for safety. set->state = SetWithState::State::Suspended; LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); + set.reset(); } } - if (status == Status::Finished && set && set->state != SetWithState::State::Creating) - /// Release pointer to make it possible destroy it by consumer - set.reset(); - return status; } @@ -103,7 +106,8 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) { if (!set || set->state != SetWithState::State::Creating) { - if (set) + /// If set building suspended by another processor, release pointer + if (set != nullptr) set.reset(); return; } @@ -111,17 +115,18 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); - size_t prev_size = set->getTotalByteCount(); bool limit_exceeded = !set->insertFromBlock(key_columns); if (limit_exceeded) { auto prev_state = set->state.exchange(SetWithState::State::Suspended); + /// Print log only after first state switch if (prev_state == SetWithState::State::Creating) { - LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {} ({} -> {} bytes)", - getDescription(), formatBytesHumanReadable(set->getTotalByteCount()), prev_size, set->getTotalByteCount()); + LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after reading {} rows and using {}", + getDescription(), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); } - // TODO(@vdimir): set->clear() ? + /// Probaply we need to clear set here, because it's unneded anymore + /// But now `Set` doesn't have such method, so reset pointer in all processors and then it should be freed set.reset(); } } @@ -142,15 +147,19 @@ FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_ IProcessor::Status FilterBySetOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); + + if (set && set->state == SetWithState::State::Suspended) + set.reset(); + if (status == Status::Finished) { bool has_filter = set && set->state == SetWithState::State::Finished; if (has_filter) { - LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {}% filtered", + LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {:.2f}% filtered", Poco::toLower(getDescription()), fmt::join(column_names, ", "), stat.consumed_rows, stat.consumed_rows_before_set, stat.result_rows, - static_cast(100 - 100.0 * stat.result_rows / stat.consumed_rows)); + 100 - 100.0 * stat.result_rows / stat.consumed_rows); } else { @@ -169,7 +178,6 @@ void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) stat.result_rows += chunk.getNumRows(); bool can_filter = set && set->state == SetWithState::State::Finished; - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {} / {}", __FILE__, __LINE__, set->finished_count.load(), set->state.load(), set->getTotalRowCount()); if (!can_filter) stat.consumed_rows_before_set += chunk.getNumRows(); diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index 072095cc3db..4542535c7ef 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -6,39 +7,50 @@ #include #include - namespace DB { -class SetWithState : public Set +struct SetWithState : public Set { -public: using Set::Set; + /// Flow: Creating -> Finished or Suspended enum class State { + /// Set is not yet created, + /// Creating processor continues to build set. + /// Filtering bypasses data. Creating, + + /// Set is finished. + /// Creating processor is finished. + /// Filtering filter stream with this set. Finished, + + /// Set building is canceled (due to limit exceeded). + /// Creating and filtering processors bypass data. Suspended, }; std::atomic state = State::Creating; + + /// Track number of processors that are currently working on this set. + /// Last one finalizes set. std::atomic_size_t finished_count = 0; }; using SetWithStatePtr = std::shared_ptr; /* - * Create a set on the fly. + * Create a set on the fly for incomming stream. * The set is created from the key columns of the input block. * Data is not changed and returned as is. - * Can be executed only in one stream. + * Can be executed in parallel, but blocks on operations with set. */ class CreatingSetsOnTheFlyTransform : public ISimpleTransform { public: - explicit CreatingSetsOnTheFlyTransform( - const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_); + CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_); String getName() const override { return "CreatingSetsOnTheFlyTransform"; } @@ -58,7 +70,6 @@ private: Poco::Logger * log; }; - /* * Filter the input chunk by the set. * When set building is not completed, just return the source data. @@ -66,8 +77,7 @@ private: class FilterBySetOnTheFlyTransform : public ISimpleTransform { public: - explicit FilterBySetOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetWithStatePtr set_); + FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_); String getName() const override { return "FilterBySetOnTheFlyTransform"; } @@ -85,16 +95,20 @@ private: /// Filter by this set when it's created SetWithStatePtr set; + /// Statistics to log struct Stat { + /// Total number of rows size_t consumed_rows = 0; + + /// Number of bypassed rows (processed before set is created) size_t consumed_rows_before_set = 0; + + /// Number of rows that passed the filter size_t result_rows = 0; } stat; Poco::Logger * log; }; - - } From 714c53ab24569f5553050b2c17c5837fab903167 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:29:05 +0000 Subject: [PATCH 171/582] fix typos --- src/Processors/PingPongProcessor.cpp | 20 +++++++++---------- src/Processors/PingPongProcessor.h | 2 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 4 ++-- .../CreateSetAndFilterOnTheFlyTransform.cpp | 4 ++-- .../CreateSetAndFilterOnTheFlyTransform.h | 2 +- 5 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index e34de025bb1..78f8a12e9d3 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -117,7 +117,7 @@ bool PingPongProcessor::recievePing() if (aux_in_port.hasData()) { aux_in_port.pull(); - is_recieved = true; + is_received = true; aux_in_port.close(); return true; } @@ -131,7 +131,7 @@ bool PingPongProcessor::canSend() const IProcessor::Status PingPongProcessor::prepare() { - if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) + if (!set_needed_once && !is_received && !aux_in_port.isFinished()) { set_needed_once = true; aux_in_port.setNeeded(); @@ -139,17 +139,17 @@ IProcessor::Status PingPongProcessor::prepare() if (order == First || is_send) { - if (!is_recieved) + if (!is_received) { - bool recieved = recievePing(); - if (!recieved) + bool received = recievePing(); + if (!received) { return Status::NeedData; } } } - if (order == Second || is_recieved) + if (order == Second || is_received) { if (!is_send && canSend()) { @@ -164,17 +164,17 @@ IProcessor::Status PingPongProcessor::prepare() { if (order == First || is_send) { - if (!is_recieved) + if (!is_received) { - bool recieved = recievePing(); - if (!recieved) + bool received = recievePing(); + if (!received) { return Status::NeedData; } } } - if (order == Second || is_recieved) + if (order == Second || is_received) { if (!is_send && canSend()) { diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 3471ea2d772..cbcead79633 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -74,7 +74,7 @@ protected: OutputPort & aux_out_port; bool is_send = false; - bool is_recieved = false; + bool is_received = false; bool ready_to_send = false; diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 5ec160f6251..b81bfd027ca 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -149,7 +149,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi input_it++; assert(input_it == stream_balancer->getInputs().end()); - /// Connect auxilary ports + /// Connect auxiliary ports crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); /// Add filtering transform, ports just connected respectively @@ -170,7 +170,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi return result_transforms; }; - /// Auxilary port stream_balancer can be connected later (by crosswise_connection). + /// Auxiliary port stream_balancer can be connected later (by crosswise_connection). /// So, use unsafe `transform` with `check_ports = false` to avoid assertions pipeline.transform(std::move(pipeline_transform), /* check_ports= */ false); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 8360eae9a8e..5ef614bd3bc 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -91,7 +91,7 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() else { /// Should not happen because processor inserted before join that reads all the data - /// But let's hanlde this case just for safety. + /// But let's handle this case just for safety. set->state = SetWithState::State::Suspended; LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); @@ -125,7 +125,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after reading {} rows and using {}", getDescription(), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); } - /// Probaply we need to clear set here, because it's unneded anymore + /// Probaply we need to clear set here, because it's unneeded anymore /// But now `Set` doesn't have such method, so reset pointer in all processors and then it should be freed set.reset(); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index 4542535c7ef..9b625d0dfaf 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -42,7 +42,7 @@ struct SetWithState : public Set using SetWithStatePtr = std::shared_ptr; /* - * Create a set on the fly for incomming stream. + * Create a set on the fly for incoming stream. * The set is created from the key columns of the input block. * Data is not changed and returned as is. * Can be executed in parallel, but blocks on operations with set. From 51e02d09f65b758eea98c25ab9f99f0ce5bf24f1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:29:31 +0000 Subject: [PATCH 172/582] set preserves_sorting = true for CreateSetAndFilterOnTheFlyStep --- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index b81bfd027ca..176cd229efa 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -36,7 +36,7 @@ static ITransformingStep::Traits getTraits(bool is_filter) .preserves_distinct_columns = true, .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = false, /// resize doesn't perserve sorting (TODO fixit) + .preserves_sorting = true, }, { .preserves_number_of_rows = !is_filter, From 470dcff89c6c83c952555aeff4fd3f289fad63e2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:52:24 +0000 Subject: [PATCH 173/582] Add tests/performance/join_set_filter.xml --- tests/performance/join_set_filter.xml | 46 +++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 tests/performance/join_set_filter.xml diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml new file mode 100644 index 00000000000..cb493fb435b --- /dev/null +++ b/tests/performance/join_set_filter.xml @@ -0,0 +1,46 @@ + + + + table_size + + 100000000 + + + + + + 100000 + full_sorting_merge + + + + CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y + AS SELECT + sipHash64(number, 't1_x') % {table_size} AS x, + sipHash64(number, 't1_y') % {table_size} AS y + FROM numbers({table_size}) + + + + CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y + AS SELECT + sipHash64(number, 't2_x') % {table_size} AS x, + sipHash64(number, 't2_y') % {table_size} AS y + FROM numbers({table_size}) + + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE less(t1.y, 10000) + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE less(t1.y, 10000) + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 100 = 0 + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 100 = 0 + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 1000 = 0 + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 1000 = 0 + + DROP TABLE IF EXISTS t1 + DROP TABLE IF EXISTS t2 + From c67ab33d90845b9156f4656f76c8c15fc233a270 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 16:13:58 +0000 Subject: [PATCH 174/582] small fix CreateSetAndFilterOnTheFlyStep --- .../CreateSetAndFilterOnTheFlyStep.cpp | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 176cd229efa..340ca07e561 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -28,7 +28,7 @@ static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts return input_it; } -static ITransformingStep::Traits getTraits(bool is_filter) +static ITransformingStep::Traits getTraits() { return ITransformingStep::Traits { @@ -39,7 +39,7 @@ static ITransformingStep::Traits getTraits(bool is_filter) .preserves_sorting = true, }, { - .preserves_number_of_rows = !is_filter, + .preserves_number_of_rows = false, } }; } @@ -77,6 +77,8 @@ private: InputPort * input_port = nullptr; OutputPort * output_port = nullptr; + /// Output ports should always be connected, and we can't add a step to the pipeline without them. + /// So, connect the port from the first processor to this dummy port and then reconnect to the second processor. std::unique_ptr dummy_input_port; }; @@ -92,7 +94,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) , rhs_input_stream_header(rhs_input_stream_.header) @@ -125,12 +127,6 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi return res; }); - if (!filtering_set) - { - LOG_DEBUG(log, "Skip filtering {} stream", position); - return; - } - Block input_header = pipeline.getHeader(); auto pipeline_transform = [&input_header, this](OutputPortRawPtrs ports) { @@ -152,6 +148,13 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi /// Connect auxiliary ports crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); + if (!filtering_set) + { + LOG_DEBUG(log, "Skip filtering {} stream", position); + result_transforms.emplace_back(std::move(stream_balancer)); + return result_transforms; + } + /// Add filtering transform, ports just connected respectively auto & outputs = stream_balancer->getOutputs(); auto output_it = outputs.begin(); From 95f87dc34e549ad32c509358d66226d973742916 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 Aug 2022 09:22:52 +0000 Subject: [PATCH 175/582] fix sanitizer assert in CreateSetAndFilterOnTheFlyStep --- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 340ca07e561..ecd2e34e56e 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -107,7 +107,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( throw Exception(ErrorCodes::LOGICAL_ERROR, "Crosswise connection is not initialized"); if (input_streams.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Step requires exactly one input stream, got {}", input_streams.size()); ColumnsWithTypeAndName header; for (const auto & name : column_names) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 6e38fd31349..4ef7fee8ada 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -29,7 +29,7 @@ public: CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_); - String getName() const override { return "CreatingSetsOnTheFly"; } + String getName() const override { return "CreateSetAndFilterOnTheFlyStep"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void describeActions(JSONBuilder::JSONMap & map) const override; From e21763e75952423fdc0543d683b5c77ee2f07522 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 Aug 2022 09:23:25 +0000 Subject: [PATCH 176/582] remove new setting from join_set_filter.xml --- tests/performance/join_set_filter.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml index cb493fb435b..7f7804853fc 100644 --- a/tests/performance/join_set_filter.xml +++ b/tests/performance/join_set_filter.xml @@ -9,7 +9,6 @@ - 100000 full_sorting_merge From afeff512b5809157709ad9c2a374a5f4e8d8a594 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 11:54:15 +0000 Subject: [PATCH 177/582] Aux port for ReadHeadBalancedProcessor is empty Block --- src/Interpreters/InterpreterSelectQuery.cpp | 8 ++++---- src/Processors/PingPongProcessor.cpp | 13 ++++++++----- src/Processors/PingPongProcessor.h | 12 +++++------- .../QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 4 +--- .../QueryPlan/CreateSetAndFilterOnTheFlyStep.h | 3 --- 5 files changed, 18 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 72fe8190969..a3fd08de558 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1447,13 +1447,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); auto creating_set_step = std::make_unique( - plan.getCurrentDataStream(), rhs_data_stream, key_names, max_rows, crosswise_connection, join_pos); + plan.getCurrentDataStream(), key_names, max_rows, crosswise_connection, join_pos); creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); auto * step_raw_ptr = creating_set_step.get(); @@ -1470,8 +1470,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

= 0 && kind_allows_filtering) { - auto * left_set = add_create_set(query_plan, joined_plan->getCurrentDataStream(), join_clause.key_names_left, JoinTableSide::Left); - auto * right_set = add_create_set(*joined_plan, query_plan.getCurrentDataStream(), join_clause.key_names_right, JoinTableSide::Right); + auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left); + auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); if (isInnerOrLeft(join_kind)) right_set->setFiltering(left_set->getSet()); diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index 78f8a12e9d3..50c7a836d00 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -3,15 +3,18 @@ namespace DB { -static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) +/// Create list with `num_ports` of regular ports and 1 auxiliary port with empty header. +template requires std::is_same_v || std::is_same_v +static T createPortsWithSpecial(const Block & header, size_t num_ports) { - InputPorts res(num_ports, header); - res.emplace_back(last_header); + T res(num_ports, header); + res.emplace_back(Block()); return res; } -PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) - : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) +PingPongProcessor::PingPongProcessor(const Block & header, size_t num_ports, Order order_) + : IProcessor(createPortsWithSpecial(header, num_ports), + createPortsWithSpecial(header, num_ports)) , aux_in_port(inputs.back()) , aux_out_port(outputs.back()) , order(order_) diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index cbcead79633..8e4eb4533bc 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -37,10 +37,7 @@ public: using enum Order; - /// The `aux_header` is a header from another instance of procssor. - /// It's required because all outputs should have the same structure. - /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. - PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); + PingPongProcessor(const Block & header, size_t num_ports, Order order_); String getName() const override { return "PingPongProcessor"; } @@ -88,9 +85,10 @@ protected: class ReadHeadBalancedProcessor : public PingPongProcessor { public: - ReadHeadBalancedProcessor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) - : PingPongProcessor(header, aux_header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) - {} + ReadHeadBalancedProcessor(const Block & header, size_t num_ports, size_t size_to_wait_, Order order_) + : PingPongProcessor(header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) + { + } bool isReady(const Chunk & chunk) override { diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index ecd2e34e56e..3787d5b2aba 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -89,7 +89,6 @@ CreateSetAndFilterOnTheFlyStep::CrosswiseConnectionPtr CreateSetAndFilterOnTheFl CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, - const DataStream & rhs_input_stream_, const Names & column_names_, size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, @@ -97,7 +96,6 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , rhs_input_stream_header(rhs_input_stream_.header) , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) @@ -136,7 +134,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi /// Add balancing transform auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; - auto stream_balancer = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows_in_set, idx); + auto stream_balancer = std::make_shared(input_header, num_ports, max_rows_in_set, idx); stream_balancer->setDescription(getStepDescription()); /// Regular inputs just bypass data for respective ports diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 4ef7fee8ada..8c2eef00af0 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -23,7 +23,6 @@ public: CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, - const DataStream & rhs_input_stream_, const Names & column_names_, size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, @@ -47,8 +46,6 @@ private: size_t max_rows_in_set; - Block rhs_input_stream_header; - SetWithStatePtr own_set; SetWithStatePtr filtering_set; From 2deb28d56e38e0f72ecbf97a559335892f468957 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 12:04:06 +0000 Subject: [PATCH 178/582] Add 02382_join_and_filtering_set --- .../02382_join_and_filtering_set.reference | 7 +++++++ .../02382_join_and_filtering_set.sql | 20 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02382_join_and_filtering_set.reference create mode 100644 tests/queries/0_stateless/02382_join_and_filtering_set.sql diff --git a/tests/queries/0_stateless/02382_join_and_filtering_set.reference b/tests/queries/0_stateless/02382_join_and_filtering_set.reference new file mode 100644 index 00000000000..58c8ccca6a2 --- /dev/null +++ b/tests/queries/0_stateless/02382_join_and_filtering_set.reference @@ -0,0 +1,7 @@ +106 +46 +42 +51 +42 +24 +10 diff --git a/tests/queries/0_stateless/02382_join_and_filtering_set.sql b/tests/queries/0_stateless/02382_join_and_filtering_set.sql new file mode 100644 index 00000000000..4b425f22c87 --- /dev/null +++ b/tests/queries/0_stateless/02382_join_and_filtering_set.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't1_x') % 100 AS x, sipHash64(number, 't1_y') % 100 AS y FROM numbers(100); + +CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't2_x') % 100 AS x, sipHash64(number, 't2_y') % 100 AS y FROM numbers(100); + +SET max_rows_in_set_to_optimize_join = 1000; +SET join_algorithm = 'full_sorting_merge'; + +-- different combinations of conditions on key/attribute columns for the left/right tables +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.x % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t2.y % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t2.x % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 2 == 0 AND t2.y % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.x % 2 == 0 AND t2.x % 2 == 0 AND t1.y % 2 == 0 AND t2.y % 2 == 0; From afb6b7d9cfbc290590ce5b90c9eb7f75810ec93f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 12:39:39 +0000 Subject: [PATCH 179/582] Test plan and pipeline for filtering step for join --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 7 +-- src/Processors/PingPongProcessor.h | 4 +- .../02383_join_and_filtering_set.reference | 10 ++++ .../02383_join_and_filtering_set.sh | 55 +++++++++++++++++++ 5 files changed, 70 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02383_join_and_filtering_set.reference create mode 100755 tests/queries/0_stateless/02383_join_and_filtering_set.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8a81a8b0b29..9f65527f0db 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining (-1 to disable).", 0) \ + M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a3fd08de558..cfd0d4815c6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1449,11 +1449,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); - auto creating_set_step = std::make_unique( - plan.getCurrentDataStream(), key_names, max_rows, crosswise_connection, join_pos); + plan.getCurrentDataStream(), key_names, settings.max_rows_in_set_to_optimize_join, crosswise_connection, join_pos); creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); auto * step_raw_ptr = creating_set_step.get(); @@ -1468,7 +1465,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

= 0 && kind_allows_filtering) + if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering) { auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 8e4eb4533bc..2b461f8409e 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -39,8 +39,6 @@ public: PingPongProcessor(const Block & header, size_t num_ports, Order order_); - String getName() const override { return "PingPongProcessor"; } - Status prepare() override; std::pair getAuxPorts(); @@ -90,6 +88,8 @@ public: { } + String getName() const override { return "ReadHeadBalancedProcessor"; } + bool isReady(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); diff --git a/tests/queries/0_stateless/02383_join_and_filtering_set.reference b/tests/queries/0_stateless/02383_join_and_filtering_set.reference new file mode 100644 index 00000000000..2ad282ca07f --- /dev/null +++ b/tests/queries/0_stateless/02383_join_and_filtering_set.reference @@ -0,0 +1,10 @@ +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok diff --git a/tests/queries/0_stateless/02383_join_and_filtering_set.sh b/tests/queries/0_stateless/02383_join_and_filtering_set.sh new file mode 100755 index 00000000000..3356be58ff7 --- /dev/null +++ b/tests/queries/0_stateless/02383_join_and_filtering_set.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# Tags: no-asan,no-msan,no-tsan,no-ubsan +# +# Test doesn't run complex queries, just test the logic of setting, so no need to run with different builds. +# Also, we run similar queries in 02382_join_and_filtering_set.sql which is enabled for these builds. +# + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -mn -q """ +CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't1_x') % 100 AS x, sipHash64(number, 't1_y') % 100 AS y FROM numbers(100); + +CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't2_x') % 100 AS x, sipHash64(number, 't2_y') % 100 AS y FROM numbers(100); +""" + +# Arguments: +# - value of max_rows_in_set_to_optimize_join +# - join kind +# - expected number of steps in plan +# - expected number of steps in pipeline +function test() { + +PARAM_VALUE=$1 +JOIN_KIND=${2:-} + +EXPECTED_PLAN_STEPS=$3 +RES=$( + $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \ + -q "EXPLAIN PLAN SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" | grep -o 'CreateSetAndFilterOnTheFlyStep' | wc -l +) +[ "$RES" -eq "$EXPECTED_PLAN_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PLAN_STEPS" + +EXPECTED_PIPELINE_STEPS=$4 +RES=$( + $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \ + -q "EXPLAIN PIPELINE SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" \ + | grep -o -e ReadHeadBalancedProcessor -e FilterBySetOnTheFlyTransform -e CreatingSetsOnTheFlyTransform | wc -l +) +[ "$RES" -eq "$EXPECTED_PIPELINE_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PIPELINE_STEPS" + +} + +test 1000 '' 2 6 + +# no filtering for left/right side +test 1000 'LEFT' 2 5 +test 1000 'RIGHT' 2 5 + +# when disabled no extra steps should be created +test 1000 'FULL' 0 0 +test 0 '' 0 0 From 7915b6948fee1173b705c4eed7636383a5288d69 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 13:03:14 +0000 Subject: [PATCH 180/582] Fix build after rebase --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 7c0402bb4b9..dbf389163be 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -303,8 +303,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (updated_steps > 0) { LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); - return updated_steps; } + return updated_steps; }; if (size_t updated_steps = join_push_down(JoinKind::Left)) From 3e06cf3a9dba2d7bedd74cf5c274e5199eb78de2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 10:10:47 +0000 Subject: [PATCH 181/582] Upd max_rows_in_set_to_optimize_join description --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9f65527f0db..84bb086fbfb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ + M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ From b0e2616aa916115e3f1011db29f75615d1bdb997 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 10:11:30 +0000 Subject: [PATCH 182/582] Style fixes in CreateSetAndFilterOnTheFlyTransform and related --- src/Processors/PingPongProcessor.cpp | 2 +- src/Processors/PingPongProcessor.h | 9 ++-- .../CreateSetAndFilterOnTheFlyStep.cpp | 48 ++++++++++--------- .../CreateSetAndFilterOnTheFlyTransform.cpp | 23 +++------ .../CreateSetAndFilterOnTheFlyTransform.h | 4 +- 5 files changed, 40 insertions(+), 46 deletions(-) diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index 50c7a836d00..e9d61386314 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -71,7 +71,7 @@ bool PingPongProcessor::processPair(PortsPair & pair) if (pair.input_port->hasData()) { Chunk chunk = pair.input_port->pull(true); - ready_to_send = isReady(chunk) || ready_to_send; + ready_to_send |= consume(chunk); pair.output_port->push(std::move(chunk)); } diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 2b461f8409e..3dbe1178332 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -20,8 +20,8 @@ namespace DB * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ * * One of the processors starts processing data, and another waits for notification. - * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. - * After that, the second one also processes data until `isReady`, then send a notification back to the first one. + * When `consume` returns true, the first stops processing, sends a ping to another and waits for notification. + * After that, the second one also processes data until `consume`, then send a notification back to the first one. * After this roundtrip, processors bypass data from regular inputs to outputs. */ class PingPongProcessor : public IProcessor @@ -43,7 +43,8 @@ public: std::pair getAuxPorts(); - virtual bool isReady(const Chunk & chunk) = 0; + /// Returns `true` when enough data consumed + virtual bool consume(const Chunk & chunk) = 0; protected: struct PortsPair @@ -90,7 +91,7 @@ public: String getName() const override { return "ReadHeadBalancedProcessor"; } - bool isReady(const Chunk & chunk) override + bool consume(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); return data_consumed > size_to_wait; diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 3787d5b2aba..e42642ceff8 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) +static void connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) { auto input_it = inputs.begin(); for (size_t i = 0; i < num_ports; ++i) @@ -25,7 +25,14 @@ static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts connect(*ports[i], *input_it); input_it++; } - return input_it; +} + +static ColumnsWithTypeAndName getColumnSubset(const Block & block, const Names & column_names) +{ + ColumnsWithTypeAndName result; + for (const auto & name : column_names) + result.emplace_back(block.getByName(name)); + return result; } static ITransformingStep::Traits getTraits() @@ -44,7 +51,6 @@ static ITransformingStep::Traits getTraits() }; } - class CreateSetAndFilterOnTheFlyStep::CrosswiseConnection : public boost::noncopyable { public: @@ -52,8 +58,10 @@ public: /// Remember ports passed on the first call and connect with ones from second call. /// Thread-safe. - bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) + void connectPorts(PortPair rhs_ports, IProcessor * proc) { + assert(!rhs_ports.first->isConnected() && !rhs_ports.second->isConnected()); + std::lock_guard lock(mux); if (input_port || output_port) { @@ -61,15 +69,16 @@ public: assert(!input_port->isConnected()); connect(*rhs_ports.second, *input_port); connect(*output_port, *rhs_ports.first, /* reconnect= */ true); - return true; } - std::tie(input_port, output_port) = rhs_ports; - assert(input_port && output_port); - assert(!input_port->isConnected() && !output_port->isConnected()); + else + { + std::tie(input_port, output_port) = rhs_ports; + assert(input_port && output_port); + assert(!input_port->isConnected() && !output_port->isConnected()); - dummy_input_port = std::make_unique(output_port->getHeader(), proc); - connect(*output_port, *dummy_input_port); - return false; + dummy_input_port = std::make_unique(output_port->getHeader(), proc); + connect(*output_port, *dummy_input_port); + } } private: @@ -107,10 +116,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Step requires exactly one input stream, got {}", input_streams.size()); - ColumnsWithTypeAndName header; - for (const auto & name : column_names) - header.emplace_back(input_streams[0].header.getByName(name)); - own_set->setHeader(header); + own_set->setHeader(getColumnSubset(input_streams[0].header, column_names)); } void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -138,13 +144,10 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi stream_balancer->setDescription(getStepDescription()); /// Regular inputs just bypass data for respective ports - auto input_it = connectAllInputs(ports, stream_balancer->getInputs(), num_ports); - assert(&*input_it == stream_balancer->getAuxPorts().first); - input_it++; - assert(input_it == stream_balancer->getInputs().end()); + connectAllInputs(ports, stream_balancer->getInputs(), num_ports); /// Connect auxiliary ports - crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); + crosswise_connection->connectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); if (!filtering_set) { @@ -164,8 +167,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi connect(port, transform->getInputPort()); result_transforms.emplace_back(std::move(transform)); } - output_it++; - assert(output_it == outputs.end()); + assert(output_it == std::prev(outputs.end())); result_transforms.emplace_back(std::move(stream_balancer)); return result_transforms; @@ -194,6 +196,8 @@ void CreateSetAndFilterOnTheFlyStep::updateOutputStream() if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + own_set->setHeader(getColumnSubset(input_streams[0].header, column_names)); + output_stream = input_streams[0]; } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 5ef614bd3bc..fa30807eaee 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include #include @@ -40,17 +41,6 @@ ColumnsWithTypeAndName getColumnsByIndices(const Block & sample_block, const Chu return block.getColumnsWithTypeAndName(); } -std::string formatBytesHumanReadable(size_t bytes) -{ - if (bytes >= 1_GiB) - return fmt::format("{:.2f} GB", static_cast(bytes) / 1_GiB); - if (bytes >= 1_MiB) - return fmt::format("{:.2f} MB", static_cast(bytes) / 1_MiB); - if (bytes >= 1_KiB) - return fmt::format("{:.2f} KB", static_cast(bytes) / 1_KiB); - return fmt::format("{:.2f} B", static_cast(bytes)); -} - } CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( @@ -60,7 +50,6 @@ CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , num_streams(num_streams_) , set(set_) - , log(&Poco::Logger::get(getName())) { } @@ -69,7 +58,7 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() IProcessor::Status status = ISimpleTransform::prepare(); if (!set || status != Status::Finished) - /// Nothing to handle with set + /// Nothing to do with set return status; /// Finalize set @@ -85,7 +74,8 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() set->finishInsert(); set->state = SetWithState::State::Finished; LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), + formatReadableSizeWithBinarySuffix(set->getTotalByteCount())); set.reset(); } else @@ -123,7 +113,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (prev_state == SetWithState::State::Creating) { LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after reading {} rows and using {}", - getDescription(), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + getDescription(), set->getTotalRowCount(), formatReadableSizeWithBinarySuffix(set->getTotalByteCount())); } /// Probaply we need to clear set here, because it's unneeded anymore /// But now `Set` doesn't have such method, so reset pointer in all processors and then it should be freed @@ -137,7 +127,6 @@ FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_ , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , set(set_) - , log(&Poco::Logger::get(getName())) { const auto & header = inputs.front().getHeader(); for (size_t idx : key_column_indices) @@ -193,7 +182,7 @@ void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) size_t result_num_rows = 0; for (auto & col : columns) { - col = col->filter(mask, 0); + col = col->filter(mask, /* negative */ false); result_num_rows = col->size(); } stat.result_rows += result_num_rows; diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index 9b625d0dfaf..d214a310a8c 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -67,7 +67,7 @@ private: /// Set to fill SetWithStatePtr set; - Poco::Logger * log; + Poco::Logger * log = &Poco::Logger::get("CreatingSetsOnTheFlyTransform"); }; /* @@ -108,7 +108,7 @@ private: size_t result_rows = 0; } stat; - Poco::Logger * log; + Poco::Logger * log = &Poco::Logger::get("FilterBySetOnTheFlyTransform"); }; } From 24f62e84860df2236f9523b55afa77953206bd85 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 11:27:08 +0000 Subject: [PATCH 183/582] Throw an error in CreatingSetsOnTheFlyTransform in case of input for finished --- .../Transforms/CreateSetAndFilterOnTheFlyTransform.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index fa30807eaee..1abadf2f727 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -81,11 +81,7 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() else { /// Should not happen because processor inserted before join that reads all the data - /// But let's handle this case just for safety. - set->state = SetWithState::State::Suspended; - LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", - getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); - set.reset(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Processor finished, but not all input was read"); } } From 506bee1e2f7bc8d55883f002abb0a41e44b39516 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 29 Aug 2022 14:31:09 +0000 Subject: [PATCH 184/582] avoid logical errors at some common operation with ColumnObject --- src/Columns/ColumnObject.cpp | 39 +++++++++++++++++++ src/Columns/ColumnObject.h | 24 +++++++----- .../Serializations/SerializationObject.cpp | 7 +++- .../01825_type_json_order_by.reference | 6 +++ .../0_stateless/01825_type_json_order_by.sql | 3 ++ 5 files changed, 69 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/01825_type_json_order_by.reference create mode 100644 tests/queries/0_stateless/01825_type_json_order_by.sql diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index e76862ad988..6a43e239e5c 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -821,6 +822,44 @@ MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.cloneResized(new_size); }); } +void ColumnObject::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const +{ + res.resize(num_rows); + std::iota(res.begin(), res.end(), 0); +} + +void ColumnObject::compareColumn(const IColumn & rhs, size_t rhs_row_num, + PaddedPODArray * row_indexes, PaddedPODArray & compare_results, + int direction, int nan_direction_hint) const +{ + return doCompareColumn(assert_cast(rhs), rhs_row_num, row_indexes, + compare_results, direction, nan_direction_hint); +} + +void ColumnObject::getExtremes(Field & min, Field & max) const +{ + if (num_rows == 0) + { + min = Object(); + max = Object(); + } + else + { + get(0, min); + get(0, max); + } +} + +MutableColumns ColumnObject::scatter(ColumnIndex num_columns, const Selector & selector) const +{ + return scatterImpl(num_columns, selector); +} + +void ColumnObject::gather(ColumnGathererStream & gatherer) +{ + gatherer.gather(*this); +} + const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) const { if (const auto * node = subcolumns.findLeaf(key)) diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 4dc5bb5ce24..9c959ba12e3 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -15,7 +15,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } /// Info that represents a scalar or array field in a decomposed view. @@ -220,6 +220,19 @@ public: ColumnPtr replicate(const Offsets & offsets) const override; MutableColumnPtr cloneResized(size_t new_size) const override; + /// Order of rows in ColumnObject is undefined. + void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const override; + void compareColumn(const IColumn & rhs, size_t rhs_row_num, + PaddedPODArray * row_indexes, PaddedPODArray & compare_results, + int direction, int nan_direction_hint) const override; + + void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} + int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } + void getExtremes(Field & min, Field & max) const override; + + MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; + void gather(ColumnGathererStream & gatherer) override; + /// All other methods throw exception. StringRef getDataAt(size_t) const override { throwMustBeConcrete(); } @@ -232,14 +245,7 @@ public: void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } void expand(const Filter &, bool) override { throwMustBeConcrete(); } - int compareAt(size_t, size_t, const IColumn &, int) const override { throwMustBeConcrete(); } - void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } - void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &) const override { throwMustBeConcrete(); } - void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override { throwMustBeConcrete(); } - MutableColumns scatter(ColumnIndex, const Selector &) const override { throwMustBeConcrete(); } - void gather(ColumnGathererStream &) override { throwMustBeConcrete(); } - void getExtremes(Field &, Field &) const override { throwMustBeConcrete(); } size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } double getRatioOfDefaultRows(double) const override { throwMustBeConcrete(); } void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwMustBeConcrete(); } @@ -247,7 +253,7 @@ public: private: [[noreturn]] static void throwMustBeConcrete() { - throw Exception("ColumnObject must be converted to ColumnTuple before use", ErrorCodes::LOGICAL_ERROR); + throw Exception("ColumnObject must be converted to ColumnTuple before use", ErrorCodes::NOT_IMPLEMENTED); } template diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index cf49fa8798d..c18eda791ea 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -258,7 +258,12 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( auto * state_object = checkAndGetState(state); if (!column_object.isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write non-finalized ColumnObject"); + { + auto finalized_object = column_object.clone(); + assert_cast(*finalized_object).finalize(); + serializeBinaryBulkWithMultipleStreams(*finalized_object, offset, limit, settings, state); + return; + } auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); diff --git a/tests/queries/0_stateless/01825_type_json_order_by.reference b/tests/queries/0_stateless/01825_type_json_order_by.reference new file mode 100644 index 00000000000..611d2835127 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_order_by.reference @@ -0,0 +1,6 @@ +0 +0 +{"k":"v"} + +{"k":"v"} +{"k":"v"} diff --git a/tests/queries/0_stateless/01825_type_json_order_by.sql b/tests/queries/0_stateless/01825_type_json_order_by.sql new file mode 100644 index 00000000000..85f913a8476 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_order_by.sql @@ -0,0 +1,3 @@ +SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON); +SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy; +SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1; From d32492ce8ad600b1693115761cc6b200b8f8381c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 29 Aug 2022 18:26:56 +0200 Subject: [PATCH 185/582] Fix GROUPING function SQL compatibility --- src/Functions/grouping.h | 6 +-- .../02315_grouping_constant_folding.reference | 52 +++++++++---------- 2 files changed, 28 insertions(+), 30 deletions(-) diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index a49e946b2cb..dc630123dcb 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -54,7 +53,7 @@ public: UInt64 value = 0; for (auto index : arguments_indexes) - value = (value << 1) + (checker(set_index, index) ? 1 : 0); + value = (value << 1) + (checker(set_index, index) ? 0 : 1); result_data.push_back(value); } @@ -73,8 +72,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - UInt64 value = (ONE << arguments_indexes.size()) - 1; - return ColumnUInt64::create(input_rows_count, value); + return ColumnUInt64::create(input_rows_count, 0); } }; diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.reference b/tests/queries/0_stateless/02315_grouping_constant_folding.reference index 5aa979b1453..c44fee183da 100644 --- a/tests/queries/0_stateless/02315_grouping_constant_folding.reference +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.reference @@ -1,29 +1,29 @@ -- { echoOn } SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); -1 0 0 3 -1 0 2 3 -1 0 4 3 -1 0 6 3 -1 0 8 3 -1 1 1 3 -1 1 3 3 -1 1 5 3 -1 1 7 3 -1 1 9 3 -5 0 0 2 -5 1 0 2 -10 0 0 0 +1 0 0 0 +1 0 2 0 +1 0 4 0 +1 0 6 0 +1 0 8 0 +1 1 1 0 +1 1 3 0 +1 1 5 0 +1 1 7 0 +1 1 9 0 +5 0 0 1 +5 1 0 1 +10 0 0 3 SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); -1 0 0 3 -1 0 2 3 -1 0 4 3 -1 0 6 3 -1 0 8 3 -1 1 1 3 -1 1 3 3 -1 1 5 3 -1 1 7 3 -1 1 9 3 -5 0 0 2 -5 1 0 2 -10 0 0 0 +1 0 0 0 +1 0 2 0 +1 0 4 0 +1 0 6 0 +1 0 8 0 +1 1 1 0 +1 1 3 0 +1 1 5 0 +1 1 7 0 +1 1 9 0 +5 0 0 1 +5 1 0 1 +10 0 0 3 From e436b4f4ccfd82cdd71dff9c6ae53ceab34d8185 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 29 Aug 2022 22:00:26 +0200 Subject: [PATCH 186/582] Metric for the number of detached parts --- programs/server/Server.cpp | 4 +- src/Interpreters/AsynchronousMetrics.cpp | 76 +++++++++ src/Interpreters/AsynchronousMetrics.h | 14 ++ .../test_detached_parts_metrics/__init__.py | 0 .../asynchronous_metrics_update_period_s.xml | 4 + .../test_detached_parts_metrics/test.py | 156 ++++++++++++++++++ 6 files changed, 253 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_detached_parts_metrics/__init__.py create mode 100644 tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml create mode 100644 tests/integration/test_detached_parts_metrics/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b621a897035..d788270ecf9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -736,7 +736,9 @@ int Server::main(const std::vector & /*args*/) std::vector servers_to_start_before_tables; /// This object will periodically calculate some metrics. AsynchronousMetrics async_metrics( - global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), + global_context, + config().getUInt("asynchronous_metrics_update_period_s", 1), + config().getUInt("asynchronous_heavy_metrics_update_period_s", 120), [&]() -> std::vector { std::vector metrics; diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f9bc22dd110..59f97384626 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -77,9 +77,11 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( ContextPtr global_context_, int update_period_seconds, + int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : WithContext(global_context_) , update_period(update_period_seconds) + , heavy_metric_update_period(heavy_metrics_update_period_seconds) , protocol_server_metrics_func(protocol_server_metrics_func_) , log(&Poco::Logger::get("AsynchronousMetrics")) { @@ -1584,6 +1586,8 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti saveAllArenasMetric(new_values, "muzzy_purged"); #endif + update_heavy_metrics(current_time, update_time, new_values); + /// Add more metrics as you wish. new_values["AsynchronousMetricsCalculationTimeSpent"] = watch.elapsedSeconds(); @@ -1601,4 +1605,76 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti values = new_values; } +void AsynchronousMetrics::update_detached_parts_stats() +{ + DetachedPartsStats current_values{}; + + for (const auto & db : DatabaseCatalog::instance().getDatabases()) + { + if (!db.second->canContainMergeTreeTables()) + continue; + + for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + + if (MergeTreeData * table_merge_tree = dynamic_cast(table.get())) + { + for (const auto & detached_part: table_merge_tree->getDetachedParts()) + { + if (!detached_part.valid_name) + continue; + + if (detached_part.prefix.empty()) + ++current_values.detached_by_user; + + ++current_values.count; + } + } + } + } + + detached_parts_stats = current_values; +} + +void AsynchronousMetrics::update_heavy_metrics(std::chrono::system_clock::time_point current_time, std::chrono::system_clock::time_point update_time, AsynchronousMetricValues & new_values) +{ + const auto time_after_previous_update = current_time - heavy_metric_previous_update_time; + const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run; + + if (update_heavy_metric) + { + heavy_metric_previous_update_time = update_time; + + Stopwatch watch; + + // test shows that listing 100000 entries consuming around 0.15 sec. + update_detached_parts_stats(); + + watch.stop(); + + // normally heavy metrics don't delay the rest of the metrics calculation + // otherwise log the warning message + auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE); + if (watch.elapsedSeconds() > (update_period.count() / 2.)) + log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG); + else if (watch.elapsedSeconds() > (update_period.count() / 4. * 3)) + log_level = std::make_pair(DB::LogsLevel::warning, Poco::Message::PRIO_WARNING); + LOG_IMPL(log, log_level.first, log_level.second, + "Update heavy metrics. " + "Update period {} sec. " + "Update heavy metrics period {} sec. " + "Heavy metrics calculation elapsed: {} sec.", + update_period.count(), + heavy_metric_update_period.count(), + watch.elapsedSeconds()); + + } + + new_values["NumberOfDetachedParts"] = detached_parts_stats.count; + new_values["NumberOfDetachedPartsByUser"] = detached_parts_stats.detached_by_user; +} + } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index e4bcb2890f3..813dd7ec608 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -50,6 +50,7 @@ public: AsynchronousMetrics( ContextPtr global_context_, int update_period_seconds, + int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); ~AsynchronousMetrics(); @@ -64,6 +65,7 @@ public: private: const std::chrono::seconds update_period; + const std::chrono::seconds heavy_metric_update_period; ProtocolServerMetricsFunc protocol_server_metrics_func; mutable std::mutex mutex; @@ -75,6 +77,15 @@ private: /// On first run we will only collect the values to subtract later. bool first_run = true; std::chrono::system_clock::time_point previous_update_time; + std::chrono::system_clock::time_point heavy_metric_previous_update_time; + + struct DetachedPartsStats + { + size_t count; + size_t detached_by_user; + }; + + DetachedPartsStats detached_parts_stats{}; #if defined(OS_LINUX) || defined(OS_FREEBSD) MemoryStatisticsOS memory_stat; @@ -187,6 +198,9 @@ private: void run(); void update(std::chrono::system_clock::time_point update_time); + void update_detached_parts_stats(); + void update_heavy_metrics(std::chrono::system_clock::time_point current_time, std::chrono::system_clock::time_point update_time, AsynchronousMetricValues & new_values); + Poco::Logger * log; }; diff --git a/tests/integration/test_detached_parts_metrics/__init__.py b/tests/integration/test_detached_parts_metrics/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..0a56d734805 --- /dev/null +++ b/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,4 @@ + + 1 + 1 + diff --git a/tests/integration/test_detached_parts_metrics/test.py b/tests/integration/test_detached_parts_metrics/test.py new file mode 100644 index 00000000000..1e3172787be --- /dev/null +++ b/tests/integration/test_detached_parts_metrics/test.py @@ -0,0 +1,156 @@ +import time +import pytest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=["configs/asynchronous_metrics_update_period_s.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def wait_until(call_back, time_to_sleep=0.5, timeout=60): + assert callable(call_back) + start_time = time.time() + deadline = time.time() + timeout + while not call_back() and time.time() < deadline: + time.sleep(time_to_sleep) + assert call_back(), "Elapsed {}".format(time.time() - start_time) + + +def is_different(a, b): + def wrap(): + res_a = a() if callable(a) else a + res_b = b() if callable(b) else b + return res_a != res_b + + return wrap + + +def test_event_time_microseconds_field(started_cluster): + cluster.start() + query_create = """ + CREATE TABLE t + ( + id Int64, + event_time Date + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id; + """ + node1.query(query_create) + + # gives us 2 partitions with 3 parts in total + node1.query("INSERT INTO t VALUES (1, toDate('2022-09-01'));") + node1.query("INSERT INTO t VALUES (2, toDate('2022-08-29'));") + node1.query("INSERT INTO t VALUES (3, toDate('2022-09-01'));") + + query_number_detached_parts_in_async_metric = """ + SELECT value + FROM system.asynchronous_metrics + WHERE metric LIKE 'NumberOfDetachedParts'; + """ + query_number_detached_by_user_parts_in_async_metric = """ + SELECT value + FROM system.asynchronous_metrics + WHERE metric LIKE 'NumberOfDetachedPartsByUser'; + """ + query_count_active_parts = """ + SELECT count(*) FROM system.parts WHERE table = 't' AND active + """ + query_count_detached_parts = """ + SELECT count(*) FROM system.detached_parts WHERE table = 't' + """ + + query_one_partition_name = """ + SELECT name FROM system.parts WHERE table = 't' AND active AND partition = '20220829' + """ + partition_name = node1.query(query_one_partition_name).strip() + + assert 0 == int(node1.query(query_count_detached_parts)) + assert 3 == int(node1.query(query_count_active_parts)) + assert 0 == int(node1.query(query_number_detached_parts_in_async_metric)) + assert 0 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) + + # detach some parts and wait until asynchronous metrics notice it + node1.query("ALTER TABLE t DETACH PARTITION '20220901';") + + assert 2 == int(node1.query(query_count_detached_parts)) + assert 1 == int(node1.query(query_count_active_parts)) + + wait_until( + is_different( + 0, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) + ) + ) + assert 2 == int(node1.query(query_number_detached_parts_in_async_metric)) + assert 2 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) + + # detach the rest parts and wait until asynchronous metrics notice it + node1.query("ALTER TABLE t DETACH PARTITION ALL") + + assert 3 == int(node1.query(query_count_detached_parts)) + assert 0 == int(node1.query(query_count_active_parts)) + + wait_until( + is_different( + 2, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) + ) + ) + assert 3 == int(node1.query(query_number_detached_parts_in_async_metric)) + assert 3 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) + + # inject some data directly and wait until asynchronous metrics notice it + node1.exec_in_container( + [ + "bash", + "-c", + "mkdir /var/lib/clickhouse/data/default/t/detached/unexpected_all_0_0_0", + ], + privileged=True, + ) + + assert 4 == int(node1.query(query_count_detached_parts)) + assert 0 == int(node1.query(query_count_active_parts)) + + wait_until( + is_different( + 3, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) + ) + ) + assert 4 == int(node1.query(query_number_detached_parts_in_async_metric)) + assert 3 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) + + # drop some data directly and wait asynchronous metrics notice it + node1.exec_in_container( + [ + "bash", + "-c", + "rm -rf /var/lib/clickhouse/data/default/t/detached/{}".format( + partition_name + ), + ], + privileged=True, + ) + + assert 3 == int(node1.query(query_count_detached_parts)) + assert 0 == int(node1.query(query_count_active_parts)) + + wait_until( + is_different( + 4, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) + ) + ) + assert 3 == int(node1.query(query_number_detached_parts_in_async_metric)) + assert 2 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) From 7a85a943f1c0cdc0000c6daeb04d855b750f3bbe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Aug 2022 01:11:04 +0000 Subject: [PATCH 187/582] fix close lock on exec, refactoring, comments added --- .../decompressor.cpp | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 73d34f1fe8e..0c42f57367d 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -361,7 +361,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #endif -uint32_t get_inode(const char * self) +uint32_t getInode(const char * self) { std::ifstream maps("/proc/self/maps"); if (maps.fail()) @@ -370,6 +370,10 @@ uint32_t get_inode(const char * self) return 0; } + /// Record example for /proc/self/maps: + /// address perms offset device inode pathname + /// 561a247de000-561a247e0000 r--p 00000000 103:01 1564 /usr/bin/cat + /// see "man 5 proc" for (std::string line; std::getline(maps, line);) { std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -405,7 +409,8 @@ int main(int/* argc*/, char* argv[]) else name = file_path; - uint32_t inode = get_inode(self); + /// get inode of this executable + uint32_t inode = getInode(self); if (inode == 0) { std::cerr << "Unable to obtain inode." << std::endl; @@ -421,6 +426,9 @@ int main(int/* argc*/, char* argv[]) return 1; } + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); + if (lockf(lock, F_LOCK, 0)) { perror("lockf"); @@ -434,7 +442,8 @@ int main(int/* argc*/, char* argv[]) return 1; } - /// if decompression was performed by another process + /// if decompression was performed by another process since this copy was started + /// then file refered by path "self" is already pointing to different inode if (input_info.st_ino != inode) { struct stat lock_info; @@ -444,6 +453,7 @@ int main(int/* argc*/, char* argv[]) return 1; } + /// size 1 of lock file indicates that another decompressor has found active executable if (lock_info.st_size == 1) execv(self, argv); @@ -512,6 +522,8 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { + /// write one byte to the lock in case other copies of compressed are running to indicate that + /// execution should be performed write(lock, "1", 1); execv(self, argv); @@ -521,6 +533,7 @@ int main(int/* argc*/, char* argv[]) return 1; } + /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 ftruncate(lock, 0); printf("No target executable - decompression only was performed.\n"); From a9cab86a73333d27970a0b07e646b9b305b05275 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Aug 2022 01:46:17 +0000 Subject: [PATCH 188/582] typo --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 0c42f57367d..ad4cd5b2f63 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -443,7 +443,7 @@ int main(int/* argc*/, char* argv[]) } /// if decompression was performed by another process since this copy was started - /// then file refered by path "self" is already pointing to different inode + /// then file referred by path "self" is already pointing to different inode if (input_info.st_ino != inode) { struct stat lock_info; From 83cbdef3c68401e0849b77f71e58e1f991917eb2 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 30 Aug 2022 12:26:23 +0800 Subject: [PATCH 189/582] Fix context re-initialization for ThreadPool --- src/Common/OpenTelemetryTraceContext.cpp | 60 ++++++++++--------- src/Common/ThreadPool.cpp | 24 +++++--- src/Common/ThreadPool.h | 74 ++++++++++++++++++++++-- 3 files changed, 118 insertions(+), 40 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index d296ed54422..98856e68c5b 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -283,6 +283,17 @@ TracingContextHolder::TracingContextHolder( this->root_span.start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + /// This object is created to initialize tracing context on a new thread, + /// it's helpful to record the thread_id so that we know the thread switching from the span log + try + { + this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); + } + catch (...) + { + /// It's acceptable that this attribute is not recorded if any exception is raised + } + /// set up trace context on current thread current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; @@ -292,39 +303,32 @@ TracingContextHolder::TracingContextHolder( TracingContextHolder::~TracingContextHolder() { - if (this->root_span.isTraceEnabled()) + if (!this->root_span.isTraceEnabled()) { - try - { - auto shared_span_log = current_thread_trace_context.span_log.lock(); - if (shared_span_log) - { - try - { - this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); - } - catch (...) - { - /// Ignore any exceptions - } - - this->root_span.finish_time_us - = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - - shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); - } - } - catch (...) - { - tryLogCurrentException(__FUNCTION__); - } - - this->root_span.trace_id = UUID(); + return; } + try + { + auto shared_span_log = current_thread_trace_context.span_log.lock(); + if (shared_span_log) + { + this->root_span.finish_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + shared_span_log->add(OpenTelemetrySpanLogElement(this->root_span)); + } + } + catch (...) + { + tryLogCurrentException(__FUNCTION__); + } + + this->root_span.trace_id = UUID(); + if (this->is_context_owner) { - // clear the context on current thread + /// Clear the context on current thread current_thread_trace_context.reset(); } else diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 447b3d98f83..739fd59ecd1 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -87,7 +87,7 @@ void ThreadPoolImpl::setQueueSize(size_t value) template template -ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::optional wait_microseconds) +ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool enable_tracing_context_propagation) { auto on_error = [&](const std::string & reason) { @@ -150,10 +150,18 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti } } - // this scheduleImpl is called in the parent thread, - // the tracing context on this thread is used as parent context for the sub-thread that runs the job - const auto ¤t_thread_context = DB::OpenTelemetry::CurrentContext(); - jobs.emplace(std::move(job), priority, current_thread_context); + if (enable_tracing_context_propagation) + { + /// Tracing context on this thread is used as parent context for the sub-thread that runs the job + const auto ¤t_thread_context = DB::OpenTelemetry::CurrentContext(); + jobs.emplace(std::move(job), priority, current_thread_context); + } + else + { + DB::OpenTelemetry::TracingContextOnThread empty; + jobs.emplace(std::move(job), priority, empty); + } + ++scheduled_jobs; new_job_or_shutdown.notify_one(); } @@ -174,9 +182,9 @@ bool ThreadPoolImpl::trySchedule(Job job, int priority, uint64_t wait_mi } template -void ThreadPoolImpl::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds) +void ThreadPoolImpl::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds, bool enable_tracing_context_propagation) { - scheduleImpl(std::move(job), priority, wait_microseconds); + scheduleImpl(std::move(job), priority, wait_microseconds, enable_tracing_context_propagation); } template @@ -348,7 +356,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ template class ThreadPoolImpl; -template class ThreadPoolImpl; +template class ThreadPoolImpl; std::unique_ptr GlobalThreadPool::the_instance; diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 65992e1b76e..e809c94e263 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -56,7 +56,7 @@ public: bool trySchedule(Job job, int priority = 0, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. - void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0); + void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0, bool enable_tracing_context_propagation = true); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitrary order. @@ -113,7 +113,7 @@ private: std::exception_ptr first_exception; template - ReturnType scheduleImpl(Job job, int priority, std::optional wait_microseconds); + ReturnType scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool enable_tracing_context_propagation = true); void worker(typename std::list::iterator thread_it); @@ -235,7 +235,7 @@ public: return true; } -private: +protected: struct State { /// Should be atomic() because of possible concurrent access between @@ -256,6 +256,72 @@ private: } }; +/// This class is used by ThreadPool only to allocate threads in GlobalThreadPool. +/// Any user code should use ThreadFromGlobalPool instead of this class to schedule a job in a thread. +/// +/// The difference between this class and ThreadFromGlobalPool is that this class disables the tracing context propagation to underlying thread. +/// If the context is propagated, not only the underlying worker will restore context but also the worker of ThreadPool. +/// +/// Since workers of ThreadPool won't exit until the ThreadPool is destroyed, the context restored by underlying worker won't be deleted for a very long time +/// which would cause wrong contexts for jobs for ThreadPool +/// +class Thread4ThreadPool : public ThreadFromGlobalPool +{ +public: + Thread4ThreadPool() = default; + + template + explicit Thread4ThreadPool(Function && func, Args &&... args) + { + state = std::make_shared(); + + /// NOTE: + /// - If this will throw an exception, the destructor won't be called + /// - this pointer cannot be passed in the lambda, since after detach() it will not be valid + GlobalThreadPool::instance().scheduleOrThrow([ + state = state, + func = std::forward(func), + args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture + { + SCOPE_EXIT(state->event.set()); + + state->thread_id = std::this_thread::get_id(); + + /// This moves are needed to destroy function and arguments before exit. + /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. + auto function = std::move(func); + auto arguments = std::move(args); + + /// Thread status holds raw pointer on query context, thus it always must be destroyed + /// before sending signal that permits to join this thread. + DB::ThreadStatus thread_status; + std::apply(function, arguments); + }, + + // default priority + 0, + + // default wait_microseconds + 0, + + /// Disable tracing context propagation on underlying thread pool because ThreadPool already has kept the context in its jobs. + false + ); + } + + Thread4ThreadPool(Thread4ThreadPool && rhs) noexcept + { + *this = std::move(rhs); + } + + Thread4ThreadPool & operator=(Thread4ThreadPool && rhs) noexcept + { + if (initialized()) + abort(); + state = std::move(rhs.state); + return *this; + } +}; /// Recommended thread pool for the case when multiple thread pools are created and destroyed. -using ThreadPool = ThreadPoolImpl; +using ThreadPool = ThreadPoolImpl; From 922a0dee741d50b6eeeca306bb3ef68e3ff9120a Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 30 Aug 2022 15:14:12 +0800 Subject: [PATCH 190/582] Fix style Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 2 +- src/Common/ThreadPool.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 98856e68c5b..f00b6e5b873 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -312,7 +312,7 @@ TracingContextHolder::~TracingContextHolder() { auto shared_span_log = current_thread_trace_context.span_log.lock(); if (shared_span_log) - { + { this->root_span.finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index e809c94e263..5b834beeca1 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -262,7 +262,7 @@ protected: /// The difference between this class and ThreadFromGlobalPool is that this class disables the tracing context propagation to underlying thread. /// If the context is propagated, not only the underlying worker will restore context but also the worker of ThreadPool. /// -/// Since workers of ThreadPool won't exit until the ThreadPool is destroyed, the context restored by underlying worker won't be deleted for a very long time +/// Since workers of ThreadPool won't exit until the ThreadPool is destroyed, the context restored by underlying worker won't be deleted for a very long time /// which would cause wrong contexts for jobs for ThreadPool /// class Thread4ThreadPool : public ThreadFromGlobalPool From c185353bf0a4d0dfd0485e523de286bca5319b0e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Aug 2022 08:06:42 +0000 Subject: [PATCH 191/582] Minor: remove unnecessary ContextAccess::make() method --- src/Access/AccessControl.cpp | 2 +- src/Access/ContextAccess.cpp | 2 +- src/Access/ContextAccess.h | 6 ------ 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 4f62c4ca203..89292fe9272 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -79,7 +79,7 @@ public: /// No user, probably the user has been dropped while it was in the cache. cache.remove(params); } - auto res = ContextAccess::make(access_control, params); + auto res = std::make_shared(access_control, params); res->initialize(); cache.add(params, res); return res; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 59c0a692a84..4e409946666 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -410,7 +410,7 @@ std::shared_ptr ContextAccess::getFullAccess() { static const std::shared_ptr res = [] { - auto full_access = ContextAccess::make(); + auto full_access = std::make_shared(); full_access->is_full_access = true; full_access->access = std::make_shared(AccessRights::getFullAccess()); full_access->access_with_implicit = full_access->access; diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 331b449125e..ce1ea2d1220 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -166,12 +166,6 @@ public: /// without any limitations. This is used for the global context. static std::shared_ptr getFullAccess(); - template - static std::shared_ptr make(Args &&... args) - { - return std::make_shared(std::forward(args)...); - } - ~ContextAccess(); private: From 80365e0448e0e9d40f92b910cd7cd97ac864a957 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 09:19:59 +0000 Subject: [PATCH 192/582] Simplify drop and create --- src/Common/ZooKeeper/ZooKeeper.cpp | 10 +-- src/Storages/StorageKeeperMap.cpp | 127 +++++++---------------------- src/Storages/StorageKeeperMap.h | 3 +- 3 files changed, 35 insertions(+), 105 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b47ab35820e..b95fa22ddb4 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -756,13 +756,13 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); - /// Will try to avoid recursive getChildren calls if child_path probably has no children. - /// It may be extremely slow when path contain a lot of leaf children. - if (!probably_flat) - tryRemoveChildrenRecursive(child_path); - if (likely(keep_child_node.empty() || keep_child_node != children.back())) { + /// Will try to avoid recursive getChildren calls if child_path probably has no children. + /// It may be extremely slow when path contain a lot of leaf children. + if (!probably_flat) + tryRemoveChildrenRecursive(child_path); + batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f3917fe6d7d..362f4bfc152 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -24,6 +24,7 @@ #include #include "Common/Exception.h" +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -81,66 +82,6 @@ std::string_view getBaseName(const std::string_view path) return path.substr(last_slash + 1); } -struct ZooKeeperLock -{ - explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_, bool defer_lock = false) - : lock_path(std::move(lock_path_)), client(std::move(client_)) - { - if (!defer_lock) - lock(); - } - - ~ZooKeeperLock() - { - if (locked) - unlock(); - } - - void lock() - { - assert(!locked); - sequence_path = client->create(std::filesystem::path(lock_path) / "lock-", "", zkutil::CreateMode::EphemeralSequential); - auto node_name = getBaseName(sequence_path); - - while (true) - { - auto children = client->getChildren(lock_path); - assert(!children.empty()); - ::sort(children.begin(), children.end()); - - auto node_it = std::find(children.begin(), children.end(), node_name); - if (node_it == children.begin()) - { - locked = true; - return; - } - - client->waitForDisappear(*(node_it - 1)); - } - } - - void unlock() - { - assert(locked); - client->tryRemove(sequence_path); - } - - // release the ownership and return the path - // for the lock - std::string release() - { - assert(locked); - locked = false; - return std::move(sequence_path); - } - -private: - std::string lock_path; - std::string sequence_path; - zkutil::ZooKeeperPtr client; - bool locked{false}; -}; - } class StorageKeeperMapSink : public SinkToStorage @@ -289,7 +230,6 @@ StorageKeeperMap::StorageKeeperMap( std::filesystem::path root_path_fs{root_path}; auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; - lock_path = metadata_path_fs / "lock"; tables_path = metadata_path_fs / "tables"; table_path = fs::path(tables_path) / toString(table_id.uuid); dropped_path = metadata_path_fs / "dropped"; @@ -425,9 +365,7 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { checkTable(); auto client = getClient(); - - ZooKeeperLock keeper_lock(lock_path, client); - client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + client->tryRemoveChildrenRecursive(root_path, true, getBaseName(metadata_path)); } bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) @@ -457,6 +395,7 @@ bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const break; default: zkutil::KeeperMultiException::check(code, ops, responses); + break; } return completely_removed; } @@ -465,33 +404,38 @@ void StorageKeeperMap::drop() { checkTable(); auto client = getClient(); - ZooKeeperLock lock(lock_path, client); - auto tables_num_str = client->get(tables_path); - auto tables_num = parse(tables_num_str); - --tables_num; + client->remove(table_path); - if (tables_num > 0) + if (!client->getChildren(tables_path).empty()) + return; + + Coordination::Requests ops; + Coordination::Responses responses; + + ops.emplace_back(zkutil::makeRemoveRequest(tables_path, -1)); + ops.emplace_back(zkutil::makeCreateRequest(dropped_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(dropped_lock_path, "", zkutil::CreateMode::Ephemeral)); + + auto code = client->tryMulti(ops, responses); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - client->set(tables_path, toString(tables_num)); + LOG_INFO(log, "Metadata is being removed by another table"); return; } - - client->create(dropped_path, "", zkutil::CreateMode::Persistent); + else if (code == Coordination::Error::ZNOTEMPTY) + { + LOG_WARNING(log, "Another table is using the same path, metadata will not be deleted"); + return; + } + else if (code != Coordination::Error::ZOK) + zkutil::KeeperMultiException::check(code, ops, responses); client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); - Coordination::Requests requests; - // we need to release lock and delete everything at the same time - // so create doesn't take a lock while delete is being run - auto current_lock_path = lock.release(); - requests.push_back(zkutil::makeRemoveRequest(current_lock_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(lock_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(tables_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(dropped_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(metadata_path, -1)); - - client->multi(requests); + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); + removeMetadataNodes(client, metadata_drop_lock); } zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const @@ -519,11 +463,6 @@ std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const return fmt::format("{}/{}", root_path, key); } -const std::string & StorageKeeperMap::lockPath() const -{ - return lock_path; -} - std::optional StorageKeeperMap::isTableValid() const { std::lock_guard lock{init_mutex}; @@ -534,11 +473,7 @@ std::optional StorageKeeperMap::isTableValid() const { // validate all metadata nodes are present Coordination::Requests requests; - - requests.push_back(zkutil::makeCheckRequest(root_path, -1)); - requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); - requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); - requests.push_back(zkutil::makeCheckRequest(tables_path, -1)); + requests.push_back(zkutil::makeCheckRequest(table_path, -1)); Coordination::Responses responses; auto client = getClient(); @@ -549,11 +484,7 @@ std::optional StorageKeeperMap::isTableValid() const { tryLogCurrentException(log); - std::array retriable_errors{ - Coordination::Error::ZCONNECTIONLOSS, Coordination::Error::ZSESSIONEXPIRED, Coordination::Error::ZOPERATIONTIMEOUT}; - bool retriable_error - = std::any_of(retriable_errors.begin(), retriable_errors.end(), [&](const auto error) { return error == e.code; }); - if (!retriable_error) + if (!Coordination::isHardwareError(e.code)) table_is_valid = false; } catch (const Exception &) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 083d4c4c9ed..22c5bb66e1a 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -103,11 +103,10 @@ public: private: bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); - std::string root_path; + const std::string root_path; std::string primary_key; std::string metadata_path; - std::string lock_path; std::string tables_path; std::string table_path; From 8590cc46c4fe1a39605b691804e9169fadd6316c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 30 Aug 2022 18:09:01 +0800 Subject: [PATCH 193/582] implement intersect + except distinct --- src/Core/Settings.h | 4 +- src/Core/SettingsEnums.cpp | 8 +- src/Core/SettingsEnums.h | 10 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../InterpreterSelectIntersectExceptQuery.cpp | 26 +++- .../InterpreterSelectWithUnionQuery.cpp | 4 +- .../NormalizeSelectWithUnionQueryVisitor.cpp | 22 +-- .../NormalizeSelectWithUnionQueryVisitor.h | 2 +- .../RewriteCountDistinctVisitor.cpp | 2 +- .../SelectIntersectExceptQueryVisitor.cpp | 54 +++++-- .../SelectIntersectExceptQueryVisitor.h | 7 +- src/Interpreters/executeQuery.cpp | 4 +- src/Parsers/ASTSelectIntersectExceptQuery.cpp | 18 ++- src/Parsers/ASTSelectIntersectExceptQuery.h | 9 +- src/Parsers/ASTSelectWithUnionQuery.cpp | 21 +-- src/Parsers/ExpressionListParsers.cpp | 32 +++-- src/Parsers/SelectUnionMode.h | 14 +- .../Transforms/IntersectOrExceptTransform.cpp | 5 +- ...04_intersect_except_const_column.reference | 85 +++++++++++ .../02004_intersect_except_const_column.sql | 20 +++ ...ersect_except_distinct_operators.reference | 133 ++++++++++++++++++ ...04_intersect_except_distinct_operators.sql | 58 ++++++++ ...02004_intersect_except_operators.reference | 10 +- ...81_intersect_except_const_column.reference | 61 -------- .../02381_intersect_except_const_column.sql | 13 -- 25 files changed, 472 insertions(+), 152 deletions(-) create mode 100644 tests/queries/0_stateless/02004_intersect_except_const_column.reference create mode 100644 tests/queries/0_stateless/02004_intersect_except_const_column.sql create mode 100644 tests/queries/0_stateless/02004_intersect_except_distinct_operators.reference create mode 100644 tests/queries/0_stateless/02004_intersect_except_distinct_operators.sql delete mode 100644 tests/queries/0_stateless/02381_intersect_except_const_column.reference delete mode 100644 tests/queries/0_stateless/02381_intersect_except_const_column.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index af32c15a867..2dd25feea8d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -554,7 +554,9 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, external_storage_connect_timeout_sec, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connect timeout in seconds. Now supported only for MySQL", 0) \ M(UInt64, external_storage_rw_timeout_sec, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Read/write timeout in seconds. Now supported only for MySQL", 0) \ \ - M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \ + M(SetOperationMode, union_default_mode, SetOperationMode::Unspecified, "Set default mode in UNION query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without mode will throw exception.", 0) \ + M(SetOperationMode, intersect_default_mode, SetOperationMode::ALL, "Set default mode in INTERSECT query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without mode will throw exception.", 0) \ + M(SetOperationMode, except_default_mode, SetOperationMode::ALL, "Set default mode in EXCEPT query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ M(Bool, legacy_column_name_of_tuple_literal, false, "List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 616026520db..daa678c0141 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -114,10 +114,10 @@ IMPLEMENT_SETTING_MULTI_ENUM(MySQLDataTypesSupport, ErrorCodes::UNKNOWN_MYSQL_DA {"date2Date32", MySQLDataTypesSupport::DATE2DATE32}, {"date2String", MySQLDataTypesSupport::DATE2STRING}}) -IMPLEMENT_SETTING_ENUM(UnionMode, ErrorCodes::UNKNOWN_UNION, - {{"", UnionMode::Unspecified}, - {"ALL", UnionMode::ALL}, - {"DISTINCT", UnionMode::DISTINCT}}) +IMPLEMENT_SETTING_ENUM(SetOperationMode, ErrorCodes::UNKNOWN_UNION, + {{"", SetOperationMode::Unspecified}, + {"ALL", SetOperationMode::ALL}, + {"DISTINCT", SetOperationMode::DISTINCT}}) IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS, {{"none", DistributedDDLOutputMode::NONE}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 308d53ff690..518c626fcc0 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -130,14 +130,14 @@ enum class MySQLDataTypesSupport DECLARE_SETTING_MULTI_ENUM(MySQLDataTypesSupport) -enum class UnionMode +enum class SetOperationMode { - Unspecified = 0, // Query UNION without UnionMode will throw exception - ALL, // Query UNION without UnionMode -> SELECT ... UNION ALL SELECT ... - DISTINCT // Query UNION without UnionMode -> SELECT ... UNION DISTINCT SELECT ... + Unspecified = 0, // Query UNION / EXCEPT / INTERSECT without SetOperationMode will throw exception + ALL, // Query UNION / EXCEPT / INTERSECT without SetOperationMode -> SELECT ... UNION / EXCEPT / INTERSECT ALL SELECT ... + DISTINCT // Query UNION / EXCEPT / INTERSECT without SetOperationMode -> SELECT ... UNION / EXCEPT / INTERSECT DISTINCT SELECT ... }; -DECLARE_SETTING_ENUM(UnionMode) +DECLARE_SETTING_ENUM(SetOperationMode) enum class DistributedDDLOutputMode { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 7b6066575ae..1e1b22cb5e0 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -346,7 +346,7 @@ BlockIO InterpreterInsertQuery::execute() const auto & union_modes = select_query.list_of_modes; /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries - const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::ALL; }; + const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; is_trivial_insert_select = std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index d6add3f77a9..a134f7bb913 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -4,11 +4,14 @@ #include #include #include +#include +#include #include #include +#include +#include #include #include -#include #include @@ -134,10 +137,29 @@ void InterpreterSelectIntersectExceptQuery::buildQueryPlan(QueryPlan & query_pla data_streams[i] = plans[i]->getCurrentDataStream(); } - auto max_threads = context->getSettingsRef().max_threads; + const Settings & settings = context->getSettingsRef(); + auto max_threads = settings.max_threads; auto step = std::make_unique(std::move(data_streams), final_operator, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); + const auto & query = query_ptr->as(); + if (query.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_DISTINCT + || query.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT) + { + /// Add distinct transform + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step = std::make_unique( + query_plan.getCurrentDataStream(), + limits, + 0, + result_header.getNames(), + false, + settings.optimize_distinct_in_order); + + query_plan.addStep(std::move(distinct_step)); + } + addAdditionalPostFilter(query_plan); query_plan.addInterpreterContext(context); } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 87a182e70ae..a679b17a5bd 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -317,13 +317,13 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) data_streams[i] = plans[i]->getCurrentDataStream(); } - auto max_threads = context->getSettingsRef().max_threads; + auto max_threads = settings.max_threads; auto union_step = std::make_unique(std::move(data_streams), max_threads); query_plan.unitePlans(std::move(union_step), std::move(plans)); const auto & query = query_ptr->as(); - if (query.union_mode == SelectUnionMode::DISTINCT) + if (query.union_mode == SelectUnionMode::UNION_DISTINCT) { /// Add distinct transform SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index 364828e6126..40c42f7728e 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -59,22 +59,22 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, continue; /// Rewrite UNION Mode - if (union_modes[i] == SelectUnionMode::Unspecified) + if (union_modes[i] == SelectUnionMode::UNION_DEFAULT) { - if (data.union_default_mode == UnionMode::ALL) - union_modes[i] = SelectUnionMode::ALL; - else if (data.union_default_mode == UnionMode::DISTINCT) - union_modes[i] = SelectUnionMode::DISTINCT; + if (data.union_default_mode == SetOperationMode::ALL) + union_modes[i] = SelectUnionMode::UNION_ALL; + else if (data.union_default_mode == SetOperationMode::DISTINCT) + union_modes[i] = SelectUnionMode::UNION_DISTINCT; else throw Exception( "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); } - if (union_modes[i] == SelectUnionMode::ALL) + if (union_modes[i] == SelectUnionMode::UNION_ALL) { if (auto * inner_union = select_list[i + 1]->as(); - inner_union && inner_union->union_mode == SelectUnionMode::ALL) + inner_union && inner_union->union_mode == SelectUnionMode::UNION_ALL) { /// Inner_union is an UNION ALL list, just lift up for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); @@ -85,7 +85,7 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, selects.push_back(select_list[i + 1]); } /// flatten all left nodes and current node to a UNION DISTINCT list - else if (union_modes[i] == SelectUnionMode::DISTINCT) + else if (union_modes[i] == SelectUnionMode::UNION_DISTINCT) { auto distinct_list = std::make_shared(); distinct_list->list_of_selects = std::make_shared(); @@ -96,7 +96,7 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); } - distinct_list->union_mode = SelectUnionMode::DISTINCT; + distinct_list->union_mode = SelectUnionMode::UNION_DISTINCT; distinct_list->is_normalized = true; selects.push_back(std::move(distinct_list)); distinct_found = true; @@ -113,7 +113,7 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, if (!distinct_found) { if (auto * inner_union = select_list[0]->as(); - inner_union && inner_union->union_mode == SelectUnionMode::ALL) + inner_union && inner_union->union_mode == SelectUnionMode::UNION_ALL) { /// Inner_union is an UNION ALL list, just lift it up for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); @@ -136,7 +136,7 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, std::reverse(selects.begin(), selects.end()); ast.is_normalized = true; - ast.union_mode = SelectUnionMode::ALL; + ast.union_mode = SelectUnionMode::UNION_ALL; ast.set_of_modes = std::move(current_set_of_modes); ast.list_of_selects->children = std::move(selects); diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h index d035e90f5a5..e8194f0dfe1 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h @@ -18,7 +18,7 @@ class NormalizeSelectWithUnionQueryMatcher public: struct Data { - const UnionMode & union_default_mode; + const SetOperationMode union_default_mode; }; static void getSelectsFromUnionListNode(ASTPtr ast_select, ASTs & selects); diff --git a/src/Interpreters/RewriteCountDistinctVisitor.cpp b/src/Interpreters/RewriteCountDistinctVisitor.cpp index a7a26a63460..cf28d8abb87 100644 --- a/src/Interpreters/RewriteCountDistinctVisitor.cpp +++ b/src/Interpreters/RewriteCountDistinctVisitor.cpp @@ -62,7 +62,7 @@ void RewriteCountDistinctFunctionMatcher::visit(ASTPtr & ast, Data & /*data*/) auto expr = std::make_shared(); expr->children.emplace_back(cloned_select_query); auto select_with_union = std::make_shared(); - select_with_union->union_mode = SelectUnionMode::Unspecified; + select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; select_with_union->is_normalized = false; select_with_union->list_of_modes.clear(); select_with_union->set_of_modes.clear(); diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 4d695263f26..756a8a48e25 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB @@ -9,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int EXPECTED_ALL_OR_DISTINCT; } /* @@ -27,9 +30,9 @@ void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) visit(*select_union, data); } -void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) +void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data & data) { - const auto & union_modes = ast.list_of_modes; + auto union_modes = std::move(ast.list_of_modes); if (union_modes.empty()) return; @@ -46,14 +49,39 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat selects.pop_back(); SelectUnionModes modes; - for (const auto & mode : union_modes) + for (auto & mode : union_modes) { + /// Rewrite intersect / except mode + if (mode == SelectUnionMode::EXCEPT_DEFAULT) + { + if (data.except_default_mode == SetOperationMode::ALL) + mode = SelectUnionMode::EXCEPT_ALL; + else if (data.except_default_mode == SetOperationMode::DISTINCT) + mode = SelectUnionMode::EXCEPT_DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in EXCEPT query, because setting (except_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + else if (mode == SelectUnionMode::INTERSECT_DEFAULT) + { + if (data.intersect_default_mode == SetOperationMode::ALL) + mode = SelectUnionMode::INTERSECT_ALL; + else if (data.intersect_default_mode == SetOperationMode::DISTINCT) + mode = SelectUnionMode::INTERSECT_DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in INTERSECT query, because setting (intersect_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + switch (mode) { - case SelectUnionMode::EXCEPT: + case SelectUnionMode::EXCEPT_ALL: + case SelectUnionMode::EXCEPT_DISTINCT: { auto left = std::make_shared(); - left->union_mode = SelectUnionMode::ALL; + left->union_mode = mode == SelectUnionMode::EXCEPT_ALL ? SelectUnionMode::UNION_ALL : SelectUnionMode::UNION_DISTINCT; left->list_of_selects = std::make_shared(); left->children.push_back(left->list_of_selects); @@ -66,17 +94,22 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat selects.pop_back(); auto except_node = std::make_shared(); - except_node->final_operator = ASTSelectIntersectExceptQuery::Operator::EXCEPT; + except_node->final_operator = mode == SelectUnionMode::EXCEPT_ALL + ? ASTSelectIntersectExceptQuery::Operator::EXCEPT_ALL + : ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT; except_node->children = {left, right}; children = {except_node}; break; } - case SelectUnionMode::INTERSECT: + case SelectUnionMode::INTERSECT_ALL: + case SelectUnionMode::INTERSECT_DISTINCT: { bool from_except = false; const auto * except_ast = typeid_cast(children.back().get()); - if (except_ast && (except_ast->final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT)) + if (except_ast + && (except_ast->final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_ALL + || except_ast->final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT)) from_except = true; ASTPtr left; @@ -94,7 +127,9 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat selects.pop_back(); auto intersect_node = std::make_shared(); - intersect_node->final_operator = ASTSelectIntersectExceptQuery::Operator::INTERSECT; + intersect_node->final_operator = mode == SelectUnionMode::INTERSECT_ALL + ? ASTSelectIntersectExceptQuery::Operator::INTERSECT_ALL + : ASTSelectIntersectExceptQuery::Operator::INTERSECT_DISTINCT; intersect_node->children = {left, right}; if (from_except) @@ -122,7 +157,6 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat children.emplace_back(std::move(right)); } - ast.union_mode = SelectUnionMode::Unspecified; ast.list_of_selects->children = std::move(children); ast.list_of_modes = std::move(modes); } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h index 5aeb15f70bb..daf6d2ea0df 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.h +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB @@ -17,7 +18,11 @@ class ASTSelectWithUnionQuery; class SelectIntersectExceptQueryMatcher { public: - struct Data {}; + struct Data + { + const SetOperationMode intersect_default_mode; + const SetOperationMode except_default_mode; + }; static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cdddd28adeb..2d669fbb13f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -521,13 +521,13 @@ static std::tuple executeQueryImpl( } { - SelectIntersectExceptQueryVisitor::Data data; + SelectIntersectExceptQueryVisitor::Data data{settings.intersect_default_mode, settings.except_default_mode}; SelectIntersectExceptQueryVisitor{data}.visit(ast); } { /// Normalize SelectWithUnionQuery - NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor::Data data{settings.union_default_mode}; NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); } diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp index 62eeefba385..75fbe2b5280 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -27,7 +27,7 @@ void ASTSelectIntersectExceptQuery::formatImpl(const FormatSettings & settings, if (it != children.begin()) { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT") + << fromOperator(final_operator) << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; } @@ -53,4 +53,20 @@ ASTs ASTSelectIntersectExceptQuery::getListOfSelects() const return selects; } +const char * ASTSelectIntersectExceptQuery::fromOperator(Operator op) +{ + switch (op) + { + case Operator::EXCEPT_ALL: + return "EXCEPT ALL"; + case Operator::EXCEPT_DISTINCT: + return "EXCEPT DISTINCT"; + case Operator::INTERSECT_ALL: + return "INTERSECT ALL"; + case Operator::INTERSECT_DISTINCT: + return "INTERSECT DISTINCT"; + default: + return ""; + } +} } diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index c95944a0c35..ad962fe25e2 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include "Parsers/ExpressionListParsers.h" namespace DB @@ -16,8 +17,10 @@ public: enum class Operator { UNKNOWN, - INTERSECT, - EXCEPT + EXCEPT_ALL, + EXCEPT_DISTINCT, + INTERSECT_ALL, + INTERSECT_DISTINCT, }; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; @@ -26,6 +29,8 @@ public: ASTs getListOfSelects() const; + static const char * fromOperator(Operator op); + /// Final operator after applying visitor. Operator final_operator = Operator::UNKNOWN; }; diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 7718b2dc2de..bc413fbe16d 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -32,14 +33,18 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F auto mode_to_str = [&](auto mode) { - if (mode == SelectUnionMode::ALL) + if (mode == SelectUnionMode::UNION_ALL) return "UNION ALL"; - else if (mode == SelectUnionMode::DISTINCT) + else if (mode == SelectUnionMode::UNION_DISTINCT) return "UNION DISTINCT"; - else if (mode == SelectUnionMode::INTERSECT) - return "INTERSECT"; - else if (mode == SelectUnionMode::EXCEPT) - return "EXCEPT"; + else if (mode == SelectUnionMode::EXCEPT_ALL) + return "EXCEPT ALL"; + else if (mode == SelectUnionMode::EXCEPT_DISTINCT) + return "EXCEPT DISTINCT"; + else if (mode == SelectUnionMode::INTERSECT_ALL) + return "INTERSECT ALL"; + else if (mode == SelectUnionMode::INTERSECT_DISTINCT) + return "INTERSECT DISTINCT"; return ""; }; @@ -77,8 +82,8 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F bool ASTSelectWithUnionQuery::hasNonDefaultUnionMode() const { - return set_of_modes.contains(SelectUnionMode::DISTINCT) || set_of_modes.contains(SelectUnionMode::INTERSECT) - || set_of_modes.contains(SelectUnionMode::EXCEPT); + return set_of_modes.contains(SelectUnionMode::UNION_DISTINCT) || set_of_modes.contains(SelectUnionMode::INTERSECT_DISTINCT) + || set_of_modes.contains(SelectUnionMode::EXCEPT_DISTINCT); } } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index ea51367ee5d..4e88e5c68e6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -139,36 +139,38 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; }; - /// Parse UNION type + /// Parse UNION / INTERSECT / EXCEPT mode + /// The mode can be DEFAULT (unspecified) / DISTINCT / ALL auto parse_separator = [&] { if (s_union_parser.ignore(pos, expected)) { - // SELECT ... UNION ALL SELECT ... if (s_all_parser.check(pos, expected)) - { - union_modes.push_back(SelectUnionMode::ALL); - } - // SELECT ... UNION DISTINCT SELECT ... + union_modes.push_back(SelectUnionMode::UNION_ALL); else if (s_distinct_parser.check(pos, expected)) - { - union_modes.push_back(SelectUnionMode::DISTINCT); - } - // SELECT ... UNION SELECT ... + union_modes.push_back(SelectUnionMode::UNION_DISTINCT); else - { - union_modes.push_back(SelectUnionMode::Unspecified); - } + union_modes.push_back(SelectUnionMode::UNION_DEFAULT); return true; } else if (s_except_parser.check(pos, expected)) { - union_modes.push_back(SelectUnionMode::EXCEPT); + if (s_all_parser.check(pos, expected)) + union_modes.push_back(SelectUnionMode::EXCEPT_ALL); + else if (s_distinct_parser.check(pos, expected)) + union_modes.push_back(SelectUnionMode::EXCEPT_DISTINCT); + else + union_modes.push_back(SelectUnionMode::EXCEPT_DEFAULT); return true; } else if (s_intersect_parser.check(pos, expected)) { - union_modes.push_back(SelectUnionMode::INTERSECT); + if (s_all_parser.check(pos, expected)) + union_modes.push_back(SelectUnionMode::INTERSECT_ALL); + else if (s_distinct_parser.check(pos, expected)) + union_modes.push_back(SelectUnionMode::INTERSECT_DISTINCT); + else + union_modes.push_back(SelectUnionMode::INTERSECT_DEFAULT); return true; } return false; diff --git a/src/Parsers/SelectUnionMode.h b/src/Parsers/SelectUnionMode.h index f4ca858d043..ca3637612aa 100644 --- a/src/Parsers/SelectUnionMode.h +++ b/src/Parsers/SelectUnionMode.h @@ -7,11 +7,15 @@ namespace DB { enum class SelectUnionMode { - Unspecified, - ALL, - DISTINCT, - EXCEPT, - INTERSECT + UNION_DEFAULT, + UNION_ALL, + UNION_DISTINCT, + EXCEPT_DEFAULT, + EXCEPT_ALL, + EXCEPT_DISTINCT, + INTERSECT_DEFAULT, + INTERSECT_ALL, + INTERSECT_DISTINCT }; using SelectUnionModes = std::vector; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 1ac82e99cf2..31a3e304505 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -111,7 +111,10 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = current_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); + filter[i] = (current_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_ALL + || current_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT) + ? !find_result.isFound() + : find_result.isFound(); if (filter[i]) ++new_rows_num; } diff --git a/tests/queries/0_stateless/02004_intersect_except_const_column.reference b/tests/queries/0_stateless/02004_intersect_except_const_column.reference new file mode 100644 index 00000000000..6e3081b017d --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_const_column.reference @@ -0,0 +1,85 @@ +-- { echo } +-- Test: crash the server +SELECT 'fooooo' INTERSECT DISTINCT SELECT 'fooooo'; +fooooo +SELECT 'fooooo' EXCEPT ALL SELECT 'fooooo'; +-- Test: intersect return incorrect result for const column +SELECT 1 FROM numbers(10) INTERSECT SELECT 1 FROM numbers(10); +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +SELECT toString(1) FROM numbers(10) INTERSECT SELECT toString(1) FROM numbers(10); +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +SELECT '1' FROM numbers(10) INTERSECT SELECT '1' FROM numbers(10); +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +SELECT 1 FROM numbers(10) INTERSECT DISTINCT SELECT 1 FROM numbers(10); +1 +SELECT toString(1) FROM numbers(10) INTERSECT DISTINCT SELECT toString(1) FROM numbers(10); +1 +SELECT '1' FROM numbers(10) INTERSECT DISTINCT SELECT '1' FROM numbers(10); +1 +-- Test: except return incorrect result for const column +SELECT 2 FROM numbers(10) EXCEPT SELECT 1 FROM numbers(5); +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +SELECT toString(2) FROM numbers(10) EXCEPT SELECT toString(1) FROM numbers(5); +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +SELECT '2' FROM numbers(10) EXCEPT SELECT '1' FROM numbers(5); +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +SELECT 2 FROM numbers(10) EXCEPT DISTINCT SELECT 1 FROM numbers(5); +2 +SELECT toString(2) FROM numbers(10) EXCEPT DISTINCT SELECT toString(1) FROM numbers(5); +2 +SELECT '2' FROM numbers(10) EXCEPT DISTINCT SELECT '1' FROM numbers(5); +2 diff --git a/tests/queries/0_stateless/02004_intersect_except_const_column.sql b/tests/queries/0_stateless/02004_intersect_except_const_column.sql new file mode 100644 index 00000000000..6fabf34d38a --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_const_column.sql @@ -0,0 +1,20 @@ +-- { echo } +-- Test: crash the server +SELECT 'fooooo' INTERSECT DISTINCT SELECT 'fooooo'; +SELECT 'fooooo' EXCEPT ALL SELECT 'fooooo'; + +-- Test: intersect return incorrect result for const column +SELECT 1 FROM numbers(10) INTERSECT SELECT 1 FROM numbers(10); +SELECT toString(1) FROM numbers(10) INTERSECT SELECT toString(1) FROM numbers(10); +SELECT '1' FROM numbers(10) INTERSECT SELECT '1' FROM numbers(10); +SELECT 1 FROM numbers(10) INTERSECT DISTINCT SELECT 1 FROM numbers(10); +SELECT toString(1) FROM numbers(10) INTERSECT DISTINCT SELECT toString(1) FROM numbers(10); +SELECT '1' FROM numbers(10) INTERSECT DISTINCT SELECT '1' FROM numbers(10); + +-- Test: except return incorrect result for const column +SELECT 2 FROM numbers(10) EXCEPT SELECT 1 FROM numbers(5); +SELECT toString(2) FROM numbers(10) EXCEPT SELECT toString(1) FROM numbers(5); +SELECT '2' FROM numbers(10) EXCEPT SELECT '1' FROM numbers(5); +SELECT 2 FROM numbers(10) EXCEPT DISTINCT SELECT 1 FROM numbers(5); +SELECT toString(2) FROM numbers(10) EXCEPT DISTINCT SELECT toString(1) FROM numbers(5); +SELECT '2' FROM numbers(10) EXCEPT DISTINCT SELECT '1' FROM numbers(5); \ No newline at end of file diff --git a/tests/queries/0_stateless/02004_intersect_except_distinct_operators.reference b/tests/queries/0_stateless/02004_intersect_except_distinct_operators.reference new file mode 100644 index 00000000000..49529aa7683 --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_distinct_operators.reference @@ -0,0 +1,133 @@ +-- { echo } + +set intersect_default_mode = 'DISTINCT'; +set except_default_mode = 'DISTINCT'; +select 1 intersect select 1; +1 +select 2 intersect select 1; +select 1 except select 1; +select 2 except select 1; +2 +select 5 from numbers(20) intersect select number from numbers(5, 5); +5 +select number from numbers(10) except select number from numbers(5); +5 +6 +7 +8 +9 +select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); +0 10 +1 11 +2 12 +3 13 +4 14 +select 1 except select 2 intersect select 1; +1 +select 1 except select 2 intersect select 2; +1 +select 1 intersect select 1 except select 2; +1 +select 1 intersect select 1 except select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; +1 +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +1 +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1; +select number%3 from numbers(10) except select 1; +0 +2 +select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +select * from (select 1 intersect select 1); +1 +with (select number from numbers(10) intersect select 5) as a select a * 10; +50 +with (select 5 except select 1) as a select a except select 5; +with (select number from numbers(10) intersect select 5) as a select a intersect select 1; +with (select number from numbers(10) intersect select 5) as a select a except select 1; +5 +select count() from (select number from numbers(10) except select 5); +9 +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); +600000 +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +20 +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10)); +30 +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); +200000 +select count() from (select 1 intersect select 1) limit 100; +1 +select count() from (select 1 except select 2) limit 100; +1 +with (select count() from (select 1 union distinct select 2 except select 1)) as max +select count() from (select 1 union all select max) limit 100; +2 +select 1 union all select 1 intersect select 1; +1 +1 +select 1 union all select 1 intersect select 2; +1 +select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; +1 +2 +4 +5 +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; +1 +2 +3 +5 +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; +2 +3 +5 +select 1 intersect (select 1 except select 2); +1 +select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); +explain syntax select 1 intersect select 1; +SELECT 1 +INTERSECT DISTINCT +SELECT 1 +explain syntax select 1 except select 1; +SELECT 1 +EXCEPT DISTINCT +SELECT 1 +explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +SELECT 1 +UNION ALL +SELECT 2 +EXCEPT DISTINCT +SELECT 2 +EXCEPT DISTINCT +SELECT 1 +UNION ALL +SELECT 1 +EXCEPT DISTINCT +SELECT 4 +set limit=1; +select 1 intersect select 1; +1 +(((select 1) intersect select 1)); +1 diff --git a/tests/queries/0_stateless/02004_intersect_except_distinct_operators.sql b/tests/queries/0_stateless/02004_intersect_except_distinct_operators.sql new file mode 100644 index 00000000000..efb48e59f3d --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_distinct_operators.sql @@ -0,0 +1,58 @@ +-- { echo } + +set intersect_default_mode = 'DISTINCT'; +set except_default_mode = 'DISTINCT'; + +select 1 intersect select 1; +select 2 intersect select 1; +select 1 except select 1; +select 2 except select 1; + +select 5 from numbers(20) intersect select number from numbers(5, 5); +select number from numbers(10) except select number from numbers(5); +select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); + +select 1 except select 2 intersect select 1; +select 1 except select 2 intersect select 2; +select 1 intersect select 1 except select 2; +select 1 intersect select 1 except select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1; + +select number%3 from numbers(10) except select 1; +select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); + +select * from (select 1 intersect select 1); +with (select number from numbers(10) intersect select 5) as a select a * 10; +with (select 5 except select 1) as a select a except select 5; +with (select number from numbers(10) intersect select 5) as a select a intersect select 1; +with (select number from numbers(10) intersect select 5) as a select a except select 1; +select count() from (select number from numbers(10) except select 5); +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10)); +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); + +select count() from (select 1 intersect select 1) limit 100; +select count() from (select 1 except select 2) limit 100; +with (select count() from (select 1 union distinct select 2 except select 1)) as max +select count() from (select 1 union all select max) limit 100; + +select 1 union all select 1 intersect select 1; +select 1 union all select 1 intersect select 2; +select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; + +select 1 intersect (select 1 except select 2); +select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); + +explain syntax select 1 intersect select 1; +explain syntax select 1 except select 1; +explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; + +set limit=1; +select 1 intersect select 1; +(((select 1) intersect select 1)); diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index a96a6bc7264..0acb550c1e8 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -116,23 +116,23 @@ select 1 union all select 2 except (select 2 except select 1 union all select 1 select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); explain syntax select 1 intersect select 1; SELECT 1 -INTERSECT +INTERSECT ALL SELECT 1 explain syntax select 1 except select 1; SELECT 1 -EXCEPT +EXCEPT ALL SELECT 1 explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; SELECT 1 UNION ALL SELECT 2 -EXCEPT +EXCEPT ALL SELECT 2 -EXCEPT +EXCEPT ALL SELECT 1 UNION ALL SELECT 1 -EXCEPT +EXCEPT ALL SELECT 4 set limit=1; select 1 intersect select 1; diff --git a/tests/queries/0_stateless/02381_intersect_except_const_column.reference b/tests/queries/0_stateless/02381_intersect_except_const_column.reference deleted file mode 100644 index 290835b412e..00000000000 --- a/tests/queries/0_stateless/02381_intersect_except_const_column.reference +++ /dev/null @@ -1,61 +0,0 @@ -fooooo -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 diff --git a/tests/queries/0_stateless/02381_intersect_except_const_column.sql b/tests/queries/0_stateless/02381_intersect_except_const_column.sql deleted file mode 100644 index b10f913dd1e..00000000000 --- a/tests/queries/0_stateless/02381_intersect_except_const_column.sql +++ /dev/null @@ -1,13 +0,0 @@ --- Test: crash the server -SELECT 'fooooo' INTERSECT SELECT 'fooooo'; -SELECT 'fooooo' EXCEPT SELECT 'fooooo'; - --- Test: intersect return incorrect result for const column -SELECT 1 FROM numbers(10) INTERSECT SELECT 1 FROM numbers(10); -SELECT toString(1) FROM numbers(10) INTERSECT SELECT toString(1) FROM numbers(10); -SELECT '1' FROM numbers(10) INTERSECT SELECT '1' FROM numbers(10); - --- Test: except return incorrect result for const column -SELECT 2 FROM numbers(10) EXCEPT SELECT 1 FROM numbers(5); -SELECT toString(2) FROM numbers(10) EXCEPT SELECT toString(1) FROM numbers(5); -SELECT '2' FROM numbers(10) EXCEPT SELECT '1' FROM numbers(5); \ No newline at end of file From 30429d75b383c38a61f75e8463c377428f2fbfd2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 10:22:35 +0000 Subject: [PATCH 194/582] Add basic tests for KeeperMap --- .../queries/0_stateless/02381_keeper_map.sql | 39 ------------------- ...p.reference => 02416_keeper_map.reference} | 0 .../queries/0_stateless/02416_keeper_map.sql | 37 ++++++++++++++++++ .../02417_keeper_map_create_drop.reference | 11 ++++++ .../02417_keeper_map_create_drop.sql | 20 ++++++++++ 5 files changed, 68 insertions(+), 39 deletions(-) delete mode 100644 tests/queries/0_stateless/02381_keeper_map.sql rename tests/queries/0_stateless/{02381_keeper_map.reference => 02416_keeper_map.reference} (100%) create mode 100644 tests/queries/0_stateless/02416_keeper_map.sql create mode 100644 tests/queries/0_stateless/02417_keeper_map_create_drop.reference create mode 100644 tests/queries/0_stateless/02417_keeper_map_create_drop.sql diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql deleted file mode 100644 index 903d530299c..00000000000 --- a/tests/queries/0_stateless/02381_keeper_map.sql +++ /dev/null @@ -1,39 +0,0 @@ --- Tags: no-fasttest - -SET database_atomic_wait_for_drop_and_detach_synchronously = 1; - -DROP TABLE IF EXISTS 02381_test; - -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } -CREATE TABLE 02381_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); - -DROP TABLE IF EXISTS 02381_test; -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); - -INSERT INTO 02381_test SELECT '1_1', number FROM numbers(10000); -SELECT COUNT(1) == 1 FROM 02381_test; - -INSERT INTO 02381_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT COUNT(1) == 10000 FROM 02381_test; -SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02381_test LIMIT 32 SETTINGS max_block_size = 1); -SELECT SUM(value) == 1 + 99 + 900 FROM 02381_test WHERE key IN ('1_1', '99_1', '900_1'); - -DROP TABLE IF EXISTS 02381_test; -DROP TABLE IF EXISTS 02381_test_memory; - -CREATE TABLE 02381_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); -CREATE TABLE 02381_test_memory AS 02381_test Engine = Memory; - -INSERT INTO 02381_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; - -INSERT INTO 02381_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; - -SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test_memory) B USING a ORDER BY a; - -TRUNCATE TABLE 02381_test; -SELECT 0 == COUNT(1) FROM 02381_test; - -DROP TABLE IF EXISTS 02381_test; -DROP TABLE IF EXISTS 02381_test_memory; diff --git a/tests/queries/0_stateless/02381_keeper_map.reference b/tests/queries/0_stateless/02416_keeper_map.reference similarity index 100% rename from tests/queries/0_stateless/02381_keeper_map.reference rename to tests/queries/0_stateless/02416_keeper_map.reference diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql new file mode 100644 index 00000000000..6688f2b4659 --- /dev/null +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -0,0 +1,37 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS 02416_test SYNC; + +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +DROP TABLE IF EXISTS 02416_test SYNC; +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +INSERT INTO 02416_test SELECT '1_1', number FROM numbers(10000); +SELECT COUNT(1) == 1 FROM 02416_test; + +INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT COUNT(1) == 10000 FROM 02416_test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02416_test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', '900_1'); + +DROP TABLE IF EXISTS 02416_test SYNC; +DROP TABLE IF EXISTS 02416_test_memory; + +CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); +CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory; + +INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + +INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test_memory) B USING a ORDER BY a; + +TRUNCATE TABLE 02416_test; +SELECT 0 == COUNT(1) FROM 02416_test; + +DROP TABLE IF EXISTS 02416_test SYNC; +DROP TABLE IF EXISTS 02416_test_memory; diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.reference b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference new file mode 100644 index 00000000000..dda215c9659 --- /dev/null +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference @@ -0,0 +1,11 @@ +1 11 +2 22 +------ +1 11 +2 22 +------ +1 11 +2 22 +------ +1 11 +2 22 diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql new file mode 100644 index 00000000000..fff1d78d589 --- /dev/null +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql @@ -0,0 +1,20 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS 02417_test SYNC; + +CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +INSERT INTO 02417_test VALUES (1, 11); +SELECT * FROM 02417_test ORDER BY key; +SELECT '------'; + +CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +INSERT INTO 02417_test_another VALUES (2, 22); +SELECT * FROM 02417_test_another ORDER BY key; +SELECT '------'; +SELECT * FROM 02417_test ORDER BY key; +SELECT '------'; + +DROP TABLE 02417_test SYNC; +SELECT * FROM 02417_test_another ORDER BY key; + +DROP TABLE 02417_test_another SYNC; From 745a06895935c46da9db3615b3eaa73f8ce46fec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 30 Aug 2022 13:41:48 +0200 Subject: [PATCH 195/582] Update 01825_type_json_order_by.sql --- tests/queries/0_stateless/01825_type_json_order_by.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01825_type_json_order_by.sql b/tests/queries/0_stateless/01825_type_json_order_by.sql index 85f913a8476..1d5768fe0b4 100644 --- a/tests/queries/0_stateless/01825_type_json_order_by.sql +++ b/tests/queries/0_stateless/01825_type_json_order_by.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON); SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy; SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1; From ee691961d9990fc78b5cf43cde9e80a0438c4a1c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 30 Aug 2022 11:43:27 +0000 Subject: [PATCH 196/582] fix build --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c5f546a9c36..c024e5da7b5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -242,7 +242,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( auto sorting_step = std::make_unique( projection_plan->getCurrentDataStream(), - query_info.projection->input_order_info->order_key_prefix_descr, + query_info.projection->input_order_info->sort_description_for_merging, output_order_descr, settings.max_block_size, limit); From 0f6f3c73b0fc4540bf0af87620e77f27f87eba25 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 Aug 2022 11:57:28 +0000 Subject: [PATCH 197/582] Minor fix --- src/Core/Settings.h | 2 +- .../Transforms/CreateSetAndFilterOnTheFlyTransform.cpp | 5 +++++ tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql | 2 ++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84bb086fbfb..eafc41b8861 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ + M(UInt64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 1abadf2f727..4278eb8e8b2 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql index a418a06803f..4ed6d965292 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql @@ -7,6 +7,8 @@ USING (key); SET join_algorithm = 'full_sorting_merge'; +SET max_rows_in_set_to_optimize_join = 0; + EXPLAIN actions=0, description=0, header=1 SELECT * FROM ( SELECT 'key2' AS key ) AS s1 JOIN ( SELECT 'key1' AS key, '1' AS value UNION ALL SELECT 'key2' AS key, '1' AS value ) AS s2 From 92f88c1429be0ee9c352439cb140aea65a88dc83 Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Tue, 30 Aug 2022 20:02:47 +0800 Subject: [PATCH 198/582] update datasketches-cpp --- contrib/datasketches-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/datasketches-cpp b/contrib/datasketches-cpp index 7d73d7610db..7abd49bb2e7 160000 --- a/contrib/datasketches-cpp +++ b/contrib/datasketches-cpp @@ -1 +1 @@ -Subproject commit 7d73d7610db31d4e1ecde0fb3a7ee90ef371207f +Subproject commit 7abd49bb2e72bf9a5029993d31dcb1872da88292 From bf03dfc661a336875b19c440a917ae120ad95d19 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 13:41:13 +0000 Subject: [PATCH 199/582] Fix create --- src/Storages/StorageKeeperMap.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 362f4bfc152..e6090ccc605 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -289,7 +289,10 @@ StorageKeeperMap::StorageKeeperMap( auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); if (code == Coordination::Error::ZOK) + { + table_is_valid = true; return; + } if (code == Coordination::Error::ZNONODE) LOG_INFO(log, "Metadata nodes were deleted in background, will retry"); From 3c098bd0cb7303aa8968ad995303608a53719262 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 13:41:36 +0000 Subject: [PATCH 200/582] Add integration test for KeeperMap --- tests/integration/test_keeper_map/__init__.py | 1 + tests/integration/test_keeper_map/test.py | 172 ++++++++++++++++++ 2 files changed, 173 insertions(+) create mode 100644 tests/integration/test_keeper_map/__init__.py create mode 100644 tests/integration/test_keeper_map/test.py diff --git a/tests/integration/test_keeper_map/__init__.py b/tests/integration/test_keeper_map/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_map/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py new file mode 100644 index 00000000000..025439f44db --- /dev/null +++ b/tests/integration/test_keeper_map/test.py @@ -0,0 +1,172 @@ +import multiprocessing +import pytest +from time import sleep +import random +from itertools import count +from sys import stdout + +from multiprocessing import Pool + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain +from helpers.network import PartitionManager + +test_recover_staled_replica_run = 1 + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=[], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def get_genuine_zk(): + print("Zoo1", cluster.get_instance_ip("zoo1")) + return cluster.get_kazoo_client("zoo1") + + +def remove_children(client, path): + children = client.get_children(path) + + for child in children: + child_path = f"{path}/{child}" + remove_children(client, child_path) + client.delete(child_path) + + +def test_create_keeper_map(started_cluster): + assert "Path '/test1' doesn't exist" in node.query_and_get_error( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1', 0) PRIMARY KEY(key);" + ) + + node.query( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + zk_client = get_genuine_zk() + + def assert_children_size(expected_size): + assert len(zk_client.get_children("/test1")) == expected_size + + assert_children_size(1) + + node.query("INSERT INTO test_keeper_map VALUES (1, 11)") + assert_children_size(2) + + node.query( + "CREATE TABLE test_keeper_map_another (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + assert_children_size(2) + node.query("INSERT INTO test_keeper_map_another VALUES (1, 11)") + assert_children_size(2) + + node.query("INSERT INTO test_keeper_map_another VALUES (2, 22)") + assert_children_size(3) + + node.query("DROP TABLE test_keeper_map SYNC") + assert_children_size(3) + + node.query("DROP TABLE test_keeper_map_another SYNC") + assert_children_size(0) + + zk_client.stop() + + +def create_drop_loop(index, stop_event): + table_name = f"test_keeper_map_{index}" + + for i in count(0, 1): + if stop_event.is_set(): + return + + stdout.write(f"Trying with {i} for {index}\n") + node.query( + f"CREATE TABLE {table_name} (key UInt64, value UInt64) ENGINE = KeeperMap('/test') PRIMARY KEY(key);" + ) + node.query(f"INSERT INTO {table_name} VALUES ({index}, {i})") + result = node.query(f"SELECT value FROM {table_name} WHERE key = {index}") + assert result.strip() == str(i) + node.query(f"DROP TABLE {table_name} SYNC") + + +def test_create_drop_keeper_map_concurrent(started_cluster): + pool = Pool() + manager = multiprocessing.Manager() + stop_event = manager.Event() + results = [] + for i in range(multiprocessing.cpu_count()): + sleep(0.2) + results.append( + pool.apply_async( + create_drop_loop, + args=( + i, + stop_event, + ), + ) + ) + + sleep(60) + stop_event.set() + + for result in results: + result.get() + + pool.close() + + client = get_genuine_zk() + assert len(client.get_children("/test")) == 0 + + +def test_keeper_map_without_zk(started_cluster): + def assert_keeper_exception_after_partition(query): + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + error = node.query_and_get_error(query) + assert "Coordination::Exception" in error + + assert_keeper_exception_after_partition( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + + node.query( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + + assert_keeper_exception_after_partition( + "INSERT INTO test_keeper_map VALUES (1, 11)" + ) + node.query("INSERT INTO test_keeper_map VALUES (1, 11)") + + assert_keeper_exception_after_partition("SELECT * FROM test_keeper_map") + node.query("SELECT * FROM test_keeper_map") + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + node.restart_clickhouse(60) + error = node.query_and_get_error("SELECT * FROM test_keeper_map") + assert "Failed to activate table because of connection issues" in error + + node.query("SELECT * FROM test_keeper_map") + + client = get_genuine_zk() + remove_children(client, "/test1") + node.restart_clickhouse(60) + error = node.query_and_get_error("SELECT * FROM test_keeper_map") + assert "Failed to activate table because of invalid metadata in ZooKeeper" in error + + node.query("DETACH TABLE test_keeper_map") + + client.stop() From d498213973ef1d1ca7a1ac35979e4ab17fa7c248 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 13:50:02 +0000 Subject: [PATCH 201/582] Style fixes --- src/Storages/StorageKeeperMap.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index e6090ccc605..dd55586208e 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -296,11 +296,11 @@ StorageKeeperMap::StorageKeeperMap( if (code == Coordination::Error::ZNONODE) LOG_INFO(log, "Metadata nodes were deleted in background, will retry"); - else + else throw Coordination::Exception(code, table_path); } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or because of wrong root_path ({})", root_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, because it is removed concurrently or because of wrong root_path ({})", root_path); } @@ -589,14 +589,12 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 4) + if (engine_args.empty() || engine_args.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-5 arguments:\n" + "Storage KeeperMap requires 1-2 arguments:\n" "root_path: path in the Keeper where the values will be stored (required)\n" - "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", default_host); auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); From 86872b2307951ac8190de3078fd734503f1d9e12 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 30 Aug 2022 18:10:26 +0200 Subject: [PATCH 202/582] Fix locking while writing backup in multiple threads. --- src/Backups/BackupImpl.cpp | 10 +++++++--- src/Backups/BackupImpl.h | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index f6442545f48..d20ad04fea4 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -682,8 +682,6 @@ ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(BackupEntryPtr entry, si void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) { - - std::lock_guard lock{mutex}; if (open_mode != OpenMode::WRITE) throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR); @@ -802,7 +800,12 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// or have only prefix of it in previous backup. Let's go long path. info.data_file_name = info.file_name; - info.archive_suffix = current_archive_suffix; + + if (use_archives) + { + std::lock_guard lock{mutex}; + info.archive_suffix = current_archive_suffix; + } bool is_data_file_required; coordination->addFileInfo(info, is_data_file_required); @@ -838,6 +841,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (use_archives) { LOG_TRACE(log, "Adding file {} to archive", adjusted_path); + std::lock_guard lock{mutex}; String archive_suffix = current_archive_suffix; bool next_suffix = false; if (current_archive_suffix.empty() && is_internal_backup) diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 525aec2fcd6..e539239d3ef 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -130,7 +130,7 @@ private: std::pair> archive_writers[2]; String current_archive_suffix; String lock_file_name; - size_t num_files_written = 0; + std::atomic num_files_written = 0; bool writing_finalized = false; const Poco::Logger * log; }; From 77d741dc259633be99875c17e50d9d53367e96b2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 30 Aug 2022 18:58:13 +0200 Subject: [PATCH 203/582] Add comments. --- src/Backups/BackupImpl.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index d20ad04fea4..a68d438fb9d 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -821,9 +821,11 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// if source and destination are compatible if (!use_archives && info.base_size == 0 && writer->supportNativeCopy(reader_description)) { - + /// Should be much faster than writing data through server. LOG_TRACE(log, "Will copy file {} using native copy", adjusted_path); - /// Should be much faster than writing data through server + + /// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow. + writer->copyFileNative(entry->tryGetDiskIfExists(), entry->getFilePath(), info.data_file_name); } else @@ -841,7 +843,11 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (use_archives) { LOG_TRACE(log, "Adding file {} to archive", adjusted_path); + + /// An archive must be written strictly in one thread, so it's correct to lock the mutex for all the time we're writing the file + /// to the archive. std::lock_guard lock{mutex}; + String archive_suffix = current_archive_suffix; bool next_suffix = false; if (current_archive_suffix.empty() && is_internal_backup) @@ -863,6 +869,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) } else { + /// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow. writer->copyFileThroughBuffer(std::move(read_buffer), info.data_file_name); } } From 19c4442933e54de1bb708a11d769f6693bb45163 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Aug 2022 19:38:57 +0200 Subject: [PATCH 204/582] Fix --- src/Common/FileCache.cpp | 11 ++++++++++- src/Common/FileCache.h | 3 +++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 0ac047e0818..a94188305dd 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -82,6 +82,8 @@ void FileCache::initialize() std::lock_guard cache_lock(mutex); if (!is_initialized) { + status_file = make_unique(fs::path(cache_base_path) / "status", StatusFile::write_full_info); + if (fs::exists(cache_base_path)) { try @@ -933,12 +935,19 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator key_prefix_it{cache_base_path}; for (; key_prefix_it != fs::directory_iterator(); ++key_prefix_it) { + if (!key_prefix_it->is_directory()) + { + if (key_prefix_it->path().filename() != "status") + LOG_DEBUG(log, "Unexpected file {} (not a directory), will skip it", key_prefix_it->path().string()); + continue; + } + fs::directory_iterator key_it{key_prefix_it->path()}; for (; key_it != fs::directory_iterator(); ++key_it) { if (!key_it->is_directory()) { - LOG_WARNING(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); + LOG_DEBUG(log, "Unexpected file {} (not a directory), will skip it", key_it->path().string()); continue; } diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 1690690d102..0e03c5e5127 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -18,6 +18,8 @@ #include #include #include +#include + namespace DB { @@ -141,6 +143,7 @@ private: Poco::Logger * log; bool is_initialized = false; + std::unique_ptr status_file; mutable std::mutex mutex; From 9bb4f556c36fa4b6f32467e9ef11dd0c3d29d284 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Aug 2022 20:18:08 +0200 Subject: [PATCH 205/582] work on remarks on the review --- src/Interpreters/AsynchronousMetrics.cpp | 16 ++--- src/Interpreters/AsynchronousMetrics.h | 17 ++--- .../test_detached_parts_metrics/test.py | 63 ++++++------------- 3 files changed, 38 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 59f97384626..05acbb51c36 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -565,7 +565,7 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif -void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_time) +void AsynchronousMetrics::update(TimePoint update_time) { Stopwatch watch; @@ -1586,7 +1586,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti saveAllArenasMetric(new_values, "muzzy_purged"); #endif - update_heavy_metrics(current_time, update_time, new_values); + updateHeavyMetricsIfNeeded(current_time, update_time, new_values); /// Add more metrics as you wish. @@ -1605,7 +1605,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti values = new_values; } -void AsynchronousMetrics::update_detached_parts_stats() +void AsynchronousMetrics::updateDetachedPartsStats() { DetachedPartsStats current_values{}; @@ -1639,7 +1639,7 @@ void AsynchronousMetrics::update_detached_parts_stats() detached_parts_stats = current_values; } -void AsynchronousMetrics::update_heavy_metrics(std::chrono::system_clock::time_point current_time, std::chrono::system_clock::time_point update_time, AsynchronousMetricValues & new_values) +void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values) { const auto time_after_previous_update = current_time - heavy_metric_previous_update_time; const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run; @@ -1650,12 +1650,12 @@ void AsynchronousMetrics::update_heavy_metrics(std::chrono::system_clock::time_p Stopwatch watch; - // test shows that listing 100000 entries consuming around 0.15 sec. - update_detached_parts_stats(); + // Test shows that listing 100000 entries consuming around 0.15 sec. + updateDetachedPartsStats(); watch.stop(); - // normally heavy metrics don't delay the rest of the metrics calculation + // Normally heavy metrics don't delay the rest of the metrics calculation // otherwise log the warning message auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE); if (watch.elapsedSeconds() > (update_period.count() / 2.)) @@ -1674,7 +1674,7 @@ void AsynchronousMetrics::update_heavy_metrics(std::chrono::system_clock::time_p } new_values["NumberOfDetachedParts"] = detached_parts_stats.count; - new_values["NumberOfDetachedPartsByUser"] = detached_parts_stats.detached_by_user; + new_values["NumberOfDetachedByUserParts"] = detached_parts_stats.detached_by_user; } } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 813dd7ec608..6e32bdb43b8 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -64,8 +64,11 @@ public: AsynchronousMetricValues getValues() const; private: - const std::chrono::seconds update_period; - const std::chrono::seconds heavy_metric_update_period; + using Duration = std::chrono::seconds; + using TimePoint = std::chrono::system_clock::time_point; + + const Duration update_period; + const Duration heavy_metric_update_period; ProtocolServerMetricsFunc protocol_server_metrics_func; mutable std::mutex mutex; @@ -76,8 +79,8 @@ private: /// Some values are incremental and we have to calculate the difference. /// On first run we will only collect the values to subtract later. bool first_run = true; - std::chrono::system_clock::time_point previous_update_time; - std::chrono::system_clock::time_point heavy_metric_previous_update_time; + TimePoint previous_update_time; + TimePoint heavy_metric_previous_update_time; struct DetachedPartsStats { @@ -196,10 +199,10 @@ private: std::unique_ptr thread; void run(); - void update(std::chrono::system_clock::time_point update_time); + void update(TimePoint update_time); - void update_detached_parts_stats(); - void update_heavy_metrics(std::chrono::system_clock::time_point current_time, std::chrono::system_clock::time_point update_time, AsynchronousMetricValues & new_values); + void updateDetachedPartsStats(); + void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values); Poco::Logger * log; }; diff --git a/tests/integration/test_detached_parts_metrics/test.py b/tests/integration/test_detached_parts_metrics/test.py index 1e3172787be..62b70ebd430 100644 --- a/tests/integration/test_detached_parts_metrics/test.py +++ b/tests/integration/test_detached_parts_metrics/test.py @@ -1,6 +1,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) @@ -19,24 +20,6 @@ def started_cluster(): cluster.shutdown() -def wait_until(call_back, time_to_sleep=0.5, timeout=60): - assert callable(call_back) - start_time = time.time() - deadline = time.time() + timeout - while not call_back() and time.time() < deadline: - time.sleep(time_to_sleep) - assert call_back(), "Elapsed {}".format(time.time() - start_time) - - -def is_different(a, b): - def wrap(): - res_a = a() if callable(a) else a - res_b = b() if callable(b) else b - return res_a != res_b - - return wrap - - def test_event_time_microseconds_field(started_cluster): cluster.start() query_create = """ @@ -64,7 +47,7 @@ def test_event_time_microseconds_field(started_cluster): query_number_detached_by_user_parts_in_async_metric = """ SELECT value FROM system.asynchronous_metrics - WHERE metric LIKE 'NumberOfDetachedPartsByUser'; + WHERE metric LIKE 'NumberOfDetachedByUserParts'; """ query_count_active_parts = """ SELECT count(*) FROM system.parts WHERE table = 't' AND active @@ -89,12 +72,11 @@ def test_event_time_microseconds_field(started_cluster): assert 2 == int(node1.query(query_count_detached_parts)) assert 1 == int(node1.query(query_count_active_parts)) - wait_until( - is_different( - 0, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) - ) + assert_eq_with_retry( + node1, + query_number_detached_parts_in_async_metric, + "2\n", ) - assert 2 == int(node1.query(query_number_detached_parts_in_async_metric)) assert 2 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) # detach the rest parts and wait until asynchronous metrics notice it @@ -103,12 +85,11 @@ def test_event_time_microseconds_field(started_cluster): assert 3 == int(node1.query(query_count_detached_parts)) assert 0 == int(node1.query(query_count_active_parts)) - wait_until( - is_different( - 2, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) - ) + assert_eq_with_retry( + node1, + query_number_detached_parts_in_async_metric, + "3\n", ) - assert 3 == int(node1.query(query_number_detached_parts_in_async_metric)) assert 3 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) # inject some data directly and wait until asynchronous metrics notice it @@ -117,19 +98,17 @@ def test_event_time_microseconds_field(started_cluster): "bash", "-c", "mkdir /var/lib/clickhouse/data/default/t/detached/unexpected_all_0_0_0", - ], - privileged=True, + ] ) assert 4 == int(node1.query(query_count_detached_parts)) assert 0 == int(node1.query(query_count_active_parts)) - wait_until( - is_different( - 3, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) - ) + assert_eq_with_retry( + node1, + query_number_detached_parts_in_async_metric, + "4\n", ) - assert 4 == int(node1.query(query_number_detached_parts_in_async_metric)) assert 3 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) # drop some data directly and wait asynchronous metrics notice it @@ -140,17 +119,15 @@ def test_event_time_microseconds_field(started_cluster): "rm -rf /var/lib/clickhouse/data/default/t/detached/{}".format( partition_name ), - ], - privileged=True, + ] ) assert 3 == int(node1.query(query_count_detached_parts)) assert 0 == int(node1.query(query_count_active_parts)) - wait_until( - is_different( - 4, lambda: int(node1.query(query_number_detached_parts_in_async_metric)) - ) + assert_eq_with_retry( + node1, + query_number_detached_parts_in_async_metric, + "3\n", ) - assert 3 == int(node1.query(query_number_detached_parts_in_async_metric)) assert 2 == int(node1.query(query_number_detached_by_user_parts_in_async_metric)) From a5f69517ae582f138547b72b0fe6d99eff5c8de3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Aug 2022 19:47:34 +0200 Subject: [PATCH 206/582] Allow to load marks in threadpool --- src/Common/CurrentMetrics.cpp | 1 + src/Common/ProfileEvents.cpp | 2 + src/Core/Settings.h | 2 + src/IO/ReadSettings.h | 2 + src/Interpreters/Context.cpp | 14 +++ src/Interpreters/Context.h | 2 + .../MergeTree/MergeTreeDataPartCompact.cpp | 6 +- .../MergeTree/MergeTreeIndexReader.cpp | 6 +- .../MergeTree/MergeTreeMarksLoader.cpp | 88 +++++++++++++++++-- src/Storages/MergeTree/MergeTreeMarksLoader.h | 9 +- .../MergeTree/MergeTreeReaderCompact.cpp | 2 + .../MergeTree/MergeTreeReaderCompact.h | 1 + .../MergeTree/MergeTreeReaderStream.cpp | 16 ++-- .../MergeTree/MergeTreeReaderStream.h | 16 ++-- .../MergeTree/MergeTreeReaderWide.cpp | 6 +- 15 files changed, 150 insertions(+), 23 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index eeb8b4e2832..9e5f3705dad 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -98,6 +98,7 @@ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ + M(BackgroundLoadingMarksTasks, "Number of currently executing background marks load tasks") \ M(S3Requests, "S3 requests") \ M(KeeperAliveConnections, "Number of alive connections") \ M(KeeperOutstandingRequets, "Number of outstanding requests") \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 297b3bdb59d..22dc551ab3a 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -146,6 +146,8 @@ M(SelectedRows, "Number of rows SELECTed from all tables.") \ M(SelectedBytes, "Number of bytes (uncompressed; for columns as they stored in memory) SELECTed from all tables.") \ \ + M(WaitMarksLoadMicroseconds, "Time spent loading marks") \ + \ M(Merge, "Number of launched background merges.") \ M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \ M(MergedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for background merges. This is the number before merge.") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 026b603177c..ce49296a508 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -598,6 +598,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \ M(UInt64, max_query_cache_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be used by a single query", 0) \ \ + M(Bool, load_marks_asynchronously, true, "Load MergeTree marks asynchronously", 0) \ + \ M(Bool, use_structure_from_insertion_table_in_table_functions, false, "Use structure from insertion table instead of schema inference from data", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index e639ecbedc2..31aeb8a904f 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -76,6 +76,8 @@ struct ReadSettings /// For 'pread_threadpool' method. Lower is more priority. size_t priority = 0; + bool load_marks_asynchronously = true; + size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d45d30f1957..d39c39cdb15 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -215,6 +215,7 @@ struct ContextSharedPart std::unique_ptr access_control; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. + mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. @@ -1711,6 +1712,17 @@ void Context::dropMarkCache() const shared->mark_cache->reset(); } +ThreadPool & Context::getLoadMarksThreadpool() const +{ + auto lock = getLock(); + if (!shared->load_marks_threadpool) + { + constexpr size_t pool_size = 50; + constexpr size_t queue_size = 1000000; + shared->load_marks_threadpool = std::make_unique(pool_size, pool_size, queue_size); + } + return *shared->load_marks_threadpool; +} void Context::setIndexUncompressedCache(size_t max_size_in_bytes) { @@ -3452,6 +3464,8 @@ ReadSettings Context::getReadSettings() const res.local_fs_prefetch = settings.local_filesystem_read_prefetch; res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch; + res.load_marks_asynchronously = settings.load_marks_asynchronously; + res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms; res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries; res.enable_filesystem_cache = settings.enable_filesystem_cache; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ea03b8e6586..21378dc096c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -126,6 +126,7 @@ using PartUUIDsPtr = std::shared_ptr; class KeeperDispatcher; class Session; struct WriteSettings; +class Threadpool; class IInputFormat; class IOutputFormat; @@ -788,6 +789,7 @@ public: void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy); std::shared_ptr getMarkCache() const; void dropMarkCache() const; + ThreadPool & getLoadMarksThreadpool() const; /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 046a7d274c0..851153cd619 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -46,9 +47,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { auto ptr = std::static_pointer_cast(shared_from_this()); + auto context = storage.getContext(); + auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + return std::make_unique( ptr, columns_to_read, metadata_snapshot, uncompressed_cache, - mark_cache, mark_ranges, reader_settings, + mark_cache, mark_ranges, reader_settings, load_marks_threadpool, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index c43c75035e4..33106f7ab64 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -1,4 +1,5 @@ #include +#include namespace { @@ -15,6 +16,9 @@ std::unique_ptr makeIndexReader( UncompressedCache * uncompressed_cache, MergeTreeReaderSettings settings) { + auto context = part->storage.getContext(); + auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + return std::make_unique( part->data_part_storage, index->getFileName(), extension, marks_count, @@ -22,7 +26,7 @@ std::unique_ptr makeIndexReader( std::move(settings), mark_cache, uncompressed_cache, part->getFileSizeOrZero(index->getFileName() + extension), &part->index_granularity_info, - ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false); + ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false, load_marks_threadpool); } } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index ce26a86f0c0..8c6fc293e6b 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -2,9 +2,23 @@ #include #include #include +#include +#include +#include +#include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundLoadingMarksTasks; +} + +namespace ProfileEvents +{ + extern const Event WaitMarksLoadMicroseconds; +} + namespace DB { @@ -23,6 +37,7 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( const MergeTreeIndexGranularityInfo & index_granularity_info_, bool save_marks_in_cache_, const ReadSettings & read_settings_, + ThreadPool * load_marks_threadpool_, size_t columns_in_mark_) : data_part_storage(std::move(data_part_storage_)) , mark_cache(mark_cache_) @@ -32,13 +47,33 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( , save_marks_in_cache(save_marks_in_cache_) , columns_in_mark(columns_in_mark_) , read_settings(read_settings_) + , load_marks_threadpool(load_marks_threadpool_) { + if (load_marks_threadpool) + { + future = loadMarksAsync(); + } } const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index) { if (!marks) - loadMarks(); + { + Stopwatch watch(CLOCK_MONOTONIC); + + if (future.valid()) + { + marks = future.get(); + future = {}; + } + else + { + marks = loadMarks(); + } + + watch.stop(); + ProfileEvents::increment(ProfileEvents::WaitMarksLoadMicroseconds, watch.elapsedMicroseconds()); + } #ifndef NDEBUG if (column_index >= columns_in_mark) @@ -95,28 +130,63 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() return res; } -void MergeTreeMarksLoader::loadMarks() +MarkCache::MappedPtr MergeTreeMarksLoader::loadMarks() { + MarkCache::MappedPtr loaded_marks; + if (mark_cache) { auto key = mark_cache->hash(fs::path(data_part_storage->getFullPath()) / mrk_path); if (save_marks_in_cache) { auto callback = [this]{ return loadMarksImpl(); }; - marks = mark_cache->getOrSet(key, callback); + loaded_marks = mark_cache->getOrSet(key, callback); } else { - marks = mark_cache->get(key); - if (!marks) - marks = loadMarksImpl(); + loaded_marks = mark_cache->get(key); + if (!loaded_marks) + loaded_marks = loadMarksImpl(); } } else - marks = loadMarksImpl(); + loaded_marks = loadMarksImpl(); - if (!marks) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to load marks: {}", String(fs::path(data_part_storage->getFullPath()) / mrk_path)); + if (!loaded_marks) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Failed to load marks: {}", + (fs::path(data_part_storage->getFullPath()) / mrk_path).string()); + } + + return loaded_marks; +} + +std::future MergeTreeMarksLoader::loadMarksAsync() +{ + ThreadGroupStatusPtr thread_group; + if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup()) + thread_group = CurrentThread::get().getThreadGroup(); + + auto task = std::make_shared>([thread_group, this] + { + setThreadName("loadMarksThread"); + + if (thread_group) + CurrentThread::attachTo(thread_group); + + SCOPE_EXIT_SAFE({ + if (thread_group) + CurrentThread::detachQuery(); + }); + + CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundLoadingMarksTasks}; + return loadMarks(); + }); + + auto task_future = task->get_future(); + load_marks_threadpool->scheduleOrThrow([task]{ (*task)(); }); + return task_future; } } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 3a1d3dc2c1b..69425900658 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -2,11 +2,13 @@ #include #include #include +#include namespace DB { struct MergeTreeIndexGranularityInfo; +class Threadpool; class MergeTreeMarksLoader { @@ -21,6 +23,7 @@ public: const MergeTreeIndexGranularityInfo & index_granularity_info_, bool save_marks_in_cache_, const ReadSettings & read_settings_, + ThreadPool * load_marks_threadpool_, size_t columns_in_mark_ = 1); const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0); @@ -36,8 +39,12 @@ private: MarkCache::MappedPtr marks; ReadSettings read_settings; - void loadMarks(); + MarkCache::MappedPtr loadMarks(); + std::future loadMarksAsync(); MarkCache::MappedPtr loadMarksImpl(); + + std::future future; + ThreadPool * load_marks_threadpool; }; } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 44b5fa1a11b..88237091547 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -22,6 +22,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( MarkCache * mark_cache_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_, + ThreadPool * load_marks_cache_threadpool_, ValueSizeMap avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) @@ -42,6 +43,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( data_part->index_granularity_info, settings.save_marks_in_cache, settings.read_settings, + load_marks_cache_threadpool_, data_part->getColumns().size()) { try diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index aa0eb949aa1..348e4802e2a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -26,6 +26,7 @@ public: MarkCache * mark_cache_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_, + ThreadPool * load_marks_cache_threadpool_, ValueSizeMap avg_value_size_hints_ = {}, const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {}, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE); diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index d9913f2639f..47f8b0f6008 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -16,14 +16,19 @@ namespace ErrorCodes MergeTreeReaderStream::MergeTreeReaderStream( DataPartStoragePtr data_part_storage_, - const String & path_prefix_, const String & data_file_extension_, size_t marks_count_, + const String & path_prefix_, + const String & data_file_extension_, + size_t marks_count_, const MarkRanges & all_mark_ranges_, const MergeTreeReaderSettings & settings_, MarkCache * mark_cache_, - UncompressedCache * uncompressed_cache_, size_t file_size_, + UncompressedCache * uncompressed_cache_, + size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_, - const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_, - bool is_low_cardinality_dictionary_) + const ReadBufferFromFileBase::ProfileCallback & profile_callback_, + clockid_t clock_type_, + bool is_low_cardinality_dictionary_, + ThreadPool * load_marks_cache_threadpool_) : settings(settings_) , profile_callback(profile_callback_) , clock_type(clock_type_) @@ -45,7 +50,8 @@ MergeTreeReaderStream::MergeTreeReaderStream( marks_count, *index_granularity_info, save_marks_in_cache, - settings.read_settings) + settings.read_settings, + load_marks_cache_threadpool_) { } diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.h b/src/Storages/MergeTree/MergeTreeReaderStream.h index f5a8ebadcba..83e314eef42 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.h +++ b/src/Storages/MergeTree/MergeTreeReaderStream.h @@ -20,13 +20,19 @@ class MergeTreeReaderStream public: MergeTreeReaderStream( DataPartStoragePtr data_part_storage_, - const String & path_prefix_, const String & data_file_extension_, size_t marks_count_, + const String & path_prefix_, + const String & data_file_extension_, + size_t marks_count_, const MarkRanges & all_mark_ranges, const MergeTreeReaderSettings & settings_, - MarkCache * mark_cache, UncompressedCache * uncompressed_cache, - size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_, - const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type, - bool is_low_cardinality_dictionary_); + MarkCache * mark_cache, + UncompressedCache * uncompressed_cache, + size_t file_size_, + const MergeTreeIndexGranularityInfo * index_granularity_info_, + const ReadBufferFromFileBase::ProfileCallback & profile_callback, + clockid_t clock_type, + bool is_low_cardinality_dictionary_, + ThreadPool * load_marks_cache_threadpool_); void seekToMark(size_t index); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 1274017b865..9ac7b5a5c5a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -177,12 +178,15 @@ void MergeTreeReaderWide::addStreams( bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; + auto context = data_part->storage.getContext(); + auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + streams.emplace(stream_name, std::make_unique( data_part->data_part_storage, stream_name, DATA_FILE_EXTENSION, data_part->getMarksCount(), all_mark_ranges, settings, mark_cache, uncompressed_cache, data_part->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), &data_part->index_granularity_info, - profile_callback, clock_type, is_lc_dict)); + profile_callback, clock_type, is_lc_dict, load_marks_threadpool)); }; serialization->enumerateStreams(callback); From 003483b6165cbee284f91c967fe5c5547a40c733 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 30 Aug 2022 20:49:40 +0200 Subject: [PATCH 207/582] Add compatibility setting --- src/Core/Settings.h | 2 + src/Core/SettingsChangesHistory.h | 1 + src/Functions/grouping.h | 33 +++++++---- src/Interpreters/ActionsVisitor.cpp | 8 +-- .../02315_grouping_constant_folding.reference | 56 +++++++++---------- .../02315_grouping_constant_folding.sql | 4 +- 6 files changed, 59 insertions(+), 45 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index af32c15a867..b41284ef20a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -612,6 +612,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \ M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \ \ + M(Bool, force_grouping_standard_compatibility, true, "Make GROUPING function to return 1 when argument is not used as an aggregation key", 0) \ + \ M(Bool, schema_inference_use_cache_for_file, true, "Use cache in schema inference while using file table function", 0) \ M(Bool, schema_inference_use_cache_for_s3, true, "Use cache in schema inference while using s3 table function", 0) \ M(Bool, schema_inference_use_cache_for_hdfs, true, "Use cache in schema inference while using hdfs table function", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8d0e69f4b29..9440cd8bae2 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -78,6 +78,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index dc630123dcb..7a9df462b23 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -18,10 +19,15 @@ protected: static constexpr UInt64 ONE = 1; const ColumnNumbers arguments_indexes; + const bool force_compatibility; + + static constexpr UInt64 COMPATIBLE_MODE[] = {1, 0}; + static constexpr UInt64 INCOMPATIBLE_MODE[] = {0, 1}; public: - FunctionGroupingBase(ColumnNumbers arguments_indexes_) + FunctionGroupingBase(ColumnNumbers arguments_indexes_, bool force_compatibility_) : arguments_indexes(std::move(arguments_indexes_)) + , force_compatibility(force_compatibility_) {} bool isVariadic() const override { return true; } @@ -47,13 +53,15 @@ public: auto result = ColumnUInt64::create(); auto & result_data = result->getData(); result_data.reserve(input_rows_count); + + const auto * result_table = likely(force_compatibility) ? COMPATIBLE_MODE : INCOMPATIBLE_MODE; for (size_t i = 0; i < input_rows_count; ++i) { UInt64 set_index = grouping_set_column->getElement(i); UInt64 value = 0; for (auto index : arguments_indexes) - value = (value << 1) + (checker(set_index, index) ? 0 : 1); + value = (value << 1) + result_table[checker(set_index, index) ? 1 : 0]; result_data.push_back(value); } @@ -64,15 +72,18 @@ public: class FunctionGroupingOrdinary : public FunctionGroupingBase { public: - explicit FunctionGroupingOrdinary(ColumnNumbers arguments_indexes_) - : FunctionGroupingBase(std::move(arguments_indexes_)) + FunctionGroupingOrdinary(ColumnNumbers arguments_indexes_, bool force_compatibility_) + : FunctionGroupingBase(std::move(arguments_indexes_), force_compatibility_) {} String getName() const override { return "groupingOrdinary"; } ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - return ColumnUInt64::create(input_rows_count, 0); + if (likely(force_compatibility)) + return ColumnUInt64::create(input_rows_count, 0); + UInt64 value = (ONE << arguments_indexes.size()) - 1; + return ColumnUInt64::create(input_rows_count, value); } }; @@ -81,8 +92,8 @@ class FunctionGroupingForRollup : public FunctionGroupingBase const UInt64 aggregation_keys_number; public: - FunctionGroupingForRollup(ColumnNumbers arguments_indexes_, UInt64 aggregation_keys_number_) - : FunctionGroupingBase(std::move(arguments_indexes_)) + FunctionGroupingForRollup(ColumnNumbers arguments_indexes_, UInt64 aggregation_keys_number_, bool force_compatibility_) + : FunctionGroupingBase(std::move(arguments_indexes_), force_compatibility_) , aggregation_keys_number(aggregation_keys_number_) {} @@ -111,8 +122,8 @@ class FunctionGroupingForCube : public FunctionGroupingBase public: - FunctionGroupingForCube(ColumnNumbers arguments_indexes_, UInt64 aggregation_keys_number_) - : FunctionGroupingBase(arguments_indexes_) + FunctionGroupingForCube(ColumnNumbers arguments_indexes_, UInt64 aggregation_keys_number_, bool force_compatibility_) + : FunctionGroupingBase(arguments_indexes_, force_compatibility_) , aggregation_keys_number(aggregation_keys_number_) {} @@ -140,8 +151,8 @@ class FunctionGroupingForGroupingSets : public FunctionGroupingBase { ColumnNumbersSetList grouping_sets; public: - FunctionGroupingForGroupingSets(ColumnNumbers arguments_indexes_, ColumnNumbersList const & grouping_sets_) - : FunctionGroupingBase(std::move(arguments_indexes_)) + FunctionGroupingForGroupingSets(ColumnNumbers arguments_indexes_, ColumnNumbersList const & grouping_sets_, bool force_compatibility_) + : FunctionGroupingBase(std::move(arguments_indexes_), force_compatibility_) { for (auto const & set : grouping_sets_) grouping_sets.emplace_back(set.begin(), set.end()); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 6c9e54a966d..0412ce14b54 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -880,20 +880,20 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & { case GroupByKind::GROUPING_SETS: { - data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), keys_info.grouping_set_keys)), { "__grouping_set" }, column_name); + data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), keys_info.grouping_set_keys, data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), { "__grouping_set" }, column_name); break; } case GroupByKind::ROLLUP: - data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), aggregation_keys_number)), { "__grouping_set" }, column_name); + data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), aggregation_keys_number, data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), { "__grouping_set" }, column_name); break; case GroupByKind::CUBE: { - data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), aggregation_keys_number)), { "__grouping_set" }, column_name); + data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), aggregation_keys_number, data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), { "__grouping_set" }, column_name); break; } case GroupByKind::ORDINARY: { - data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes))), {}, column_name); + data.addFunction(std::make_shared(std::make_shared(std::move(arguments_indexes), data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), {}, column_name); break; } default: diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.reference b/tests/queries/0_stateless/02315_grouping_constant_folding.reference index c44fee183da..6e591de2661 100644 --- a/tests/queries/0_stateless/02315_grouping_constant_folding.reference +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.reference @@ -1,29 +1,29 @@ -- { echoOn } -SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); -1 0 0 0 -1 0 2 0 -1 0 4 0 -1 0 6 0 -1 0 8 0 -1 1 1 0 -1 1 3 0 -1 1 5 0 -1 1 7 0 -1 1 9 0 -5 0 0 1 -5 1 0 1 -10 0 0 3 -SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); -1 0 0 0 -1 0 2 0 -1 0 4 0 -1 0 6 0 -1 0 8 0 -1 1 1 0 -1 1 3 0 -1 1 5 0 -1 1 7 0 -1 1 9 0 -5 0 0 1 -5 1 0 1 -10 0 0 3 +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0; +1 0 0 3 +1 0 2 3 +1 0 4 3 +1 0 6 3 +1 0 8 3 +1 1 1 3 +1 1 3 3 +1 1 5 3 +1 1 7 3 +1 1 9 3 +5 0 0 2 +5 1 0 2 +10 0 0 0 +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0; +1 0 0 3 +1 0 2 3 +1 0 4 3 +1 0 6 3 +1 0 8 3 +1 1 1 3 +1 1 3 3 +1 1 5 3 +1 1 7 3 +1 1 9 3 +5 0 0 2 +5 1 0 2 +10 0 0 0 diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.sql b/tests/queries/0_stateless/02315_grouping_constant_folding.sql index c4ef087a308..ff259b7be79 100644 --- a/tests/queries/0_stateless/02315_grouping_constant_folding.sql +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.sql @@ -5,9 +5,9 @@ CREATE TABLE test02315(a UInt64, b UInt64) ENGINE=MergeTree() ORDER BY (a, b); INSERT INTO test02315 SELECT number % 2 as a, number as b FROM numbers(10); -- { echoOn } -SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0; -SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0; -- { echoOff } DROP TABLE test02315; From 6b23abe2b6412c0e9d8a721aa7c4b07565a1bfbe Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Tue, 30 Aug 2022 21:57:33 +0200 Subject: [PATCH 208/582] Update src/Interpreters/AsynchronousMetrics.cpp Co-authored-by: Alexander Tokmakov --- src/Interpreters/AsynchronousMetrics.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 05acbb51c36..0f715c50338 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1655,8 +1655,8 @@ void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, Tim watch.stop(); - // Normally heavy metrics don't delay the rest of the metrics calculation - // otherwise log the warning message + /// Normally heavy metrics don't delay the rest of the metrics calculation + /// otherwise log the warning message auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE); if (watch.elapsedSeconds() > (update_period.count() / 2.)) log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG); From 412973daeeb5692dbd76bff1a002e3bb8a416401 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Tue, 30 Aug 2022 21:57:39 +0200 Subject: [PATCH 209/582] Update src/Interpreters/AsynchronousMetrics.cpp Co-authored-by: Alexander Tokmakov --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 0f715c50338..173638ff381 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1650,7 +1650,7 @@ void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, Tim Stopwatch watch; - // Test shows that listing 100000 entries consuming around 0.15 sec. + /// Test shows that listing 100000 entries consuming around 0.15 sec. updateDetachedPartsStats(); watch.stop(); From e97640e2f0f1d52c1e48db22c7629eecc980295f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 07:21:20 +0000 Subject: [PATCH 210/582] Fix reference for keeper test --- tests/queries/0_stateless/02417_keeper_map_create_drop.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.reference b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference index dda215c9659..25bc8c288fb 100644 --- a/tests/queries/0_stateless/02417_keeper_map_create_drop.reference +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference @@ -1,5 +1,4 @@ 1 11 -2 22 ------ 1 11 2 22 From 5ef4c52290347f6936b392f74f2a026a45e750cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:14:28 +0000 Subject: [PATCH 211/582] Add support for keys limit --- src/Storages/StorageKeeperMap.cpp | 124 ++++++++++++++++++++++-------- src/Storages/StorageKeeperMap.h | 12 ++- 2 files changed, 102 insertions(+), 34 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index dd55586208e..70f32bbcd74 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -30,6 +30,7 @@ #include #include #include "Core/UUID.h" +#include "base/types.h" #include #include @@ -46,6 +47,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int KEEPER_EXCEPTION; extern const int LOGICAL_ERROR; + extern const int LIMIT_EXCEEDED; } namespace @@ -128,18 +130,48 @@ public: void onFinish() override { - auto & zookeeper = storage.getClient(); + auto zookeeper = storage.getClient(); Coordination::Requests requests; + auto keys_limit = storage.keysLimit(); + + size_t current_keys_num = 0; + size_t new_keys_num = 0; + + if (keys_limit != 0) + { + Coordination::Stat root_stat; + zookeeper->get(storage.rootKeeperPath(), &root_stat); + // exclude metadata node + current_keys_num = root_stat.numChildren - 1; + } + for (const auto & [key, value] : new_values) { auto path = storage.fullPathForKey(key); if (zookeeper->exists(path)) + { requests.push_back(zkutil::makeSetRequest(path, value, -1)); + } else + { requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + ++new_keys_num; + } + } + + if (new_keys_num != 0) + { + auto will_be = current_keys_num + new_keys_num; + if (keys_limit != 0 && will_be > keys_limit) + throw Exception( + ErrorCodes::LIMIT_EXCEEDED, + "Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}", + new_keys_num, + keys_limit, + will_be); } zookeeper->multi(requests); @@ -209,12 +241,14 @@ StorageKeeperMap::StorageKeeperMap( bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path) + bool create_missing_root_path, + UInt64 keys_limit_) : IStorage(table_id) , WithContext(context_->getGlobalContext()) , root_path(zkutil::extractZooKeeperPath(root_path_, false)) , primary_key(primary_key_) , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) + , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { if (table_id.uuid == UUIDHelpers::Nil) @@ -227,6 +261,24 @@ StorageKeeperMap::StorageKeeperMap( if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); + + auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); + if (config_keys_limit != 0 && keys_limit > config_keys_limit) + { + LOG_WARNING( + log, + "Keys limit for {} defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use " + "config defined value", + getStorageID().getFullTableName(), + keys_limit, + config_keys_limit); + keys_limit = config_keys_limit; + } + else if (keys_limit > 0) + { + LOG_INFO(log, "Keys limit for {} will be set to {}", getStorageID().getFullTableName(), keys_limit); + } + std::filesystem::path root_path_fs{root_path}; auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; @@ -351,7 +403,7 @@ Pipe StorageKeeperMap::read( return Pipe::unitePipes(std::move(pipes)); }; - auto & client = getClient(); + auto client = getClient(); if (all_scan) return process_keys(std::make_shared>(client->getChildren(root_path))); @@ -441,7 +493,7 @@ void StorageKeeperMap::drop() removeMetadataNodes(client, metadata_drop_lock); } -zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const +zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const { std::lock_guard lock{zookeeper_mutex}; if (!zookeeper_client || zookeeper_client->expired()) @@ -466,6 +518,11 @@ std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const return fmt::format("{}/{}", root_path, key); } +UInt64 StorageKeeperMap::keysLimit() const +{ + return keys_limit; +} + std::optional StorageKeeperMap::isTableValid() const { std::lock_guard lock{init_mutex}; @@ -586,38 +643,45 @@ Block StorageKeeperMap::getSampleBlock(const Names &) const namespace { - StoragePtr create(const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 2) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-2 arguments:\n" - "root_path: path in the Keeper where the values will be stored (required)\n" - "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - default_host); - auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); +StoragePtr create(const StorageFactory::Arguments & args) +{ + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1-2 arguments:\n" + "root_path: path in the Keeper where the values will be stored (required)\n" + "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", + "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)\n", + default_host); - bool create_missing_root_path = true; - if (engine_args.size() > 1) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); + auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); - StorageInMemoryMetadata metadata; - metadata.setColumns(args.columns); - metadata.setConstraints(args.constraints); + bool create_missing_root_path = true; + if (engine_args.size() > 1) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); - if (!args.storage_def->primary_key) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + UInt64 keys_limit = 0; + if (engine_args.size() > 2) + keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); - auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); - if (primary_key_names.size() != 1) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->primary_key) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + return std::make_shared( + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path, keys_limit); +} - return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path); - } } void registerStorageKeeperMap(StorageFactory & factory) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 22c5bb66e1a..fa3a9e4bd2e 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -30,7 +30,8 @@ public: bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path); + bool create_missing_root_path, + UInt64 keys_limit_); Pipe read( const Names & column_names, @@ -62,12 +63,11 @@ public: return node->getColumnName() == primary_key; } - zkutil::ZooKeeperPtr & getClient() const; + zkutil::ZooKeeperPtr getClient() const; const std::string & rootKeeperPath() const; std::string fullPathForKey(std::string_view key) const; - const std::string & lockPath() const; - std::optional isTableValid() const; + UInt64 keysLimit() const; template void checkTable() const @@ -103,6 +103,8 @@ public: private: bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); + std::optional isTableValid() const; + const std::string root_path; std::string primary_key; @@ -116,6 +118,8 @@ private: std::string zookeeper_name; + uint64_t keys_limit{0}; + mutable std::mutex zookeeper_mutex; mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; From c73f41868bd0059b0f235e8f933d049821c79e66 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:14:50 +0000 Subject: [PATCH 212/582] Add tests for keys_limit --- .../02418_keeper_map_keys_limit.reference | 4 ++++ .../02418_keeper_map_keys_limit.sql | 23 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02418_keeper_map_keys_limit.reference create mode 100644 tests/queries/0_stateless/02418_keeper_map_keys_limit.sql diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.reference b/tests/queries/0_stateless/02418_keeper_map_keys_limit.reference new file mode 100644 index 00000000000..95c45d6da51 --- /dev/null +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.reference @@ -0,0 +1,4 @@ +2 +3 +4 +4 diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql new file mode 100644 index 00000000000..aedeba661dd --- /dev/null +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -0,0 +1,23 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS 02418_test SYNC; + +CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 3) PRIMARY KEY(key); + +INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2); +SELECT count() FROM 02418_test; + +INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 } + +INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3); +SELECT count() FROM 02418_test; + +CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 4) PRIMARY KEY(key); +INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 } +INSERT INTO 02418_test_another VALUES (4, 4.4); + +SELECT count() FROM 02418_test; +SELECT count() FROM 02418_test_another; + +DROP TABLE 02418_test SYNC; +DROP TABLE 02418_test_another SYNC; From 1ff14aed023a02ea9e0fd365c4edacab82f07e70 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:23:44 +0000 Subject: [PATCH 213/582] Fix dropping table --- src/Storages/StorageKeeperMap.cpp | 10 +++++----- src/Storages/StorageKeeperMap.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 70f32bbcd74..66b7c1731f1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -330,7 +330,7 @@ StorageKeeperMap::StorageKeeperMap( else { auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); - if (!removeMetadataNodes(client, metadata_drop_lock)) + if (!dropTable(client, metadata_drop_lock)) continue; } } @@ -423,8 +423,10 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont client->tryRemoveChildrenRecursive(root_path, true, getBaseName(metadata_path)); } -bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) +bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { + zookeeper->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + bool completely_removed = false; Coordination::Requests ops; ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); @@ -487,10 +489,8 @@ void StorageKeeperMap::drop() else if (code != Coordination::Error::ZOK) zkutil::KeeperMultiException::check(code, ops, responses); - client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); - auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); - removeMetadataNodes(client, metadata_drop_lock); + dropTable(client, metadata_drop_lock); } zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index fa3a9e4bd2e..3e47e5ee9c6 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -101,7 +101,7 @@ public: } private: - bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); + bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); std::optional isTableValid() const; From 1af9a81297e224ddad324d908669a60f375df044 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:29:40 +0000 Subject: [PATCH 214/582] Sync after reconnect --- src/Storages/StorageKeeperMap.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 66b7c1731f1..157b35ce995 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -503,6 +503,8 @@ zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const zookeeper_client = getContext()->getZooKeeper(); else zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + + zookeeper_client->sync(rootKeeperPath()); } return zookeeper_client; From 928adb2b38f8d1e067c7a7d81aa337e89092e7ea Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Aug 2022 09:13:47 +0000 Subject: [PATCH 215/582] [WIP] Limit number of analyze for one query --- src/Interpreters/Context.h | 21 +++++++++++++++++++ src/Interpreters/InterpreterSelectQuery.cpp | 11 ++++++++++ .../02337_join_analyze_stuck.reference | 0 .../0_stateless/02337_join_analyze_stuck.sql | 15 +++++++++++++ 4 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/02337_join_analyze_stuck.reference create mode 100644 tests/queries/0_stateless/02337_join_analyze_stuck.sql diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ea03b8e6586..13d57c0b962 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -367,6 +367,27 @@ public: // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. OpenTelemetryTraceContext query_trace_context; + /// Some counters for current query execution. + /// Most of them are workarounds and should be removed in the future. + struct KitchenSink + { + std::atomic analyze_counter = 0; + + KitchenSink() = default; + + KitchenSink(const KitchenSink & rhs) + : analyze_counter(rhs.analyze_counter.load()) + {} + + KitchenSink & operator=(const KitchenSink & rhs) + { + analyze_counter = rhs.analyze_counter.load(); + return *this; + } + }; + + KitchenSink kitchen_sink; + private: using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 76f199d8e41..ac7cc4c1b4e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,6 +98,7 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; extern const int ILLEGAL_FINAL; extern const int ILLEGAL_PREWHERE; + extern const int TOO_DEEP_PIPELINE; extern const int TOO_MANY_COLUMNS; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; @@ -496,6 +497,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( auto analyze = [&] (bool try_move_to_prewhere) { + if (context->hasQueryContext()) + { + std::atomic & current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter; + ++current_query_analyze_count; + if (settings.max_pipeline_depth && current_query_analyze_count >= settings.max_pipeline_depth) + throw DB::Exception(ErrorCodes::TOO_DEEP_PIPELINE, "Query analyze overflow. Try to increase `max_pipeline_depth` or simplify the query"); + } + /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; if (view) @@ -639,6 +648,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( analyze(shouldMoveToPrewhere()); bool need_analyze_again = false; + if (analysis_result.prewhere_constant_filter_description.always_false || analysis_result.prewhere_constant_filter_description.always_true) { if (analysis_result.prewhere_constant_filter_description.always_true) @@ -647,6 +657,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query.setExpression(ASTSelectQuery::Expression::PREWHERE, std::make_shared(0u)); need_analyze_again = true; } + if (analysis_result.where_constant_filter_description.always_false || analysis_result.where_constant_filter_description.always_true) { if (analysis_result.where_constant_filter_description.always_true) diff --git a/tests/queries/0_stateless/02337_join_analyze_stuck.reference b/tests/queries/0_stateless/02337_join_analyze_stuck.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02337_join_analyze_stuck.sql b/tests/queries/0_stateless/02337_join_analyze_stuck.sql new file mode 100644 index 00000000000..9bdc418f028 --- /dev/null +++ b/tests/queries/0_stateless/02337_join_analyze_stuck.sql @@ -0,0 +1,15 @@ +-- Tags: long + +-- https://github.com/ClickHouse/ClickHouse/issues/21557 + +SET max_pipeline_depth = 1000; + +EXPLAIN SYNTAX +WITH + x AS ( SELECT number FROM numbers(10) ), + cross_sales AS ( + SELECT 1 AS xx + FROM x, x AS d1, x AS d2, x AS d3, x AS d4, x AS d5, x AS d6, x AS d7, x AS d8, x AS d9 + WHERE x.number = d9.number + ) +SELECT xx FROM cross_sales WHERE xx = 2000; -- { serverError TOO_DEEP_PIPELINE } From f38f39ed6fcfce9ad6a70dbc3bf01c30b3a297d3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Aug 2022 09:21:07 +0000 Subject: [PATCH 216/582] Use max_analyze_depth --- src/Core/Settings.h | 3 +++ src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- tests/queries/0_stateless/02337_join_analyze_stuck.sql | 4 ++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index af32c15a867..c14c411bd49 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -347,6 +347,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(UInt64, max_subquery_depth, 100, "If a query has more than specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \ M(UInt64, max_pipeline_depth, 1000, "If a query has more than specified stages in the query pipeline, throw an exception. Pipeline has stages for every relational operator. This allows to limit the complexity of the queries.", 0) \ + M(UInt64, max_analyze_depth, 10000, "", 0) \ M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \ M(UInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \ M(UInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.", 0) \ @@ -672,6 +673,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) MAKE_OBSOLETE(M, UInt64, background_message_broker_schedule_pool_size, 16) \ MAKE_OBSOLETE(M, UInt64, background_distributed_schedule_pool_size, 16) \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ + MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ + /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ac7cc4c1b4e..c596c5233ce 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -501,8 +501,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( { std::atomic & current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter; ++current_query_analyze_count; - if (settings.max_pipeline_depth && current_query_analyze_count >= settings.max_pipeline_depth) - throw DB::Exception(ErrorCodes::TOO_DEEP_PIPELINE, "Query analyze overflow. Try to increase `max_pipeline_depth` or simplify the query"); + if (settings.max_analyze_depth && current_query_analyze_count >= settings.max_analyze_depth) + throw DB::Exception(ErrorCodes::TOO_DEEP_RECURSION, "Query analyze overflow. Try to increase `max_analyze_depth` or simplify the query"); } /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. diff --git a/tests/queries/0_stateless/02337_join_analyze_stuck.sql b/tests/queries/0_stateless/02337_join_analyze_stuck.sql index 9bdc418f028..4e3d0bc44c5 100644 --- a/tests/queries/0_stateless/02337_join_analyze_stuck.sql +++ b/tests/queries/0_stateless/02337_join_analyze_stuck.sql @@ -2,7 +2,7 @@ -- https://github.com/ClickHouse/ClickHouse/issues/21557 -SET max_pipeline_depth = 1000; +SET max_analyze_depth = 1000; EXPLAIN SYNTAX WITH @@ -12,4 +12,4 @@ WITH FROM x, x AS d1, x AS d2, x AS d3, x AS d4, x AS d5, x AS d6, x AS d7, x AS d8, x AS d9 WHERE x.number = d9.number ) -SELECT xx FROM cross_sales WHERE xx = 2000; -- { serverError TOO_DEEP_PIPELINE } +SELECT xx FROM cross_sales WHERE xx = 2000; -- { serverError TOO_DEEP_RECURSION } From c829a1914dc68b59422bf0d7847b181382e898da Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Aug 2022 09:27:50 +0000 Subject: [PATCH 217/582] Disable max_analyze_depth for StorageRabbitMQ/NATS --- src/Storages/NATS/StorageNATS.cpp | 2 ++ src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 3c1a04c7824..fc3079a7aa7 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -144,6 +144,8 @@ ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const modified_context->setSetting("input_format_skip_unknown_fields", true); modified_context->setSetting("input_format_allow_errors_ratio", 0.); modified_context->setSetting("input_format_allow_errors_num", nats_settings->nats_skip_broken_messages.value); + /// Since we are reusing the same context for all queries executed simultaneously, we don't want to used shared `analyze_count` + modified_context->setSetting("max_analyze_depth", Field{0}); if (!schema_name.empty()) modified_context->setSetting("format_schema", schema_name); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f831f81cd22..70838daec24 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -242,6 +242,8 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const modified_context->setSetting("input_format_skip_unknown_fields", true); modified_context->setSetting("input_format_allow_errors_ratio", 0.); modified_context->setSetting("input_format_allow_errors_num", rabbitmq_settings->rabbitmq_skip_broken_messages.value); + /// Since we are reusing the same context for all queries executed simultaneously, we don't want to used shared `analyze_count` + modified_context->setSetting("max_analyze_depth", Field{0}); if (!schema_name.empty()) modified_context->setSetting("format_schema", schema_name); From 60454bd722e7689ef7a4deddab7dca31632b854b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 22 Aug 2022 10:11:27 +0000 Subject: [PATCH 218/582] Fix error codes in InterpreterSelectQuery.cpp --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c596c5233ce..ce6e0c822e5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,7 +98,7 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; extern const int ILLEGAL_FINAL; extern const int ILLEGAL_PREWHERE; - extern const int TOO_DEEP_PIPELINE; + extern const int TOO_DEEP_RECURSION; extern const int TOO_MANY_COLUMNS; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; From 63f7301ee0b6550d955f761f69ad988449058703 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 22 Aug 2022 10:12:28 +0000 Subject: [PATCH 219/582] Create new query context in StorageRabbitMQ/Nat --- src/Storages/NATS/StorageNATS.cpp | 17 ++++++++--------- src/Storages/NATS/StorageNATS.h | 3 +-- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 16 ++++++++-------- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +-- 4 files changed, 18 insertions(+), 21 deletions(-) diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index fc3079a7aa7..e1f5a69d343 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -87,9 +87,6 @@ StorageNATS::StorageNATS( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - nats_context = addSettings(getContext()); - nats_context->makeQueryContext(); - try { connection = std::make_shared(configuration, log); @@ -138,7 +135,7 @@ String StorageNATS::getTableBasedName(String name, const StorageID & table_id) } -ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const +ContextMutablePtr StorageNATS::createNewContext(ContextPtr local_context, bool query_context) const { auto modified_context = Context::createCopy(local_context); modified_context->setSetting("input_format_skip_unknown_fields", true); @@ -158,6 +155,8 @@ ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const if (!setting_name.starts_with("nats_")) modified_context->setSetting(setting_name, setting.getValue()); } + if (query_context) + modified_context->makeQueryContext(); return modified_context; } @@ -293,7 +292,6 @@ void StorageNATS::read( std::lock_guard lock(loop_mutex); auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); - auto modified_context = addSettings(local_context); if (!connection->isConnected()) { @@ -304,6 +302,7 @@ void StorageNATS::read( Pipes pipes; pipes.reserve(num_created_consumers); + auto modified_context = createNewContext(local_context); for (size_t i = 0; i < num_created_consumers; ++i) { auto nats_source = std::make_shared(*this, storage_snapshot, modified_context, column_names, 1); @@ -335,14 +334,14 @@ void StorageNATS::read( { auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); query_plan.addStep(std::move(read_step)); - query_plan.addInterpreterContext(modified_context); + query_plan.addInterpreterContext(createNewContext(local_context)); } } SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - auto modified_context = addSettings(local_context); + auto modified_context = createNewContext(local_context); std::string subject = modified_context->getSettingsRef().stream_like_engine_insert_queue.changed ? modified_context->getSettingsRef().stream_like_engine_insert_queue.value : ""; @@ -612,7 +611,7 @@ bool StorageNATS::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); + InterpreterInsertQuery interpreter(insert, createNewContext(getContext(), true), false, true, true); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); @@ -630,7 +629,7 @@ bool StorageNATS::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { LOG_DEBUG(log, "Current queue size: {}", buffers[0]->queueSize()); - auto source = std::make_shared(*this, storage_snapshot, nats_context, column_names, block_size); + auto source = std::make_shared(*this, storage_snapshot, createNewContext(getContext(), true), column_names, block_size); sources.emplace_back(source); pipes.emplace_back(source); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 185b39250c8..666cc1b099f 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -64,7 +64,6 @@ public: void startStreaming() { if (!mv_attached) { streaming_task->activateAndSchedule(); } } private: - ContextMutablePtr nats_context; std::unique_ptr nats_settings; std::vector subjects; @@ -134,7 +133,7 @@ private: static Names parseList(const String & list, char delim); static String getTableBasedName(String name, const StorageID & table_id); - ContextMutablePtr addSettings(ContextPtr context) const; + ContextMutablePtr createNewContext(ContextPtr local_context, bool query_context) const; size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool stop_loop); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 70838daec24..0999834a342 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -138,9 +138,6 @@ StorageRabbitMQ::StorageRabbitMQ( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - rabbitmq_context = addSettings(getContext()); - rabbitmq_context->makeQueryContext(); - if (queue_base.empty()) { /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to @@ -236,7 +233,7 @@ String StorageRabbitMQ::getTableBasedName(String name, const StorageID & table_i } -ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const +ContextMutablePtr StorageRabbitMQ::createNewContext(ContextPtr local_context, bool query_context) const { auto modified_context = Context::createCopy(local_context); modified_context->setSetting("input_format_skip_unknown_fields", true); @@ -257,6 +254,9 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const modified_context->setSetting(setting_name, setting.getValue()); } + if (query_context) + modified_context->makeQueryContext(); + return modified_context; } @@ -695,7 +695,6 @@ void StorageRabbitMQ::read( std::lock_guard lock(loop_mutex); auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); - auto modified_context = addSettings(local_context); if (!connection->isConnected()) { @@ -710,6 +709,7 @@ void StorageRabbitMQ::read( Pipes pipes; pipes.reserve(num_created_consumers); + auto modified_context = createNewContext(local_context) for (size_t i = 0; i < num_created_consumers; ++i) { auto rabbit_source = std::make_shared( @@ -742,7 +742,7 @@ void StorageRabbitMQ::read( { auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); query_plan.addStep(std::move(read_step)); - query_plan.addInterpreterContext(modified_context); + query_plan.addInterpreterContext(createNewContext(local_context)); } } @@ -1056,7 +1056,7 @@ bool StorageRabbitMQ::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); + InterpreterInsertQuery interpreter(insert, createNewContext(getContext(), true), false, true, true); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); @@ -1074,7 +1074,7 @@ bool StorageRabbitMQ::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { auto source = std::make_shared( - *this, storage_snapshot, rabbitmq_context, column_names, block_size, false); + *this, storage_snapshot, createNewContext(getContext(), true), column_names, block_size, false); sources.emplace_back(source); pipes.emplace_back(source); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 455b2fe8f09..b29ecc48750 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -77,7 +77,6 @@ public: void decrementReader(); private: - ContextMutablePtr rabbitmq_context; std::unique_ptr rabbitmq_settings; const String exchange_name; @@ -177,7 +176,7 @@ private: static AMQP::ExchangeType defineExchangeType(String exchange_type_); static String getTableBasedName(String name, const StorageID & table_id); - ContextMutablePtr addSettings(ContextPtr context) const; + ContextMutablePtr createNewContext(ContextPtr local_context, bool query_context = false) const; size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); From 473c9106a500512a826596c4bdd3042a0b3dc995 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Aug 2022 15:35:52 +0000 Subject: [PATCH 220/582] Revert "Create new query context in StorageRabbitMQ/Nat" This reverts commit 2d6ac7344a25134a625e879da63b9a4aa20d9a69. --- src/Storages/NATS/StorageNATS.cpp | 17 +++++++++-------- src/Storages/NATS/StorageNATS.h | 3 ++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 16 ++++++++-------- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 ++- 4 files changed, 21 insertions(+), 18 deletions(-) diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index e1f5a69d343..fc3079a7aa7 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -87,6 +87,9 @@ StorageNATS::StorageNATS( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + nats_context = addSettings(getContext()); + nats_context->makeQueryContext(); + try { connection = std::make_shared(configuration, log); @@ -135,7 +138,7 @@ String StorageNATS::getTableBasedName(String name, const StorageID & table_id) } -ContextMutablePtr StorageNATS::createNewContext(ContextPtr local_context, bool query_context) const +ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const { auto modified_context = Context::createCopy(local_context); modified_context->setSetting("input_format_skip_unknown_fields", true); @@ -155,8 +158,6 @@ ContextMutablePtr StorageNATS::createNewContext(ContextPtr local_context, bool q if (!setting_name.starts_with("nats_")) modified_context->setSetting(setting_name, setting.getValue()); } - if (query_context) - modified_context->makeQueryContext(); return modified_context; } @@ -292,6 +293,7 @@ void StorageNATS::read( std::lock_guard lock(loop_mutex); auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); + auto modified_context = addSettings(local_context); if (!connection->isConnected()) { @@ -302,7 +304,6 @@ void StorageNATS::read( Pipes pipes; pipes.reserve(num_created_consumers); - auto modified_context = createNewContext(local_context); for (size_t i = 0; i < num_created_consumers; ++i) { auto nats_source = std::make_shared(*this, storage_snapshot, modified_context, column_names, 1); @@ -334,14 +335,14 @@ void StorageNATS::read( { auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); query_plan.addStep(std::move(read_step)); - query_plan.addInterpreterContext(createNewContext(local_context)); + query_plan.addInterpreterContext(modified_context); } } SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - auto modified_context = createNewContext(local_context); + auto modified_context = addSettings(local_context); std::string subject = modified_context->getSettingsRef().stream_like_engine_insert_queue.changed ? modified_context->getSettingsRef().stream_like_engine_insert_queue.value : ""; @@ -611,7 +612,7 @@ bool StorageNATS::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, createNewContext(getContext(), true), false, true, true); + InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); @@ -629,7 +630,7 @@ bool StorageNATS::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { LOG_DEBUG(log, "Current queue size: {}", buffers[0]->queueSize()); - auto source = std::make_shared(*this, storage_snapshot, createNewContext(getContext(), true), column_names, block_size); + auto source = std::make_shared(*this, storage_snapshot, nats_context, column_names, block_size); sources.emplace_back(source); pipes.emplace_back(source); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 666cc1b099f..185b39250c8 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -64,6 +64,7 @@ public: void startStreaming() { if (!mv_attached) { streaming_task->activateAndSchedule(); } } private: + ContextMutablePtr nats_context; std::unique_ptr nats_settings; std::vector subjects; @@ -133,7 +134,7 @@ private: static Names parseList(const String & list, char delim); static String getTableBasedName(String name, const StorageID & table_id); - ContextMutablePtr createNewContext(ContextPtr local_context, bool query_context) const; + ContextMutablePtr addSettings(ContextPtr context) const; size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool stop_loop); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0999834a342..70838daec24 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -138,6 +138,9 @@ StorageRabbitMQ::StorageRabbitMQ( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + rabbitmq_context = addSettings(getContext()); + rabbitmq_context->makeQueryContext(); + if (queue_base.empty()) { /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to @@ -233,7 +236,7 @@ String StorageRabbitMQ::getTableBasedName(String name, const StorageID & table_i } -ContextMutablePtr StorageRabbitMQ::createNewContext(ContextPtr local_context, bool query_context) const +ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const { auto modified_context = Context::createCopy(local_context); modified_context->setSetting("input_format_skip_unknown_fields", true); @@ -254,9 +257,6 @@ ContextMutablePtr StorageRabbitMQ::createNewContext(ContextPtr local_context, bo modified_context->setSetting(setting_name, setting.getValue()); } - if (query_context) - modified_context->makeQueryContext(); - return modified_context; } @@ -695,6 +695,7 @@ void StorageRabbitMQ::read( std::lock_guard lock(loop_mutex); auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); + auto modified_context = addSettings(local_context); if (!connection->isConnected()) { @@ -709,7 +710,6 @@ void StorageRabbitMQ::read( Pipes pipes; pipes.reserve(num_created_consumers); - auto modified_context = createNewContext(local_context) for (size_t i = 0; i < num_created_consumers; ++i) { auto rabbit_source = std::make_shared( @@ -742,7 +742,7 @@ void StorageRabbitMQ::read( { auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); query_plan.addStep(std::move(read_step)); - query_plan.addInterpreterContext(createNewContext(local_context)); + query_plan.addInterpreterContext(modified_context); } } @@ -1056,7 +1056,7 @@ bool StorageRabbitMQ::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, createNewContext(getContext(), true), false, true, true); + InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); @@ -1074,7 +1074,7 @@ bool StorageRabbitMQ::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { auto source = std::make_shared( - *this, storage_snapshot, createNewContext(getContext(), true), column_names, block_size, false); + *this, storage_snapshot, rabbitmq_context, column_names, block_size, false); sources.emplace_back(source); pipes.emplace_back(source); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index b29ecc48750..455b2fe8f09 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -77,6 +77,7 @@ public: void decrementReader(); private: + ContextMutablePtr rabbitmq_context; std::unique_ptr rabbitmq_settings; const String exchange_name; @@ -176,7 +177,7 @@ private: static AMQP::ExchangeType defineExchangeType(String exchange_type_); static String getTableBasedName(String name, const StorageID & table_id); - ContextMutablePtr createNewContext(ContextPtr local_context, bool query_context = false) const; + ContextMutablePtr addSettings(ContextPtr context) const; size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); From 6c15db9b11e3d97ecaa2778fbe06bc2fbda71f0b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Aug 2022 14:20:54 +0000 Subject: [PATCH 221/582] Setting max_analyze_depth forbid only second analyze --- src/Core/Settings.h | 3 +-- src/Interpreters/InterpreterSelectQuery.cpp | 27 ++++++++++++--------- 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c14c411bd49..696d63448f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -346,8 +346,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_temporary_non_const_columns, 0, "Similar to the 'max_temporary_columns' setting but applies only to non-constant columns. This makes sense, because constant columns are cheap and it is reasonable to allow more of them.", 0) \ \ M(UInt64, max_subquery_depth, 100, "If a query has more than specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \ - M(UInt64, max_pipeline_depth, 1000, "If a query has more than specified stages in the query pipeline, throw an exception. Pipeline has stages for every relational operator. This allows to limit the complexity of the queries.", 0) \ - M(UInt64, max_analyze_depth, 10000, "", 0) \ + M(UInt64, max_analyze_depth, 5000, "Maximum number of analyses performed by interpreter.", 0) \ M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \ M(UInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \ M(UInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ce6e0c822e5..2794908ae4a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,7 +98,6 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; extern const int ILLEGAL_FINAL; extern const int ILLEGAL_PREWHERE; - extern const int TOO_DEEP_RECURSION; extern const int TOO_MANY_COLUMNS; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; @@ -497,14 +496,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( auto analyze = [&] (bool try_move_to_prewhere) { - if (context->hasQueryContext()) - { - std::atomic & current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter; - ++current_query_analyze_count; - if (settings.max_analyze_depth && current_query_analyze_count >= settings.max_analyze_depth) - throw DB::Exception(ErrorCodes::TOO_DEEP_RECURSION, "Query analyze overflow. Try to increase `max_analyze_depth` or simplify the query"); - } - /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; if (view) @@ -648,8 +639,18 @@ InterpreterSelectQuery::InterpreterSelectQuery( analyze(shouldMoveToPrewhere()); bool need_analyze_again = false; + bool can_analyze_again = false; + if (context->hasQueryContext()) + { + /// Check number of calls of 'analyze' function. + /// If it is too big, we will not analyze the query again not to have exponential blowup. + std::atomic & current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter; + ++current_query_analyze_count; + can_analyze_again = settings.max_analyze_depth == 0 || current_query_analyze_count < settings.max_analyze_depth; + } - if (analysis_result.prewhere_constant_filter_description.always_false || analysis_result.prewhere_constant_filter_description.always_true) + if (can_analyze_again && (analysis_result.prewhere_constant_filter_description.always_false || + analysis_result.prewhere_constant_filter_description.always_true)) { if (analysis_result.prewhere_constant_filter_description.always_true) query.setExpression(ASTSelectQuery::Expression::PREWHERE, {}); @@ -658,7 +659,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( need_analyze_again = true; } - if (analysis_result.where_constant_filter_description.always_false || analysis_result.where_constant_filter_description.always_true) + if (can_analyze_again && (analysis_result.where_constant_filter_description.always_false || + analysis_result.where_constant_filter_description.always_true)) { if (analysis_result.where_constant_filter_description.always_true) query.setExpression(ASTSelectQuery::Expression::WHERE, {}); @@ -669,7 +671,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (need_analyze_again) { - LOG_TRACE(log, "Running 'analyze' second time"); + size_t current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter.load(); + LOG_TRACE(log, "Running 'analyze' second time (current analyze depth: {})", current_query_analyze_count); /// Reuse already built sets for multiple passes of analysis prepared_sets = query_analyzer->getPreparedSets(); From 33508507bf34567705b82448a7679818918a7a02 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Aug 2022 14:26:36 +0000 Subject: [PATCH 222/582] Upd 02337_join_analyze_stuck --- .../02337_join_analyze_stuck.reference | 26 +++++++++++++++++++ .../0_stateless/02337_join_analyze_stuck.sql | 6 +---- 2 files changed, 27 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02337_join_analyze_stuck.reference b/tests/queries/0_stateless/02337_join_analyze_stuck.reference index e69de29bb2d..4c8ac5b69c5 100644 --- a/tests/queries/0_stateless/02337_join_analyze_stuck.reference +++ b/tests/queries/0_stateless/02337_join_analyze_stuck.reference @@ -0,0 +1,26 @@ +WITH + x AS + ( + SELECT number + FROM numbers(10) + ), + cross_sales AS + ( + SELECT 1 AS xx + FROM + x, + x AS d1, + x AS d2, + x AS d3, + x AS d4, + x AS d5, + x AS d6, + x AS d7, + x AS d8, + x AS d9 + WHERE x.number = d9.number + ) +SELECT xx +FROM +cross_sales +WHERE xx = 2000 diff --git a/tests/queries/0_stateless/02337_join_analyze_stuck.sql b/tests/queries/0_stateless/02337_join_analyze_stuck.sql index 4e3d0bc44c5..43508e8d619 100644 --- a/tests/queries/0_stateless/02337_join_analyze_stuck.sql +++ b/tests/queries/0_stateless/02337_join_analyze_stuck.sql @@ -1,9 +1,5 @@ --- Tags: long - -- https://github.com/ClickHouse/ClickHouse/issues/21557 -SET max_analyze_depth = 1000; - EXPLAIN SYNTAX WITH x AS ( SELECT number FROM numbers(10) ), @@ -12,4 +8,4 @@ WITH FROM x, x AS d1, x AS d2, x AS d3, x AS d4, x AS d5, x AS d6, x AS d7, x AS d8, x AS d9 WHERE x.number = d9.number ) -SELECT xx FROM cross_sales WHERE xx = 2000; -- { serverError TOO_DEEP_RECURSION } +SELECT xx FROM cross_sales WHERE xx = 2000; From 7d8e0250d9cee0ca3f8e435432959f750b948300 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 09:43:48 +0000 Subject: [PATCH 223/582] Mark 02337_join_analyze_stuck long --- tests/queries/0_stateless/02337_join_analyze_stuck.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02337_join_analyze_stuck.sql b/tests/queries/0_stateless/02337_join_analyze_stuck.sql index 43508e8d619..62dd0888673 100644 --- a/tests/queries/0_stateless/02337_join_analyze_stuck.sql +++ b/tests/queries/0_stateless/02337_join_analyze_stuck.sql @@ -1,3 +1,5 @@ +-- Tags: long + -- https://github.com/ClickHouse/ClickHouse/issues/21557 EXPLAIN SYNTAX From 7d5e13c6ba945ba3f2e5ab2675cc4dc7586db850 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 10:40:32 +0000 Subject: [PATCH 224/582] Use async exist for write --- src/Storages/StorageKeeperMap.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 157b35ce995..50c34871410 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -147,17 +147,23 @@ public: current_keys_num = root_stat.numChildren - 1; } + std::vector>> exist_responses; for (const auto & [key, value] : new_values) { auto path = storage.fullPathForKey(key); - if (zookeeper->exists(path)) + exist_responses.push_back({&key, zookeeper->asyncExists(path)}); + } + + for (auto & [key, response] : exist_responses) + { + if (response.get().error == Coordination::Error::ZOK) { - requests.push_back(zkutil::makeSetRequest(path, value, -1)); + requests.push_back(zkutil::makeSetRequest(storage.fullPathForKey(*key), new_values[*key], -1)); } else { - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeCreateRequest(storage.fullPathForKey(*key), new_values[*key], zkutil::CreateMode::Persistent)); ++new_keys_num; } } From ebcc022782e51af1f4e8c5db6d787ac2286a406c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Aug 2022 12:48:12 +0200 Subject: [PATCH 225/582] Fix fast test --- .../0_stateless/01283_max_threads_simple_query_optimization.sql | 1 + tests/queries/0_stateless/01323_too_many_threads_bug.sql | 1 + .../01524_do_not_merge_across_partitions_select_final.sql | 1 + tests/queries/0_stateless/02263_lazy_mark_load.sh | 2 +- 4 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql b/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql index d5f731568d1..2814c87c933 100644 --- a/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql +++ b/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS data_01283; set remote_filesystem_read_method = 'read'; set local_filesystem_read_method = 'pread'; +set load_marks_asynchronously = 0; CREATE TABLE data_01283 engine=MergeTree() ORDER BY key diff --git a/tests/queries/0_stateless/01323_too_many_threads_bug.sql b/tests/queries/0_stateless/01323_too_many_threads_bug.sql index d3254d49728..c2cce81d200 100644 --- a/tests/queries/0_stateless/01323_too_many_threads_bug.sql +++ b/tests/queries/0_stateless/01323_too_many_threads_bug.sql @@ -2,6 +2,7 @@ drop table if exists table_01323_many_parts; set remote_filesystem_read_method = 'read'; set local_filesystem_read_method = 'pread'; +set load_marks_asynchronously = 0; create table table_01323_many_parts (x UInt64) engine = MergeTree order by x partition by x % 100; set max_partitions_per_insert_block = 100; diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index a5423d1a3ff..de228c90753 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -33,6 +33,7 @@ OPTIMIZE TABLE select_final FINAL; SET remote_filesystem_read_method = 'read'; SET local_filesystem_read_method = 'pread'; +set load_marks_asynchronously = 0; SELECT max(x) FROM select_final FINAL; diff --git a/tests/queries/0_stateless/02263_lazy_mark_load.sh b/tests/queries/0_stateless/02263_lazy_mark_load.sh index b2558b5741c..bf37556bfa6 100755 --- a/tests/queries/0_stateless/02263_lazy_mark_load.sh +++ b/tests/queries/0_stateless/02263_lazy_mark_load.sh @@ -30,7 +30,7 @@ EOF ${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test" ${CLICKHOUSE_CLIENT} -q "INSERT INTO lazy_mark_test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)" ${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE" -${CLICKHOUSE_CLIENT} --log_queries=1 --query_id "${QUERY_ID}" -q "SELECT * FROM lazy_mark_test WHERE n3==11" +${CLICKHOUSE_CLIENT} --log_queries=1 --query_id "${QUERY_ID}" -q "SELECT * FROM lazy_mark_test WHERE n3==11 SETTINGS load_marks_asynchronously=0" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -q "select ProfileEvents['FileOpen'] from system.query_log where query_id = '${QUERY_ID}' and type = 'QueryFinish' and current_database = currentDatabase()" From cedf75ed5ebf2912490200ee82b6a110afc7bb70 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 Aug 2022 13:54:57 +0000 Subject: [PATCH 226/582] Enable clang-tidy for headers clang-tidy now also checks code in header files. Because the analyzer finds tons of issues, activate the check only for directory "base/" (see file ".clang-tidy"). All other directories, in particular "src/" are left to future work. While many findings were fixed, some were not (and suppressed instead). Reasons for this include: a) the file is 1:1 copypaste of a 3rd-party lib (e.g. pcg_extras.h) and fixing stuff would make upgrades/fixes more difficult b) a fix would have broken lots of using code --- .clang-tidy | 8 +++ base/base/Decimal.h | 8 +-- base/base/DecomposedFloat.h | 66 +++++++++---------- base/base/JSON.h | 14 ++-- base/base/arithmeticOverflow.h | 3 + base/base/bit_cast.h | 2 +- base/base/defines.h | 4 +- base/base/extended_types.h | 18 +++-- base/base/iostream_debug_helpers.h | 1 + base/base/itoa.h | 10 +-- base/base/scope_guard.h | 25 ++++--- base/base/sort.h | 2 +- base/base/strong_typedef.h | 5 ++ base/base/unit.h | 2 + base/base/wide_integer_to_string.h | 4 +- base/pcg-random/pcg_extras.hpp | 6 +- src/Backups/BackupImpl.cpp | 2 +- src/IO/WriteHelpers.h | 4 +- src/Processors/Merges/Algorithms/Graphite.cpp | 12 ++-- tests/instructions/clang-tidy.txt | 2 - 20 files changed, 104 insertions(+), 94 deletions(-) delete mode 100644 tests/instructions/clang-tidy.txt diff --git a/.clang-tidy b/.clang-tidy index 2ca1402ddf1..860e7b3189f 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -1,6 +1,14 @@ +# To run clang-tidy from CMake, build ClickHouse with -DENABLE_CLANG_TIDY=1. To show all warnings, it is +# recommended to pass "-k0" to Ninja. + # Enable all checks + disale selected checks. Feel free to remove disabled checks from below list if # a) the new check is not controversial (this includes many checks in readability-* and google-*) or # b) too noisy (checks with > 100 new warnings are considered noisy, this includes e.g. cppcoreguidelines-*). + +# TODO Let clang-tidy check headers in further directories +# --> HeaderFilterRegex: '^.*/(src|base|programs|utils)/.*(h|hpp)$' +HeaderFilterRegex: '^.*/(base)/.*(h|hpp)$' + Checks: '*, -abseil-*, diff --git a/base/base/Decimal.h b/base/base/Decimal.h index 1efb8ba8d92..22cb577b1b2 100644 --- a/base/base/Decimal.h +++ b/base/base/Decimal.h @@ -52,15 +52,15 @@ struct Decimal constexpr Decimal(Decimal &&) noexcept = default; constexpr Decimal(const Decimal &) = default; - constexpr Decimal(const T & value_): value(value_) {} + constexpr Decimal(const T & value_): value(value_) {} // NOLINT(google-explicit-constructor) template - constexpr Decimal(const Decimal & x): value(x.value) {} + constexpr Decimal(const Decimal & x): value(x.value) {} // NOLINT(google-explicit-constructor) constexpr Decimal & operator=(Decimal &&) noexcept = default; constexpr Decimal & operator = (const Decimal &) = default; - constexpr operator T () const { return value; } + constexpr operator T () const { return value; } // NOLINT(google-explicit-constructor) template constexpr U convertTo() const @@ -111,7 +111,7 @@ public: using Base::Base; using NativeType = Base::NativeType; - constexpr DateTime64(const Base & v): Base(v) {} + constexpr DateTime64(const Base & v): Base(v) {} // NOLINT(google-explicit-constructor) }; } diff --git a/base/base/DecomposedFloat.h b/base/base/DecomposedFloat.h index 652b28966b2..f152637b94e 100644 --- a/base/base/DecomposedFloat.h +++ b/base/base/DecomposedFloat.h @@ -36,14 +36,14 @@ struct DecomposedFloat { using Traits = FloatTraits; - DecomposedFloat(T x) + explicit DecomposedFloat(T x) { memcpy(&x_uint, &x, sizeof(x)); } typename Traits::UInt x_uint; - bool is_negative() const + bool isNegative() const { return x_uint >> (Traits::bits - 1); } @@ -53,7 +53,7 @@ struct DecomposedFloat { return (exponent() == 0 && mantissa() == 0) ? 0 - : (is_negative() + : (isNegative() ? -1 : 1); } @@ -63,7 +63,7 @@ struct DecomposedFloat return (x_uint >> (Traits::mantissa_bits)) & (((1ull << (Traits::exponent_bits + 1)) - 1) >> 1); } - int16_t normalized_exponent() const + int16_t normalizedExponent() const { return int16_t(exponent()) - ((1ull << (Traits::exponent_bits - 1)) - 1); } @@ -73,20 +73,20 @@ struct DecomposedFloat return x_uint & ((1ull << Traits::mantissa_bits) - 1); } - int64_t mantissa_with_sign() const + int64_t mantissaWithSign() const { - return is_negative() ? -mantissa() : mantissa(); + return isNegative() ? -mantissa() : mantissa(); } /// NOTE Probably floating point instructions can be better. - bool is_integer_in_representable_range() const + bool isIntegerInRepresentableRange() const { return x_uint == 0 - || (normalized_exponent() >= 0 /// The number is not less than one + || (normalizedExponent() >= 0 /// The number is not less than one /// The number is inside the range where every integer has exact representation in float - && normalized_exponent() <= static_cast(Traits::mantissa_bits) + && normalizedExponent() <= static_cast(Traits::mantissa_bits) /// After multiplying by 2^exp, the fractional part becomes zero, means the number is integer - && ((mantissa() & ((1ULL << (Traits::mantissa_bits - normalized_exponent())) - 1)) == 0)); + && ((mantissa() & ((1ULL << (Traits::mantissa_bits - normalizedExponent())) - 1)) == 0)); } @@ -102,15 +102,15 @@ struct DecomposedFloat return sign(); /// Different signs - if (is_negative() && rhs > 0) + if (isNegative() && rhs > 0) return -1; - if (!is_negative() && rhs < 0) + if (!isNegative() && rhs < 0) return 1; /// Fractional number with magnitude less than one - if (normalized_exponent() < 0) + if (normalizedExponent() < 0) { - if (!is_negative()) + if (!isNegative()) return rhs > 0 ? -1 : 1; else return rhs >= 0 ? -1 : 1; @@ -121,11 +121,11 @@ struct DecomposedFloat { if (rhs == std::numeric_limits::lowest()) { - assert(is_negative()); + assert(isNegative()); - if (normalized_exponent() < static_cast(8 * sizeof(Int) - is_signed_v)) + if (normalizedExponent() < static_cast(8 * sizeof(Int) - is_signed_v)) return 1; - if (normalized_exponent() > static_cast(8 * sizeof(Int) - is_signed_v)) + if (normalizedExponent() > static_cast(8 * sizeof(Int) - is_signed_v)) return -1; if (mantissa() == 0) @@ -136,44 +136,44 @@ struct DecomposedFloat } /// Too large number: abs(float) > abs(rhs). Also the case with infinities and NaN. - if (normalized_exponent() >= static_cast(8 * sizeof(Int) - is_signed_v)) - return is_negative() ? -1 : 1; + if (normalizedExponent() >= static_cast(8 * sizeof(Int) - is_signed_v)) + return isNegative() ? -1 : 1; using UInt = std::conditional_t<(sizeof(Int) > sizeof(typename Traits::UInt)), make_unsigned_t, typename Traits::UInt>; UInt uint_rhs = rhs < 0 ? -rhs : rhs; /// Smaller octave: abs(rhs) < abs(float) /// FYI, TIL: octave is also called "binade", https://en.wikipedia.org/wiki/Binade - if (uint_rhs < (static_cast(1) << normalized_exponent())) - return is_negative() ? -1 : 1; + if (uint_rhs < (static_cast(1) << normalizedExponent())) + return isNegative() ? -1 : 1; /// Larger octave: abs(rhs) > abs(float) - if (normalized_exponent() + 1 < static_cast(8 * sizeof(Int) - is_signed_v) - && uint_rhs >= (static_cast(1) << (normalized_exponent() + 1))) - return is_negative() ? 1 : -1; + if (normalizedExponent() + 1 < static_cast(8 * sizeof(Int) - is_signed_v) + && uint_rhs >= (static_cast(1) << (normalizedExponent() + 1))) + return isNegative() ? 1 : -1; /// The same octave - /// uint_rhs == 2 ^ normalized_exponent + mantissa * 2 ^ (normalized_exponent - mantissa_bits) + /// uint_rhs == 2 ^ normalizedExponent + mantissa * 2 ^ (normalizedExponent - mantissa_bits) - bool large_and_always_integer = normalized_exponent() >= static_cast(Traits::mantissa_bits); + bool large_and_always_integer = normalizedExponent() >= static_cast(Traits::mantissa_bits); UInt a = large_and_always_integer - ? static_cast(mantissa()) << (normalized_exponent() - Traits::mantissa_bits) - : static_cast(mantissa()) >> (Traits::mantissa_bits - normalized_exponent()); + ? static_cast(mantissa()) << (normalizedExponent() - Traits::mantissa_bits) + : static_cast(mantissa()) >> (Traits::mantissa_bits - normalizedExponent()); - UInt b = uint_rhs - (static_cast(1) << normalized_exponent()); + UInt b = uint_rhs - (static_cast(1) << normalizedExponent()); if (a < b) - return is_negative() ? 1 : -1; + return isNegative() ? 1 : -1; if (a > b) - return is_negative() ? -1 : 1; + return isNegative() ? -1 : 1; /// Float has no fractional part means that the numbers are equal. - if (large_and_always_integer || (mantissa() & ((1ULL << (Traits::mantissa_bits - normalized_exponent())) - 1)) == 0) + if (large_and_always_integer || (mantissa() & ((1ULL << (Traits::mantissa_bits - normalizedExponent())) - 1)) == 0) return 0; else /// Float has fractional part means its abs value is larger. - return is_negative() ? -1 : 1; + return isNegative() ? -1 : 1; } diff --git a/base/base/JSON.h b/base/base/JSON.h index 214e9f88e9b..850b74715c6 100644 --- a/base/base/JSON.h +++ b/base/base/JSON.h @@ -38,6 +38,7 @@ */ +// NOLINTBEGIN(google-explicit-constructor) #ifdef __clang__ # pragma clang diagnostic push # pragma clang diagnostic ignored "-Wdeprecated-dynamic-exception-spec" @@ -46,6 +47,7 @@ POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception) #ifdef __clang__ # pragma clang diagnostic pop #endif +// NOLINTEND(google-explicit-constructor) class JSON { @@ -61,7 +63,7 @@ public: checkInit(); } - JSON(const std::string & s) : ptr_begin(s.data()), ptr_end(s.data() + s.size()), level(0) + explicit JSON(std::string_view s) : ptr_begin(s.data()), ptr_end(s.data() + s.size()), level(0) { checkInit(); } @@ -71,13 +73,7 @@ public: *this = rhs; } - JSON & operator=(const JSON & rhs) - { - ptr_begin = rhs.ptr_begin; - ptr_end = rhs.ptr_end; - level = rhs.level; - return *this; - } + JSON & operator=(const JSON & rhs) = default; const char * data() const { return ptr_begin; } const char * dataEnd() const { return ptr_end; } @@ -169,7 +165,7 @@ public: /// Перейти к следующему элементу массива или следующей name-value паре объекта. iterator & operator++(); - iterator operator++(int); + iterator operator++(int); // NOLINT(cert-dcl21-cpp) /// Есть ли в строке escape-последовательности bool hasEscapes() const; diff --git a/base/base/arithmeticOverflow.h b/base/base/arithmeticOverflow.h index 9a0e27505e1..d7242058658 100644 --- a/base/base/arithmeticOverflow.h +++ b/base/base/arithmeticOverflow.h @@ -3,6 +3,7 @@ #include #include +// NOLINTBEGIN(google-runtime-int) namespace common { @@ -206,3 +207,5 @@ namespace common return false; } } + +// NOLINTEND(google-runtime-int) diff --git a/base/base/bit_cast.h b/base/base/bit_cast.h index 5b4b0931b62..d1246b45590 100644 --- a/base/base/bit_cast.h +++ b/base/base/bit_cast.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/base/base/defines.h b/base/base/defines.h index c8c408b9c93..671253ed9e8 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -143,8 +143,8 @@ /// Macros for suppressing TSA warnings for specific reads/writes (instead of suppressing it for the whole function) /// Consider adding a comment before using these macros. -# define TSA_SUPPRESS_WARNING_FOR_READ(x) [&]() TSA_NO_THREAD_SAFETY_ANALYSIS -> const auto & { return (x); }() -# define TSA_SUPPRESS_WARNING_FOR_WRITE(x) [&]() TSA_NO_THREAD_SAFETY_ANALYSIS -> auto & { return (x); }() +# define TSA_SUPPRESS_WARNING_FOR_READ(x) ([&]() TSA_NO_THREAD_SAFETY_ANALYSIS -> const auto & { return (x); }()) +# define TSA_SUPPRESS_WARNING_FOR_WRITE(x) ([&]() TSA_NO_THREAD_SAFETY_ANALYSIS -> auto & { return (x); }()) /// This macro is useful when only one thread writes to a member /// and you want to read this member from the same thread without locking a mutex. diff --git a/base/base/extended_types.h b/base/base/extended_types.h index 7deb8e17b8e..b58df45a97e 100644 --- a/base/base/extended_types.h +++ b/base/base/extended_types.h @@ -5,7 +5,6 @@ #include #include - using Int128 = wide::integer<128, signed>; using UInt128 = wide::integer<128, unsigned>; using Int256 = wide::integer<256, signed>; @@ -18,7 +17,7 @@ static_assert(sizeof(UInt256) == 32); /// (std::common_type), are "set in stone". Attempting to specialize them causes undefined behavior. /// So instead of using the std type_traits, we use our own version which allows extension. template -struct is_signed +struct is_signed // NOLINT(readability-identifier-naming) { static constexpr bool value = std::is_signed_v; }; @@ -30,7 +29,7 @@ template inline constexpr bool is_signed_v = is_signed::value; template -struct is_unsigned +struct is_unsigned // NOLINT(readability-identifier-naming) { static constexpr bool value = std::is_unsigned_v; }; @@ -51,7 +50,7 @@ template concept is_integer = template concept is_floating_point = std::is_floating_point_v; template -struct is_arithmetic +struct is_arithmetic // NOLINT(readability-identifier-naming) { static constexpr bool value = std::is_arithmetic_v; }; @@ -66,9 +65,9 @@ template inline constexpr bool is_arithmetic_v = is_arithmetic::value; template -struct make_unsigned +struct make_unsigned // NOLINT(readability-identifier-naming) { - typedef std::make_unsigned_t type; + using type = std::make_unsigned_t; }; template <> struct make_unsigned { using type = UInt128; }; @@ -79,9 +78,9 @@ template <> struct make_unsigned { using type = UInt256; }; template using make_unsigned_t = typename make_unsigned::type; template -struct make_signed +struct make_signed // NOLINT(readability-identifier-naming) { - typedef std::make_signed_t type; + using type = std::make_signed_t; }; template <> struct make_signed { using type = Int128; }; @@ -92,7 +91,7 @@ template <> struct make_signed { using type = Int256; }; template using make_signed_t = typename make_signed::type; template -struct is_big_int +struct is_big_int // NOLINT(readability-identifier-naming) { static constexpr bool value = false; }; @@ -104,4 +103,3 @@ template <> struct is_big_int { static constexpr bool value = true; }; template inline constexpr bool is_big_int_v = is_big_int::value; - diff --git a/base/base/iostream_debug_helpers.h b/base/base/iostream_debug_helpers.h index 3a3f1a741ad..db974c911df 100644 --- a/base/base/iostream_debug_helpers.h +++ b/base/base/iostream_debug_helpers.h @@ -120,6 +120,7 @@ Out & dumpDispatchPriorities(Out & out, T && x, std::decay_t(out, x); } +// NOLINTNEXTLINE(google-explicit-constructor) struct LowPriority { LowPriority(void *) {} }; template diff --git a/base/base/itoa.h b/base/base/itoa.h index da7c2ffc73e..33457f73411 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -91,10 +91,10 @@ template using DivisionBy10PowN = typename SelectType < N, - Division, /// divide by 10 - Division, /// divide by 100 - Division, /// divide by 10000 - Division /// divide by 100000000 + Division, /// divide by 10 + Division, /// divide by 100 + Division, /// divide by 10000 + Division /// divide by 100000000 >::Result; template @@ -352,7 +352,7 @@ static inline char * writeUIntText(T x, char * p) static_assert(is_unsigned_v); int len = digits10(x); - auto pp = p + len; + auto *pp = p + len; while (x >= 100) { const auto i = x % 100; diff --git a/base/base/scope_guard.h b/base/base/scope_guard.h index 3d8f8ba0fe0..fca4f838cab 100644 --- a/base/base/scope_guard.h +++ b/base/base/scope_guard.h @@ -5,13 +5,13 @@ #include template -class [[nodiscard]] basic_scope_guard +class [[nodiscard]] BasicScopeBuard { public: - constexpr basic_scope_guard() = default; - constexpr basic_scope_guard(basic_scope_guard && src) : function{src.release()} {} + constexpr BasicScopeBuard() = default; + constexpr BasicScopeBuard(BasicScopeBuard && src) : function{src.release()} {} // NOLINT(hicpp-noexcept-move, performance-noexcept-move-constructor) - constexpr basic_scope_guard & operator=(basic_scope_guard && src) + constexpr BasicScopeBuard & operator=(BasicScopeBuard && src) // NOLINT(hicpp-noexcept-move, performance-noexcept-move-constructor) { if (this != &src) { @@ -23,11 +23,11 @@ public: template requires std::is_convertible_v - constexpr basic_scope_guard(basic_scope_guard && src) : function{src.release()} {} + constexpr BasicScopeBuard(BasicScopeBuard && src) : function{src.release()} {} // NOLINT(google-explicit-constructor) template requires std::is_convertible_v - constexpr basic_scope_guard & operator=(basic_scope_guard && src) + constexpr BasicScopeBuard & operator=(BasicScopeBuard && src) { if (this != &src) { @@ -39,13 +39,13 @@ public: template requires std::is_convertible_v - constexpr basic_scope_guard(const G & function_) : function{function_} {} + constexpr BasicScopeBuard(const G & function_) : function{function_} {} // NOLINT(google-explicit-constructor) template requires std::is_convertible_v - constexpr basic_scope_guard(G && function_) : function{std::move(function_)} {} + constexpr BasicScopeBuard(G && function_) : function{std::move(function_)} {} // NOLINT(google-explicit-constructor, bugprone-forwarding-reference-overload, bugprone-move-forwarding-reference) - ~basic_scope_guard() { invoke(); } + ~BasicScopeBuard() { invoke(); } static constexpr bool is_nullable = std::is_constructible_v; @@ -70,7 +70,7 @@ public: template requires std::is_convertible_v - basic_scope_guard & join(basic_scope_guard && other) + BasicScopeBuard & join(BasicScopeBuard && other) { if (other.function) { @@ -102,14 +102,13 @@ private: F function = F{}; }; -using scope_guard = basic_scope_guard>; +using scope_guard = BasicScopeBuard>; template -inline basic_scope_guard make_scope_guard(F && function_) { return std::forward(function_); } +inline BasicScopeBuard make_scope_guard(F && function_) { return std::forward(function_); } #define SCOPE_EXIT_CONCAT(n, ...) \ const auto scope_exit##n = make_scope_guard([&] { __VA_ARGS__; }) #define SCOPE_EXIT_FWD(n, ...) SCOPE_EXIT_CONCAT(n, __VA_ARGS__) #define SCOPE_EXIT(...) SCOPE_EXIT_FWD(__LINE__, __VA_ARGS__) - diff --git a/base/base/sort.h b/base/base/sort.h index 589469fffaa..912545979dc 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -14,7 +14,7 @@ template class DebugLessComparator { public: - constexpr DebugLessComparator(Comparator & cmp_) + constexpr DebugLessComparator(Comparator & cmp_) // NOLINT(google-explicit-constructor) : cmp(cmp_) {} diff --git a/base/base/strong_typedef.h b/base/base/strong_typedef.h index c9ea30b73fd..2ddea6412f5 100644 --- a/base/base/strong_typedef.h +++ b/base/base/strong_typedef.h @@ -34,8 +34,10 @@ public: template ::type> Self & operator=(T && rhs) { t = std::move(rhs); return *this;} + // NOLINTBEGIN(google-explicit-constructor) operator const T & () const { return t; } operator T & () { return t; } + // NOLINTEND(google-explicit-constructor) bool operator==(const Self & rhs) const { return t == rhs.t; } bool operator<(const Self & rhs) const { return t < rhs.t; } @@ -58,7 +60,10 @@ namespace std }; } +// NOLINTBEGIN(bugprone-macro-parentheses) + #define STRONG_TYPEDEF(T, D) \ struct D ## Tag {}; \ using D = StrongTypedef; \ +// NOLINTEND(bugprone-macro-parentheses) diff --git a/base/base/unit.h b/base/base/unit.h index 682b43512fc..1fb530be1f0 100644 --- a/base/base/unit.h +++ b/base/base/unit.h @@ -10,9 +10,11 @@ constexpr size_t GiB = 1024 * MiB; # pragma clang diagnostic ignored "-Wreserved-identifier" #endif +// NOLINTBEGIN(google-runtime-int) constexpr size_t operator"" _KiB(unsigned long long val) { return val * KiB; } constexpr size_t operator"" _MiB(unsigned long long val) { return val * MiB; } constexpr size_t operator"" _GiB(unsigned long long val) { return val * GiB; } +// NOLINTEND(google-runtime-int) #ifdef HAS_RESERVED_IDENTIFIER # pragma clang diagnostic pop diff --git a/base/base/wide_integer_to_string.h b/base/base/wide_integer_to_string.h index 8b794fe9bcb..160bf599516 100644 --- a/base/base/wide_integer_to_string.h +++ b/base/base/wide_integer_to_string.h @@ -51,8 +51,8 @@ struct fmt::formatter> { constexpr auto parse(format_parse_context & ctx) { - auto it = ctx.begin(); - auto end = ctx.end(); + const auto * it = ctx.begin(); + const auto * end = ctx.end(); /// Only support {}. if (it != end && *it != '}') diff --git a/base/pcg-random/pcg_extras.hpp b/base/pcg-random/pcg_extras.hpp index f5ba4d48849..78ce726d48b 100644 --- a/base/pcg-random/pcg_extras.hpp +++ b/base/pcg-random/pcg_extras.hpp @@ -49,6 +49,8 @@ #include #endif +// NOLINTBEGIN(readability-identifier-naming, modernize-use-using, bugprone-macro-parentheses, google-explicit-constructor) + /* * Abstractions for compiler-specific directives */ @@ -90,8 +92,6 @@ #define PCG_EMULATED_128BIT_MATH 1 #endif -// NOLINTBEGIN(*) - namespace pcg_extras { /* @@ -553,6 +553,6 @@ std::ostream& operator<<(std::ostream& out, printable_typename) { } // namespace pcg_extras -// NOLINTEND(*) +// NOLINTEND(readability-identifier-naming, modernize-use-using, bugprone-macro-parentheses, google-explicit-constructor) #endif // PCG_EXTRAS_HPP_INCLUDED diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index f6442545f48..9f0f49824cb 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -537,7 +537,7 @@ SizeAndChecksum BackupImpl::getFileSizeAndChecksum(const String & file_name) con if (!info) throw Exception( ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name)); - return std::pair(info->size, info->checksum); + return {info->size, info->checksum}; } BackupEntryPtr BackupImpl::readFile(const String & file_name) const diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index f9892ac6194..7acea87c0d7 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -146,14 +146,14 @@ inline size_t writeFloatTextFastPath(T x, char * buffer) /// The library Ryu has low performance on integers. /// This workaround improves performance 6..10 times. - if (DecomposedFloat64(x).is_integer_in_representable_range()) + if (DecomposedFloat64(x).isIntegerInRepresentableRange()) result = itoa(Int64(x), buffer) - buffer; else result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; } else { - if (DecomposedFloat32(x).is_integer_in_representable_range()) + if (DecomposedFloat32(x).isIntegerInRepresentableRange()) result = itoa(Int32(x), buffer) - buffer; else result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index 2448a1e2a94..c5c611366ff 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -103,17 +103,17 @@ Graphite::RollupRule selectPatternForPath( if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll) { /// There is only default pattern for both retention and aggregation - return std::pair(&pattern, &pattern); + return {&pattern, &pattern}; } if (pattern.type != first_match->type) { if (first_match->type == first_match->TypeRetention) { - return std::pair(first_match, &pattern); + return {first_match, &pattern}; } if (first_match->type == first_match->TypeAggregation) { - return std::pair(&pattern, first_match); + return {&pattern, first_match}; } } } @@ -125,7 +125,7 @@ Graphite::RollupRule selectPatternForPath( if (pattern.type == pattern.TypeAll) { /// Only for not default patterns with both function and retention parameters - return std::pair(&pattern, &pattern); + return {&pattern, &pattern}; } if (first_match->type == first_match->TypeUndef) { @@ -136,11 +136,11 @@ Graphite::RollupRule selectPatternForPath( { if (first_match->type == first_match->TypeRetention) { - return std::pair(first_match, &pattern); + return {first_match, &pattern}; } if (first_match->type == first_match->TypeAggregation) { - return std::pair(&pattern, first_match); + return {&pattern, first_match}; } } } diff --git a/tests/instructions/clang-tidy.txt b/tests/instructions/clang-tidy.txt deleted file mode 100644 index 84145564bf0..00000000000 --- a/tests/instructions/clang-tidy.txt +++ /dev/null @@ -1,2 +0,0 @@ -# clang-tidy has been integrated into CMake: -# --> Build ClickHouse with -DENABLE_CLANG_TIDY=1 and see cmake/clang_tidy.cmake for details From d43f4e6317b9528b7a6843c1b0e7663499d6c342 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Aug 2022 10:57:27 +0000 Subject: [PATCH 227/582] fix: typo --- base/base/scope_guard.h | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/base/base/scope_guard.h b/base/base/scope_guard.h index fca4f838cab..8524beac7ea 100644 --- a/base/base/scope_guard.h +++ b/base/base/scope_guard.h @@ -5,13 +5,13 @@ #include template -class [[nodiscard]] BasicScopeBuard +class [[nodiscard]] BasicScopeGuard { public: - constexpr BasicScopeBuard() = default; - constexpr BasicScopeBuard(BasicScopeBuard && src) : function{src.release()} {} // NOLINT(hicpp-noexcept-move, performance-noexcept-move-constructor) + constexpr BasicScopeGuard() = default; + constexpr BasicScopeGuard(BasicScopeGuard && src) : function{src.release()} {} // NOLINT(hicpp-noexcept-move, performance-noexcept-move-constructor) - constexpr BasicScopeBuard & operator=(BasicScopeBuard && src) // NOLINT(hicpp-noexcept-move, performance-noexcept-move-constructor) + constexpr BasicScopeGuard & operator=(BasicScopeGuard && src) // NOLINT(hicpp-noexcept-move, performance-noexcept-move-constructor) { if (this != &src) { @@ -23,11 +23,11 @@ public: template requires std::is_convertible_v - constexpr BasicScopeBuard(BasicScopeBuard && src) : function{src.release()} {} // NOLINT(google-explicit-constructor) + constexpr BasicScopeGuard(BasicScopeGuard && src) : function{src.release()} {} // NOLINT(google-explicit-constructor) template requires std::is_convertible_v - constexpr BasicScopeBuard & operator=(BasicScopeBuard && src) + constexpr BasicScopeGuard & operator=(BasicScopeGuard && src) { if (this != &src) { @@ -39,13 +39,13 @@ public: template requires std::is_convertible_v - constexpr BasicScopeBuard(const G & function_) : function{function_} {} // NOLINT(google-explicit-constructor) + constexpr BasicScopeGuard(const G & function_) : function{function_} {} // NOLINT(google-explicit-constructor) template requires std::is_convertible_v - constexpr BasicScopeBuard(G && function_) : function{std::move(function_)} {} // NOLINT(google-explicit-constructor, bugprone-forwarding-reference-overload, bugprone-move-forwarding-reference) + constexpr BasicScopeGuard(G && function_) : function{std::move(function_)} {} // NOLINT(google-explicit-constructor, bugprone-forwarding-reference-overload, bugprone-move-forwarding-reference) - ~BasicScopeBuard() { invoke(); } + ~BasicScopeGuard() { invoke(); } static constexpr bool is_nullable = std::is_constructible_v; @@ -70,7 +70,7 @@ public: template requires std::is_convertible_v - BasicScopeBuard & join(BasicScopeBuard && other) + BasicScopeGuard & join(BasicScopeGuard && other) { if (other.function) { @@ -102,11 +102,11 @@ private: F function = F{}; }; -using scope_guard = BasicScopeBuard>; +using scope_guard = BasicScopeGuard>; template -inline BasicScopeBuard make_scope_guard(F && function_) { return std::forward(function_); } +inline BasicScopeGuard make_scope_guard(F && function_) { return std::forward(function_); } #define SCOPE_EXIT_CONCAT(n, ...) \ const auto scope_exit##n = make_scope_guard([&] { __VA_ARGS__; }) From 007ae0e6cc5e92fc04df7590458f68a79f3432d8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 30 Aug 2022 23:36:02 +0200 Subject: [PATCH 228/582] Fix incremental backups for Log family. --- src/Backups/BackupImpl.cpp | 2 +- src/IO/ConcatSeekableReadBuffer.cpp | 5 ++ src/IO/ConcatSeekableReadBuffer.h | 2 +- src/IO/HashingReadBuffer.h | 19 +++-- .../PartMetadataManagerWithCache.cpp | 1 + .../test_backup_restore_new/test.py | 83 +++++++++++++++++++ 6 files changed, 105 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index f6442545f48..c9bd0be8e47 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -625,7 +625,7 @@ CheckBackupResult checkBaseBackupForFile(const SizeAndChecksum & base_backup_inf { /// We cannot reuse base backup because our file is smaller /// than file stored in previous backup - if (new_entry_info.size > base_backup_info.first) + if (new_entry_info.size < base_backup_info.first) return CheckBackupResult::HasNothing; if (base_backup_info.first == new_entry_info.size) diff --git a/src/IO/ConcatSeekableReadBuffer.cpp b/src/IO/ConcatSeekableReadBuffer.cpp index c5d48376e2f..0943d1eac45 100644 --- a/src/IO/ConcatSeekableReadBuffer.cpp +++ b/src/IO/ConcatSeekableReadBuffer.cpp @@ -9,6 +9,11 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } +ConcatSeekableReadBuffer::BufferInfo::BufferInfo(BufferInfo && src) noexcept + : in(std::exchange(src.in, nullptr)), own_in(std::exchange(src.own_in, false)), size(std::exchange(src.size, 0)) +{ +} + ConcatSeekableReadBuffer::BufferInfo::~BufferInfo() { if (own_in) diff --git a/src/IO/ConcatSeekableReadBuffer.h b/src/IO/ConcatSeekableReadBuffer.h index 5d7dca82524..c8c16c5d887 100644 --- a/src/IO/ConcatSeekableReadBuffer.h +++ b/src/IO/ConcatSeekableReadBuffer.h @@ -30,7 +30,7 @@ private: struct BufferInfo { BufferInfo() = default; - BufferInfo(BufferInfo &&) = default; + BufferInfo(BufferInfo && src) noexcept; ~BufferInfo(); SeekableReadBuffer * in = nullptr; bool own_in = false; diff --git a/src/IO/HashingReadBuffer.h b/src/IO/HashingReadBuffer.h index 5d42c64478c..a0a029e6f85 100644 --- a/src/IO/HashingReadBuffer.h +++ b/src/IO/HashingReadBuffer.h @@ -18,29 +18,38 @@ public: { working_buffer = in.buffer(); pos = in.position(); + hashing_begin = pos; + } - /// calculate hash from the data already read - if (!working_buffer.empty()) + uint128 getHash() + { + if (pos > hashing_begin) { - calculateHash(pos, working_buffer.end() - pos); + calculateHash(hashing_begin, pos - hashing_begin); + hashing_begin = pos; } + return IHashingBuffer::getHash(); } private: bool nextImpl() override { + if (pos > hashing_begin) + calculateHash(hashing_begin, pos - hashing_begin); + in.position() = pos; bool res = in.next(); working_buffer = in.buffer(); - pos = in.position(); // `pos` may be different from working_buffer.begin() when using sophisticated ReadBuffers. - calculateHash(pos, working_buffer.end() - pos); + pos = in.position(); + hashing_begin = pos; return res; } ReadBuffer & in; + BufferBase::Position hashing_begin; }; } diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index 9930aca2576..5a291373e6c 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -191,6 +191,7 @@ void PartMetadataManagerWithCache::getKeysAndCheckSums(Strings & keys, std::vect { ReadBufferFromString rbuf(values[i]); HashingReadBuffer hbuf(rbuf); + hbuf.ignoreAll(); checksums.push_back(hbuf.getHash()); } } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 2fe3bb99e45..ca0d6a632a0 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -224,6 +224,89 @@ def test_incremental_backup_after_renaming_table(): assert instance.query("SELECT count(), sum(x) FROM test.table2") == "100\t4950\n" +def test_incremental_backup_for_log_family(): + backup_name = new_backup_name() + create_and_fill_table(engine="Log") + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + instance.query("INSERT INTO test.table VALUES (65, 'a'), (66, 'b')") + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "102\t5081\n" + + backup_name2 = new_backup_name() + instance.query(f"BACKUP TABLE test.table TO {backup_name2}") + + backup_name_inc = new_backup_name() + instance.query( + f"BACKUP TABLE test.table TO {backup_name_inc} SETTINGS base_backup = {backup_name}" + ) + + metadata_path = os.path.join( + get_path_to_backup(backup_name), "metadata/test/table.sql" + ) + + metadata_path2 = os.path.join( + get_path_to_backup(backup_name2), "metadata/test/table.sql" + ) + + metadata_path_inc = os.path.join( + get_path_to_backup(backup_name_inc), "metadata/test/table.sql" + ) + + assert os.path.isfile(metadata_path) + assert os.path.isfile(metadata_path2) + assert not os.path.isfile(metadata_path_inc) + assert os.path.getsize(metadata_path) > 0 + assert os.path.getsize(metadata_path) == os.path.getsize(metadata_path2) + + x_bin_path = os.path.join(get_path_to_backup(backup_name), "data/test/table/x.bin") + y_bin_path = os.path.join(get_path_to_backup(backup_name), "data/test/table/y.bin") + + x_bin_path2 = os.path.join( + get_path_to_backup(backup_name2), "data/test/table/x.bin" + ) + y_bin_path2 = os.path.join( + get_path_to_backup(backup_name2), "data/test/table/y.bin" + ) + + x_bin_path_inc = os.path.join( + get_path_to_backup(backup_name_inc), "data/test/table/x.bin" + ) + + y_bin_path_inc = os.path.join( + get_path_to_backup(backup_name_inc), "data/test/table/y.bin" + ) + + assert os.path.isfile(x_bin_path) + assert os.path.isfile(y_bin_path) + assert os.path.isfile(x_bin_path2) + assert os.path.isfile(y_bin_path2) + assert os.path.isfile(x_bin_path_inc) + assert os.path.isfile(y_bin_path_inc) + + x_bin_size = os.path.getsize(x_bin_path) + y_bin_size = os.path.getsize(y_bin_path) + x_bin_size2 = os.path.getsize(x_bin_path2) + y_bin_size2 = os.path.getsize(y_bin_path2) + x_bin_size_inc = os.path.getsize(x_bin_path_inc) + y_bin_size_inc = os.path.getsize(y_bin_path_inc) + + assert x_bin_size > 0 + assert y_bin_size > 0 + assert x_bin_size2 > 0 + assert y_bin_size2 > 0 + assert x_bin_size_inc > 0 + assert y_bin_size_inc > 0 + assert x_bin_size2 == x_bin_size + x_bin_size_inc + assert y_bin_size2 == y_bin_size + y_bin_size_inc + + instance.query(f"RESTORE TABLE test.table AS test.table2 FROM {backup_name_inc}") + + assert instance.query("SELECT count(), sum(x) FROM test.table2") == "102\t5081\n" + + def test_backup_not_found_or_already_exists(): backup_name = new_backup_name() From 2d976753e1e7d62ee3d35ca3453de31aef2d21aa Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Aug 2022 13:50:59 +0200 Subject: [PATCH 229/582] Fix incorrect directory remove in zero copy replication --- src/Storages/MergeTree/DataPartsExchange.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 02821de8629..9f8313a4700 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -773,6 +773,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( ThrottlerPtr throttler) { assert(!tmp_prefix.empty()); + const auto data_settings = data.getSettings(); /// We will remove directory if it's already exists. Make precautions. if (tmp_prefix.empty() //-V560 @@ -800,7 +801,14 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( { LOG_WARNING(log, "Directory {} already exists, probably result of a failed fetch. Will remove it before fetching part.", data_part_storage_builder->getFullPath()); - data_part_storage_builder->removeRecursive(); + + /// Even if it's a temporary part it could be downloaded with zero copy replication and this function + /// is executed as a callback. + /// + /// We don't control the amount of refs for temporary parts so we cannot decide can we remove blobs + /// or not. So we are not doing it + bool keep_shared = disk->supportZeroCopyReplication() && data_settings->allow_remote_fs_zero_copy_replication; + data_part_storage_builder->removeSharedRecursive(keep_shared); } data_part_storage_builder->createDirectories(); From 187b10dec5acfc6083347c3203e7839a3d87ac83 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 Aug 2022 13:05:40 +0200 Subject: [PATCH 230/582] Assign to all at once, improve logging --- tests/ci/cherry_pick.py | 29 ++++++++++++++--------------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index a0a77a2684e..b46dca93026 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -206,14 +206,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise ) self.cherrypick_pr.add_to_labels(Labels.LABEL_CHERRYPICK) self.cherrypick_pr.add_to_labels(Labels.LABEL_DO_NOT_TEST) - if self.pr.assignees: - logging.info( - "Assing to assignees of the original PR: %s", - ", ".join(user.login for user in self.pr.assignees), - ) - self.cherrypick_pr.add_to_assignees(*self.pr.assignees) - logging.info("Assign to the author of the original PR: %s", self.pr.user.login) - self.cherrypick_pr.add_to_assignees(self.pr.user) + self._assign_new_pr(self.cherrypick_pr) def create_backport(self): # Checkout the backport branch from the remote and make all changes to @@ -244,14 +237,20 @@ Merge it only if you intend to backport changes to the target branch, otherwise head=self.backport_branch, ) self.backport_pr.add_to_labels(Labels.LABEL_BACKPORT) + self._assign_new_pr(self.backport_pr) + + def _assign_new_pr(self, new_pr: PullRequest): + # It looks there some race when multiple .add_to_assignees are executed, + # so we'll add all at once + assignees = [self.pr.user] if self.pr.assignees: - logging.info( - "Assing to assignees of the original PR: %s", - ", ".join(user.login for user in self.pr.assignees), - ) - self.cherrypick_pr.add_to_assignees(*self.pr.assignees) - logging.info("Assign to the author of the original PR: %s", self.pr.user.login) - self.backport_pr.add_to_assignees(self.pr.user) + assignees.extend(self.pr.assignees) + logging.info( + "Assing #%s to author and assignees of the original PR: %s", + new_pr.number, + ", ".join(user.login for user in assignees), + ) + new_pr.add_to_assignees(*assignees) @property def backported(self) -> bool: From 48164b039a4bfc0637efc2290a72cb9bd33246ec Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Aug 2022 12:45:05 +0000 Subject: [PATCH 231/582] Consistent sanitizer naming in CI Sanitizer build/test jobs were sometimes named with full form ('undefined') - which could be confusing - and sometimes named with abbreviated form ('ubsan'). Now always using the short form. --- .github/workflows/backport_branches.yml | 4 +-- .github/workflows/master.yml | 38 +++++++++++------------ .github/workflows/pull_request.yml | 40 ++++++++++++------------- .github/workflows/release_branches.yml | 38 +++++++++++------------ tests/ci/ci_config.py | 26 ++++++++-------- 5 files changed, 73 insertions(+), 73 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index e1b2b1fad01..a1086452184 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -437,7 +437,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 EOF @@ -521,7 +521,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread) + CHECK_NAME=Stress test (tsan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 2acc1468328..f0c616424d7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1287,7 +1287,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1326,7 +1326,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1365,7 +1365,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1404,7 +1404,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1443,7 +1443,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1519,7 +1519,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1558,7 +1558,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1597,7 +1597,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1830,7 +1830,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address) + CHECK_NAME=Stateful tests (asan) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1867,7 +1867,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread) + CHECK_NAME=Stateful tests (tsan) REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1904,7 +1904,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory) + CHECK_NAME=Stateful tests (msan) REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2018,7 +2018,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address) + CHECK_NAME=Stress test (asan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2058,7 +2058,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread) + CHECK_NAME=Stress test (tsan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2094,7 +2094,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory) + CHECK_NAME=Stress test (msan) REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports @@ -2130,7 +2130,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_undefined REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined) + CHECK_NAME=Stress test (ubsan) REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports @@ -2319,7 +2319,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=4 @@ -2357,7 +2357,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=4 @@ -2395,7 +2395,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=4 @@ -2433,7 +2433,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=4 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 513df8487c4..c43369780a6 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1300,7 +1300,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1339,7 +1339,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1378,7 +1378,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1417,7 +1417,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1456,7 +1456,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1532,7 +1532,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1571,7 +1571,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1610,7 +1610,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1766,7 +1766,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_flaky_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests flaky check (address) + CHECK_NAME=Stateless tests flaky check (asan) REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1927,7 +1927,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address) + CHECK_NAME=Stateful tests (asan) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1964,7 +1964,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread) + CHECK_NAME=Stateful tests (tsan) REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2001,7 +2001,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory) + CHECK_NAME=Stateful tests (msan) REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2115,7 +2115,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address) + CHECK_NAME=Stress test (asan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2155,7 +2155,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread) + CHECK_NAME=Stress test (tsan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2191,7 +2191,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory) + CHECK_NAME=Stress test (msan) REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports @@ -2227,7 +2227,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_undefined REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined) + CHECK_NAME=Stress test (ubsan) REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports @@ -2599,7 +2599,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=4 @@ -2637,7 +2637,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=4 @@ -2675,7 +2675,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=4 @@ -2713,7 +2713,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=4 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index afeebf3c2b7..6403d00157f 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -591,7 +591,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -630,7 +630,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address) + CHECK_NAME=Stateless tests (asan) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -669,7 +669,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -708,7 +708,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -747,7 +747,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread) + CHECK_NAME=Stateless tests (tsan) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -823,7 +823,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -862,7 +862,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -901,7 +901,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory) + CHECK_NAME=Stateless tests (msan) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1134,7 +1134,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address) + CHECK_NAME=Stateful tests (asan) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1171,7 +1171,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread) + CHECK_NAME=Stateful tests (tsan) REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1208,7 +1208,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory) + CHECK_NAME=Stateful tests (msan) REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1322,7 +1322,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address) + CHECK_NAME=Stress test (asan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -1362,7 +1362,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread) + CHECK_NAME=Stress test (tsan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -1398,7 +1398,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory) + CHECK_NAME=Stress test (msan) REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports @@ -1434,7 +1434,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_undefined REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined) + CHECK_NAME=Stress test (ubsan) REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports @@ -1623,7 +1623,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=4 @@ -1661,7 +1661,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=4 @@ -1699,7 +1699,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=4 @@ -1737,7 +1737,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread) + CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=4 diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 61dfb07f762..eb934b0a3dd 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -187,13 +187,13 @@ CI_CONFIG = { "tests_config": { # required_build - build name for artifacts # force_tests - force success status for tests - "Stateful tests (address)": { + "Stateful tests (asan)": { "required_build": "package_asan", }, - "Stateful tests (thread)": { + "Stateful tests (tsan)": { "required_build": "package_tsan", }, - "Stateful tests (memory)": { + "Stateful tests (msan)": { "required_build": "package_msan", }, "Stateful tests (ubsan)": { @@ -214,13 +214,13 @@ CI_CONFIG = { "Stateful tests (release, DatabaseReplicated)": { "required_build": "package_release", }, - "Stateless tests (address)": { + "Stateless tests (asan)": { "required_build": "package_asan", }, - "Stateless tests (thread)": { + "Stateless tests (tsan)": { "required_build": "package_tsan", }, - "Stateless tests (memory)": { + "Stateless tests (msan)": { "required_build": "package_msan", }, "Stateless tests (ubsan)": { @@ -247,16 +247,16 @@ CI_CONFIG = { "Stateless tests (release, s3 storage)": { "required_build": "package_release", }, - "Stress test (address)": { + "Stress test (asan)": { "required_build": "package_asan", }, - "Stress test (thread)": { + "Stress test (tsan)": { "required_build": "package_tsan", }, - "Stress test (undefined)": { + "Stress test (ubsan)": { "required_build": "package_ubsan", }, - "Stress test (memory)": { + "Stress test (msan)": { "required_build": "package_msan", }, "Stress test (debug)": { @@ -265,13 +265,13 @@ CI_CONFIG = { "Integration tests (asan)": { "required_build": "package_asan", }, - "Integration tests (thread)": { + "Integration tests (tsan)": { "required_build": "package_tsan", }, "Integration tests (release)": { "required_build": "package_release", }, - "Integration tests (memory)": { + "Integration tests (msan)": { "required_build": "package_msan", }, "Integration tests flaky check (asan)": { @@ -313,7 +313,7 @@ CI_CONFIG = { "AST fuzzer (UBSan)": { "required_build": "package_ubsan", }, - "Stateless tests flaky check (address)": { + "Stateless tests flaky check (asan)": { "required_build": "package_asan", }, "ClickHouse Keeper Jepsen": { From a4f71ba6b8d0074c30af7b5563e19d68e4c86618 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Aug 2022 14:53:33 +0200 Subject: [PATCH 232/582] Fix one more bug --- src/Storages/MergeTree/MergeTreeData.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee2cbf4fe8a..6569c0ea78d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1592,7 +1592,17 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif else { LOG_WARNING(log, "Removing temporary directory {}", full_path); - disk->removeRecursive(it->path()); + /// Even if it's a temporary part it could be downloaded with zero copy replication and this function + /// is executed as a callback. + /// + /// We don't control the amount of refs for temporary parts so we cannot decide can we remove blobs + /// or not. So we are not doing it + + bool keep_shared = disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication; + if (keep_shared) + LOG_WARNING(log, "Since zero-copy replication is enabled we are not going to remove blobs from shared storage for {}", full_path); + + disk->removeSharedRecursive(it->path(), keep_shared, {}); ++cleared_count; } } From c427491bcb030002631f9a032f55194919c54938 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Aug 2022 15:05:45 +0200 Subject: [PATCH 233/582] Better --- src/Storages/MergeTree/MergeTreeData.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6569c0ea78d..f745d11438f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1592,15 +1592,19 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif else { LOG_WARNING(log, "Removing temporary directory {}", full_path); + /// Even if it's a temporary part it could be downloaded with zero copy replication and this function /// is executed as a callback. /// /// We don't control the amount of refs for temporary parts so we cannot decide can we remove blobs /// or not. So we are not doing it - - bool keep_shared = disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication; - if (keep_shared) - LOG_WARNING(log, "Since zero-copy replication is enabled we are not going to remove blobs from shared storage for {}", full_path); + bool keep_shared = false; + if (it->path().find("tmp-fetch") != std::string::npos) + { + keep_shared = disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication; + if (keep_shared) + LOG_WARNING(log, "Since zero-copy replication is enabled we are not going to remove blobs from shared storage for {}", full_path); + } disk->removeSharedRecursive(it->path(), keep_shared, {}); ++cleared_count; From 0e6b3b870acb12efbbd27eaef67c9ed1f267b704 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 13:08:27 +0000 Subject: [PATCH 234/582] Add support for delete in RocksDB --- src/Interpreters/InterpreterDeleteQuery.cpp | 34 ++++++++---- src/Interpreters/MutationsInterpreter.cpp | 14 +++-- src/Interpreters/MutationsInterpreter.h | 6 ++- .../RocksDB/StorageEmbeddedRocksDB.cpp | 53 +++++++++++++++++++ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 3 ++ .../02416_rocksdb_delete.reference | 9 ++++ .../0_stateless/02416_rocksdb_delete.sql | 21 ++++++++ 7 files changed, 126 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02416_rocksdb_delete.reference create mode 100644 tests/queries/0_stateless/02416_rocksdb_delete.sql diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 5e5bebfdc58..9e6e51583d4 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -34,11 +34,6 @@ InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterDeleteQuery::execute() { - if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete) - { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); - } - FunctionNameNormalizer().visit(query_ptr.get()); const ASTDeleteQuery & delete_query = query_ptr->as(); auto table_id = getContext()->resolveStorageID(delete_query, Context::ResolveOrdinary); @@ -49,10 +44,6 @@ BlockIO InterpreterDeleteQuery::execute() /// First check table storage for validations. StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); - auto merge_tree = std::dynamic_pointer_cast(table); - if (!merge_tree) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only MergeTree family tables are supported"); - checkStorageSupportsTransactionsIfNeeded(table, getContext()); if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); @@ -69,6 +60,31 @@ BlockIO InterpreterDeleteQuery::execute() auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto merge_tree = std::dynamic_pointer_cast(table); + if (!merge_tree) + { + /// Convert to MutationCommand + MutationCommands mutation_commands; + MutationCommand mut_command; + + mut_command.type = MutationCommand::Type::DELETE; + mut_command.predicate = delete_query.predicate; + + auto command = std::make_shared(); + command->type = ASTAlterCommand::DELETE; + command->predicate = delete_query.predicate; + + mutation_commands.emplace_back(mut_command); + + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); + table->mutate(mutation_commands, getContext()); + return {}; + } + + if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); + /// Convert to MutationCommand MutationCommands mutation_commands; MutationCommand mut_command; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index d3679aac891..90ef10805bf 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -226,7 +226,7 @@ bool isStorageTouchedByMutations( ASTPtr select_query = prepareQueryAffectedAST(commands, storage, context_copy); /// Interpreter must be alive, when we use result of execute() method. - /// For some reason it may copy context and and give it into ExpressionTransform + /// For some reason it may copy context and give it into ExpressionTransform /// after that we will use context from destroyed stack frame in our stream. InterpreterSelectQuery interpreter( select_query, context_copy, storage, metadata_snapshot, SelectQueryOptions().ignoreLimits().ignoreProjections()); @@ -288,13 +288,15 @@ MutationsInterpreter::MutationsInterpreter( const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_) + bool can_execute_, + bool return_deleted_rows_) : storage(std::move(storage_)) , metadata_snapshot(metadata_snapshot_) , commands(std::move(commands_)) , context(Context::createCopy(context_)) , can_execute(can_execute_) , select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits().ignoreProjections()) + , return_deleted_rows(return_deleted_rows_) { mutation_ast = prepare(!can_execute); } @@ -478,8 +480,12 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); - auto negated_predicate = makeASTFunction("isZeroOrNull", getPartitionAndPredicateExpressionForMutationCommand(command)); - stages.back().filters.push_back(negated_predicate); + auto predicate = getPartitionAndPredicateExpressionForMutationCommand(command); + + if (!return_deleted_rows) + predicate = makeASTFunction("isZeroOrNull", predicate); + + stages.back().filters.push_back(predicate); } else if (command.type == MutationCommand::UPDATE) { diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 94525bf6b8c..1f63cc9fdeb 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -43,7 +43,8 @@ public: const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_); + bool can_execute_, + bool return_deleted_rows_ = false); void validate(); @@ -156,6 +157,9 @@ private: /// Columns, that we need to read for calculation of skip indices, projections or TTL expressions. ColumnDependencies dependencies; + + // wether we should return deleted or nondeleted rows on DELETE mutation + bool return_deleted_rows; }; } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index c9ddd9147b9..c274ec47dae 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -10,11 +11,15 @@ #include #include +#include #include #include #include #include +#include + +#include #include #include @@ -192,6 +197,54 @@ void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & initDB(); } +void StorageEmbeddedRocksDB::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const +{ + for (const auto & command : commands) + { + if (command.type != MutationCommand::Type::DELETE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only DELETE mutation supported for EmbeddedRocksDB"); + } +} + +void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPtr context_) +{ + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto storage = getStorageID(); + auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context_); + + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context_, true, /*return_deleted_row*/ true); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); + PullingPipelineExecutor executor(pipeline); + + Block block; + while (executor.pull(block)) + { + auto column_it = std::find_if(block.begin(), block.end(), [&](const auto & column) { return column.name == primary_key; }); + assert(column_it != block.end()); + + auto column = column_it->column; + auto size = column->size(); + + rocksdb::WriteBatch batch; + WriteBufferFromOwnString wb_key; + for (size_t i = 0; i < size; ++i) + { + wb_key.restart(); + + column_it->type->getDefaultSerialization()->serializeBinary(*column, i, wb_key); + auto status = batch.Delete(wb_key.str()); + if (!status.ok()) + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); + + } + + auto status = rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + if (!status.ok()) + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); + } + +} + void StorageEmbeddedRocksDB::initDB() { rocksdb::Status status; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index ab87eac3e66..fe3ed1c6138 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -49,6 +49,9 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; + void mutate(const MutationCommands &, ContextPtr) override; + bool supportsParallelInsert() const override { return true; } bool supportsIndexForIn() const override { return true; } bool mayBenefitFromIndexForIn( diff --git a/tests/queries/0_stateless/02416_rocksdb_delete.reference b/tests/queries/0_stateless/02416_rocksdb_delete.reference new file mode 100644 index 00000000000..0739e9f5210 --- /dev/null +++ b/tests/queries/0_stateless/02416_rocksdb_delete.reference @@ -0,0 +1,9 @@ +1 Some string +2 Some other string +3 random +4 random2 +----------- +3 random +4 random2 +----------- +3 random diff --git a/tests/queries/0_stateless/02416_rocksdb_delete.sql b/tests/queries/0_stateless/02416_rocksdb_delete.sql new file mode 100644 index 00000000000..1f26ce477d4 --- /dev/null +++ b/tests/queries/0_stateless/02416_rocksdb_delete.sql @@ -0,0 +1,21 @@ +-- Tags: no-ordinary-database, no-fasttest + +DROP TABLE IF EXISTS 02416_rocksdb_delete; + +CREATE TABLE 02416_rocksdb_delete (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); + +INSERT INTO 02416_rocksdb_delete VALUES (1, 'Some string'), (2, 'Some other string'), (3, 'random'), (4, 'random2'); + +SELECT * FROM 02416_rocksdb_delete ORDER BY key; +SELECT '-----------'; + +DELETE FROM 02416_rocksdb_delete WHERE value LIKE 'Some%string'; + +SELECT * FROM 02416_rocksdb_delete ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02416_rocksdb_delete DELETE WHERE key >= 4; + +SELECT * FROM 02416_rocksdb_delete ORDER BY key; + +DROP TABLE IF EXISTS 02416_rocksdb_delete; From 3a183236a2cdf6d96c16801ebc141fb17b163171 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Aug 2022 15:39:53 +0200 Subject: [PATCH 235/582] Fix --- src/Storages/MergeTree/MergeTreeMarksLoader.cpp | 8 ++++++++ src/Storages/MergeTree/MergeTreeMarksLoader.h | 2 ++ 2 files changed, 10 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 8c6fc293e6b..7d8ee994ac9 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -55,6 +55,14 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( } } +MergeTreeMarksLoader::~MergeTreeMarksLoader() +{ + if (future.valid()) + { + future.wait(); + } +} + const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index) { if (!marks) diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 69425900658..60ccc953e9b 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -26,6 +26,8 @@ public: ThreadPool * load_marks_threadpool_, size_t columns_in_mark_ = 1); + ~MergeTreeMarksLoader(); + const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0); private: From 55c20ce45b09b395719e142c3ca30dc3af4b5a9d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 31 Aug 2022 21:43:08 +0800 Subject: [PATCH 236/582] fix tryGetPreparedSet crash when same set expression built from different columns --- src/Storages/MergeTree/KeyCondition.cpp | 5 +++++ .../0_stateless/02416_in_set_same_ast_diff_columns.reference | 1 + .../0_stateless/02416_in_set_same_ast_diff_columns.sql | 3 +++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.reference create mode 100644 tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b42fe49a1d0..609f3ddac59 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -297,8 +297,13 @@ public: assert(indexes_mapping.size() == data_types.size()); for (size_t i = 0; i < indexes_mapping.size(); ++i) + { + /// Same set expression can be construct from different column(s) with different types + if (indexes_mapping[i].tuple_index >= candidate_set->getDataTypes().size()) + return false; if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) return false; + } return true; }; diff --git a/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.reference b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql new file mode 100644 index 00000000000..36ab424f34f --- /dev/null +++ b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql @@ -0,0 +1,3 @@ +CREATE TABLE set_crash (key1 Int32, id1 Int64, c1 Int64) ENGINE = MergeTree PARTITION BY id1 ORDER BY key1; +INSERT INTO set_crash VALUES (-1, 1, 0); +SELECT 1 in (-1,1) FROM set_crash WHERE (key1, id1) in (-1, 1); From 5f1827a3168b5d4070f74d3252f213d7afbdd401 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 31 Aug 2022 21:52:57 +0800 Subject: [PATCH 237/582] fix style test --- .../queries/0_stateless/02416_in_set_same_ast_diff_columns.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql index 36ab424f34f..3f7b27201b0 100644 --- a/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql +++ b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql @@ -1,3 +1,3 @@ -CREATE TABLE set_crash (key1 Int32, id1 Int64, c1 Int64) ENGINE = MergeTree PARTITION BY id1 ORDER BY key1; +CREATE TABLE set_crash (key1 Int32, id1 Int64, c1 Int64) ENGINE = MergeTree PARTITION BY id1 ORDER BY key1; INSERT INTO set_crash VALUES (-1, 1, 0); SELECT 1 in (-1,1) FROM set_crash WHERE (key1, id1) in (-1, 1); From 55e675a407aec7c82f97a10a703a9e306debf35d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 31 Aug 2022 21:57:07 +0800 Subject: [PATCH 238/582] fix test again --- .../queries/0_stateless/02416_in_set_same_ast_diff_columns.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql index 3f7b27201b0..c3475f37e7e 100644 --- a/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql +++ b/tests/queries/0_stateless/02416_in_set_same_ast_diff_columns.sql @@ -1,3 +1,3 @@ CREATE TABLE set_crash (key1 Int32, id1 Int64, c1 Int64) ENGINE = MergeTree PARTITION BY id1 ORDER BY key1; INSERT INTO set_crash VALUES (-1, 1, 0); -SELECT 1 in (-1,1) FROM set_crash WHERE (key1, id1) in (-1, 1); +SELECT 1 in (-1, 1) FROM set_crash WHERE (key1, id1) in (-1, 1); From acf87c1d10fb39291af387a2dd0bd22b5467b613 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 Aug 2022 14:10:29 +0000 Subject: [PATCH 239/582] Fix nested JSON Objects schema inference --- src/Formats/JSONUtils.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 895833abf4d..9aa0645f580 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -218,7 +218,6 @@ namespace JSONUtils { auto object = field.getObject(); DataTypes value_types; - bool have_object_value = false; for (const auto key_value_pair : object) { auto type = getDataTypeFromFieldImpl(key_value_pair.second, settings, numbers_parsed_from_json_strings); @@ -226,10 +225,7 @@ namespace JSONUtils continue; if (isObject(type)) - { - have_object_value = true; - break; - } + return std::make_shared("json", true); value_types.push_back(type); } @@ -242,7 +238,7 @@ namespace JSONUtils for (size_t i = 1; i < value_types.size(); ++i) are_types_equal &= value_types[i]->equals(*value_types[0]); - if (have_object_value || !are_types_equal) + if (!are_types_equal) return std::make_shared("json", true); return std::make_shared(std::make_shared(), value_types[0]); From 53ebe8a0bde8d989c25470974e2fbdea3318176f Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 Aug 2022 14:11:00 +0000 Subject: [PATCH 240/582] Add test --- tests/queries/0_stateless/02416_json_object_inference.reference | 1 + tests/queries/0_stateless/02416_json_object_inference.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02416_json_object_inference.reference create mode 100644 tests/queries/0_stateless/02416_json_object_inference.sql diff --git a/tests/queries/0_stateless/02416_json_object_inference.reference b/tests/queries/0_stateless/02416_json_object_inference.reference new file mode 100644 index 00000000000..01537523906 --- /dev/null +++ b/tests/queries/0_stateless/02416_json_object_inference.reference @@ -0,0 +1 @@ +a Object(Nullable(\'json\')) diff --git a/tests/queries/0_stateless/02416_json_object_inference.sql b/tests/queries/0_stateless/02416_json_object_inference.sql new file mode 100644 index 00000000000..b861468a08a --- /dev/null +++ b/tests/queries/0_stateless/02416_json_object_inference.sql @@ -0,0 +1,2 @@ +-- Tags: no-fasttest +desc format(JSONEachRow, '{"a" : {"b" : {"c" : 1, "d" : "str"}}}'); From f043790e6dac08681fbe05c57c622f64599e3111 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 14:33:53 +0000 Subject: [PATCH 241/582] Add support for update --- src/Interpreters/InterpreterDeleteQuery.cpp | 4 - src/Interpreters/MutationsInterpreter.cpp | 7 +- src/Interpreters/MutationsInterpreter.h | 6 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 96 +++++++++++++------ .../02416_rocksdb_delete.reference | 9 -- .../0_stateless/02416_rocksdb_delete.sql | 21 ---- .../02416_rocksdb_delete_update.reference | 32 +++++++ .../02416_rocksdb_delete_update.sql | 42 ++++++++ 8 files changed, 152 insertions(+), 65 deletions(-) delete mode 100644 tests/queries/0_stateless/02416_rocksdb_delete.reference delete mode 100644 tests/queries/0_stateless/02416_rocksdb_delete.sql create mode 100644 tests/queries/0_stateless/02416_rocksdb_delete_update.reference create mode 100644 tests/queries/0_stateless/02416_rocksdb_delete_update.sql diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 9e6e51583d4..dcb8a2cb165 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -70,10 +70,6 @@ BlockIO InterpreterDeleteQuery::execute() mut_command.type = MutationCommand::Type::DELETE; mut_command.predicate = delete_query.predicate; - auto command = std::make_shared(); - command->type = ASTAlterCommand::DELETE; - command->predicate = delete_query.predicate; - mutation_commands.emplace_back(mut_command); table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 90ef10805bf..c496995ba65 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -289,6 +289,7 @@ MutationsInterpreter::MutationsInterpreter( MutationCommands commands_, ContextPtr context_, bool can_execute_, + bool return_all_columns_, bool return_deleted_rows_) : storage(std::move(storage_)) , metadata_snapshot(metadata_snapshot_) @@ -296,6 +297,7 @@ MutationsInterpreter::MutationsInterpreter( , context(Context::createCopy(context_)) , can_execute(can_execute_) , select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits().ignoreProjections()) + , return_all_columns(return_all_columns_) , return_deleted_rows(return_deleted_rows_) { mutation_ast = prepare(!can_execute); @@ -474,6 +476,9 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// First, break a sequence of commands into stages. for (auto & command : commands) { + // we can return deleted rows only if it's the only present command + assert(command.type == MutationCommand::DELETE || !return_deleted_rows); + if (command.type == MutationCommand::DELETE) { mutation_kind.set(MutationKind::MUTATE_OTHER); @@ -795,7 +800,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < prepared_stages.size(); ++i) { - if (!prepared_stages[i].filters.empty()) + if (return_all_columns || !prepared_stages[i].filters.empty()) { for (const auto & column : all_columns) prepared_stages[i].output_columns.insert(column.name); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 1f63cc9fdeb..336c5f11162 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -44,6 +44,7 @@ public: MutationCommands commands_, ContextPtr context_, bool can_execute_, + bool return_all_columns_ = false, bool return_deleted_rows_ = false); void validate(); @@ -158,7 +159,10 @@ private: /// Columns, that we need to read for calculation of skip indices, projections or TTL expressions. ColumnDependencies dependencies; - // wether we should return deleted or nondeleted rows on DELETE mutation + // whether all columns should be returned, not just updated + bool return_all_columns; + + // whether we should return deleted or nondeleted rows on DELETE mutation bool return_deleted_rows; }; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index c274ec47dae..68ffb87783e 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -199,50 +199,88 @@ void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & void StorageEmbeddedRocksDB::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for (const auto & command : commands) - { - if (command.type != MutationCommand::Type::DELETE) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only DELETE mutation supported for EmbeddedRocksDB"); - } + if (commands.empty()) + return; + + if (commands.size() > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutations cannot be combined for EmbeddedRocksDB"); + + const auto command_type = commands.front().type; + if (command_type != MutationCommand::Type::UPDATE && command_type != MutationCommand::Type::DELETE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only DELETE and UPDATE mutation supported for EmbeddedRocksDB"); } void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPtr context_) { + if (commands.empty()) + return; + + assert(commands.size() == 1); + auto metadata_snapshot = getInMemoryMetadataPtr(); auto storage = getStorageID(); auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context_); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context_, true, /*return_deleted_row*/ true); + if (commands.front().type == MutationCommand::Type::DELETE) + { + auto interpreter = std::make_unique( + storage_ptr, + metadata_snapshot, + commands, + context_, + /*can_execute_*/ true, + /*return_all_columns_*/ true, + /*return_deleted_rows_*/ true); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); + PullingPipelineExecutor executor(pipeline); + + auto sink = std::make_shared(*this, metadata_snapshot); + + Block block; + while (executor.pull(block)) + { + auto column_it = std::find_if(block.begin(), block.end(), [&](const auto & column) { return column.name == primary_key; }); + assert(column_it != block.end()); + + auto column = column_it->column; + auto size = column->size(); + + rocksdb::WriteBatch batch; + WriteBufferFromOwnString wb_key; + for (size_t i = 0; i < size; ++i) + { + wb_key.restart(); + + column_it->type->getDefaultSerialization()->serializeBinary(*column, i, wb_key); + auto status = batch.Delete(wb_key.str()); + if (!status.ok()) + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); + } + + auto status = rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + if (!status.ok()) + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); + } + + return; + } + + assert(commands.front().type == MutationCommand::Type::UPDATE); + if (commands.front().column_to_update_expression.contains(primary_key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated"); + + auto interpreter = std::make_unique( + storage_ptr, metadata_snapshot, commands, context_, /*can_execute_*/ true, /*return_all_columns*/ true); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); + auto sink = std::make_shared(*this, metadata_snapshot); + Block block; while (executor.pull(block)) { - auto column_it = std::find_if(block.begin(), block.end(), [&](const auto & column) { return column.name == primary_key; }); - assert(column_it != block.end()); - - auto column = column_it->column; - auto size = column->size(); - - rocksdb::WriteBatch batch; - WriteBufferFromOwnString wb_key; - for (size_t i = 0; i < size; ++i) - { - wb_key.restart(); - - column_it->type->getDefaultSerialization()->serializeBinary(*column, i, wb_key); - auto status = batch.Delete(wb_key.str()); - if (!status.ok()) - throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); - - } - - auto status = rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); - if (!status.ok()) - throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); + sink->consume(Chunk{block.getColumns(), block.rows()}); } - } void StorageEmbeddedRocksDB::initDB() diff --git a/tests/queries/0_stateless/02416_rocksdb_delete.reference b/tests/queries/0_stateless/02416_rocksdb_delete.reference deleted file mode 100644 index 0739e9f5210..00000000000 --- a/tests/queries/0_stateless/02416_rocksdb_delete.reference +++ /dev/null @@ -1,9 +0,0 @@ -1 Some string -2 Some other string -3 random -4 random2 ------------ -3 random -4 random2 ------------ -3 random diff --git a/tests/queries/0_stateless/02416_rocksdb_delete.sql b/tests/queries/0_stateless/02416_rocksdb_delete.sql deleted file mode 100644 index 1f26ce477d4..00000000000 --- a/tests/queries/0_stateless/02416_rocksdb_delete.sql +++ /dev/null @@ -1,21 +0,0 @@ --- Tags: no-ordinary-database, no-fasttest - -DROP TABLE IF EXISTS 02416_rocksdb_delete; - -CREATE TABLE 02416_rocksdb_delete (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); - -INSERT INTO 02416_rocksdb_delete VALUES (1, 'Some string'), (2, 'Some other string'), (3, 'random'), (4, 'random2'); - -SELECT * FROM 02416_rocksdb_delete ORDER BY key; -SELECT '-----------'; - -DELETE FROM 02416_rocksdb_delete WHERE value LIKE 'Some%string'; - -SELECT * FROM 02416_rocksdb_delete ORDER BY key; -SELECT '-----------'; - -ALTER TABLE 02416_rocksdb_delete DELETE WHERE key >= 4; - -SELECT * FROM 02416_rocksdb_delete ORDER BY key; - -DROP TABLE IF EXISTS 02416_rocksdb_delete; diff --git a/tests/queries/0_stateless/02416_rocksdb_delete_update.reference b/tests/queries/0_stateless/02416_rocksdb_delete_update.reference new file mode 100644 index 00000000000..8ca8c0ca5a2 --- /dev/null +++ b/tests/queries/0_stateless/02416_rocksdb_delete_update.reference @@ -0,0 +1,32 @@ +1 Some string 0 +2 Some other string 0 +3 random 0 +4 random2 0 +----------- +3 random 0 +4 random2 0 +----------- +3 random 0 +----------- +0 +----------- +1 String 10 +2 String 20 +3 String 30 +4 String 40 +----------- +1 String 10 +2 String 20 +3 Another 30 +4 Another 40 +----------- +1 String 10 +2 String 20 +3 Another 30 +4 Another 40 +----------- +1 String 102 +2 String 202 +3 Another 302 +4 Another 402 +----------- diff --git a/tests/queries/0_stateless/02416_rocksdb_delete_update.sql b/tests/queries/0_stateless/02416_rocksdb_delete_update.sql new file mode 100644 index 00000000000..28953a108d7 --- /dev/null +++ b/tests/queries/0_stateless/02416_rocksdb_delete_update.sql @@ -0,0 +1,42 @@ +-- Tags: no-ordinary-database, no-fasttest + +DROP TABLE IF EXISTS 02416_rocksdb; + +CREATE TABLE 02416_rocksdb (key UInt64, value String, value2 UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key); + +INSERT INTO 02416_rocksdb VALUES (1, 'Some string', 0), (2, 'Some other string', 0), (3, 'random', 0), (4, 'random2', 0); + +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +DELETE FROM 02416_rocksdb WHERE value LIKE 'Some%string'; + +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02416_rocksdb DELETE WHERE key >= 4; + +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +DELETE FROM 02416_rocksdb WHERE 1 = 1; +SELECT count() FROM 02416_rocksdb; +SELECT '-----------'; + +INSERT INTO 02416_rocksdb VALUES (1, 'String', 10), (2, 'String', 20), (3, 'String', 30), (4, 'String', 40); +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02416_rocksdb UPDATE value = 'Another' WHERE key > 2; +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02416_rocksdb UPDATE key = key * 10 WHERE 1 = 1; -- { serverError 36 } +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02416_rocksdb UPDATE value2 = value2 * 10 + 2 WHERE 1 = 1; +SELECT * FROM 02416_rocksdb ORDER BY key; +SELECT '-----------'; + +DROP TABLE IF EXISTS 02416_rocksdb; From 0f747e7987a6eb4ae986206b75dd595e89e0d113 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Aug 2022 19:23:46 +0200 Subject: [PATCH 242/582] Move cache files --- src/CMakeLists.txt | 1 + src/Common/tests/gtest_lru_file_cache.cpp | 2 +- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 4 ++-- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 4 ++-- src/Disks/IO/CachedOnDiskWriteBufferFromFile.h | 2 +- src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp | 4 ++-- src/Disks/ObjectStorages/Cached/CachedObjectStorage.h | 4 ++-- src/Disks/ObjectStorages/Cached/registerDiskCache.cpp | 6 +++--- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 1 - src/Disks/ObjectStorages/DiskObjectStorage.h | 1 - src/Disks/ObjectStorages/IObjectStorage.h | 1 - src/Disks/ObjectStorages/LocalObjectStorage.cpp | 2 -- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 -- src/Disks/ObjectStorages/S3/parseConfig.h | 2 -- src/IO/ReadSettings.h | 2 +- src/IO/WriteBufferFromS3.cpp | 2 +- src/Interpreters/AsynchronousMetrics.cpp | 4 ++-- src/{Common => Interpreters/Cache}/FileCache.cpp | 4 ++-- src/{Common => Interpreters/Cache}/FileCache.h | 8 ++++---- src/{Common => Interpreters/Cache}/FileCacheFactory.cpp | 0 src/{Common => Interpreters/Cache}/FileCacheFactory.h | 4 ++-- .../FileCacheType.h => Interpreters/Cache/FileCacheKey.h} | 0 src/{Common => Interpreters/Cache}/FileCacheSettings.cpp | 0 src/{Common => Interpreters/Cache}/FileCacheSettings.h | 2 +- src/{Common => Interpreters/Cache}/FileCache_fwd.h | 0 src/{Common => Interpreters/Cache}/FileSegment.cpp | 2 +- src/{Common => Interpreters/Cache}/FileSegment.h | 3 ++- src/{Common => Interpreters/Cache}/IFileCachePriority.h | 2 +- .../Cache}/LRUFileCachePriority.cpp | 2 +- src/{Common => Interpreters/Cache}/LRUFileCachePriority.h | 2 +- src/Interpreters/InterpreterDescribeCacheQuery.cpp | 4 ++-- src/Interpreters/InterpreterShowTablesQuery.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 ++-- src/Storages/System/StorageSystemFilesystemCache.cpp | 6 +++--- src/Storages/System/StorageSystemRemoteDataPaths.cpp | 4 ++-- 35 files changed, 43 insertions(+), 50 deletions(-) rename src/{Common => Interpreters/Cache}/FileCache.cpp (99%) rename src/{Common => Interpreters/Cache}/FileCache.h (98%) rename src/{Common => Interpreters/Cache}/FileCacheFactory.cpp (100%) rename src/{Common => Interpreters/Cache}/FileCacheFactory.h (92%) rename src/{Common/FileCacheType.h => Interpreters/Cache/FileCacheKey.h} (100%) rename src/{Common => Interpreters/Cache}/FileCacheSettings.cpp (100%) rename src/{Common => Interpreters/Cache}/FileCacheSettings.h (94%) rename src/{Common => Interpreters/Cache}/FileCache_fwd.h (100%) rename src/{Common => Interpreters/Cache}/FileSegment.cpp (99%) rename src/{Common => Interpreters/Cache}/FileSegment.h (99%) rename src/{Common => Interpreters/Cache}/IFileCachePriority.h (98%) rename src/{Common => Interpreters/Cache}/LRUFileCachePriority.cpp (98%) rename src/{Common => Interpreters/Cache}/LRUFileCachePriority.h (97%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3ece5fd410b..fd8771c1529 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -247,6 +247,7 @@ add_object_library(clickhouse_databases Databases) add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_interpreters Interpreters) +add_object_library(clickhouse_interpreters_cache Interpreters/Cache) add_object_library(clickhouse_interpreters_access Interpreters/Access) add_object_library(clickhouse_interpreters_mysql Interpreters/MySQL) add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProxy) diff --git a/src/Common/tests/gtest_lru_file_cache.cpp b/src/Common/tests/gtest_lru_file_cache.cpp index bb454cc41eb..6a5a54fc84d 100644 --- a/src/Common/tests/gtest_lru_file_cache.cpp +++ b/src/Common/tests/gtest_lru_file_cache.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 365496c6d06..ed623272c12 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -1,13 +1,13 @@ #pragma once -#include +#include #include #include #include #include #include #include -#include +#include namespace CurrentMetrics diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 21c120fd4c8..56a21307c36 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -1,7 +1,7 @@ #include "CachedOnDiskWriteBufferFromFile.h" -#include -#include +#include +#include #include #include #include diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index abc50078874..fa861fea14b 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace Poco diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index a68f9fdaf2d..7e829847846 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 258b70ed579..68ded61a9f1 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include namespace Poco { diff --git a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp index 84b95bd3d16..902ebd0fcc8 100644 --- a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp +++ b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index bac0d8e4c12..00ef01645cd 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 6e2d719e3ad..34056f17b3c 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 89d0e0d6d03..dc6683cfe95 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -17,7 +17,6 @@ #include #include #include -#include #include diff --git a/src/Disks/ObjectStorages/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/LocalObjectStorage.cpp index 64512a07919..dbb3a7c2aba 100644 --- a/src/Disks/ObjectStorages/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/LocalObjectStorage.cpp @@ -1,8 +1,6 @@ #include #include -#include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4cdb867e48c..b34eddf63f0 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -22,8 +22,6 @@ #include #include -#include - namespace DB { diff --git a/src/Disks/ObjectStorages/S3/parseConfig.h b/src/Disks/ObjectStorages/S3/parseConfig.h index 725c86ce462..97fafd620f9 100644 --- a/src/Disks/ObjectStorages/S3/parseConfig.h +++ b/src/Disks/ObjectStorages/S3/parseConfig.h @@ -6,7 +6,6 @@ #include #include -#include #include #include #include @@ -14,7 +13,6 @@ #include #include #include -#include namespace DB diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index e639ecbedc2..fb093c52f67 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 2f2479766d5..6207ba53bd8 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f9bc22dd110..d34a31ac93e 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -11,10 +11,10 @@ #include #include #include -#include +#include #include #include -#include +#include #include #include #include diff --git a/src/Common/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp similarity index 99% rename from src/Common/FileCache.cpp rename to src/Interpreters/Cache/FileCache.cpp index 44ecac2cc02..cb9ffb98976 100644 --- a/src/Common/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -2,7 +2,8 @@ #include #include -#include +#include +#include #include #include #include @@ -10,7 +11,6 @@ #include #include #include -#include namespace fs = std::filesystem; diff --git a/src/Common/FileCache.h b/src/Interpreters/Cache/FileCache.h similarity index 98% rename from src/Common/FileCache.h rename to src/Interpreters/Cache/FileCache.h index b5b1e917e76..f45a37631ff 100644 --- a/src/Common/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -13,11 +13,11 @@ #include #include -#include -#include -#include +#include +#include +#include #include -#include +#include namespace DB { diff --git a/src/Common/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp similarity index 100% rename from src/Common/FileCacheFactory.cpp rename to src/Interpreters/Cache/FileCacheFactory.cpp diff --git a/src/Common/FileCacheFactory.h b/src/Interpreters/Cache/FileCacheFactory.h similarity index 92% rename from src/Common/FileCacheFactory.h rename to src/Interpreters/Cache/FileCacheFactory.h index 0d99da0343c..82e0ec8f928 100644 --- a/src/Common/FileCacheFactory.h +++ b/src/Interpreters/Cache/FileCacheFactory.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include +#include #include #include diff --git a/src/Common/FileCacheType.h b/src/Interpreters/Cache/FileCacheKey.h similarity index 100% rename from src/Common/FileCacheType.h rename to src/Interpreters/Cache/FileCacheKey.h diff --git a/src/Common/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp similarity index 100% rename from src/Common/FileCacheSettings.cpp rename to src/Interpreters/Cache/FileCacheSettings.cpp diff --git a/src/Common/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h similarity index 94% rename from src/Common/FileCacheSettings.h rename to src/Interpreters/Cache/FileCacheSettings.h index 1e4f668e8bb..c6155edad85 100644 --- a/src/Common/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace Poco { namespace Util { class AbstractConfiguration; } } // NOLINT(cppcoreguidelines-virtual-class-destructor) diff --git a/src/Common/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h similarity index 100% rename from src/Common/FileCache_fwd.h rename to src/Interpreters/Cache/FileCache_fwd.h diff --git a/src/Common/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp similarity index 99% rename from src/Common/FileSegment.cpp rename to src/Interpreters/Cache/FileSegment.cpp index 1e1a309a8c7..547e6849dd6 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Common/FileSegment.h b/src/Interpreters/Cache/FileSegment.h similarity index 99% rename from src/Common/FileSegment.h rename to src/Interpreters/Cache/FileSegment.h index 8b826576e1f..f3fb367792a 100644 --- a/src/Common/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -5,7 +5,8 @@ #include #include #include -#include +#include + namespace Poco { class Logger; } diff --git a/src/Common/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h similarity index 98% rename from src/Common/IFileCachePriority.h rename to src/Interpreters/Cache/IFileCachePriority.h index f80266f9eea..de5e3f1428c 100644 --- a/src/Common/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Common/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp similarity index 98% rename from src/Common/LRUFileCachePriority.cpp rename to src/Interpreters/Cache/LRUFileCachePriority.cpp index c4d6313e4d6..17fbd2c2092 100644 --- a/src/Common/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace CurrentMetrics diff --git a/src/Common/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h similarity index 97% rename from src/Common/LRUFileCachePriority.h rename to src/Interpreters/Cache/LRUFileCachePriority.h index 2cdcc981e04..2345d3c47db 100644 --- a/src/Common/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include namespace DB diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index d5e4a81e98e..7ebb6ef64c6 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -5,8 +5,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index d623b6c71d6..35e1a27f24b 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 43bb15b70bb..106b1611f80 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -7,8 +7,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index ba94cb01b75..cd9324b3253 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -2,9 +2,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include #include #include diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 662255d6afd..fe7aaf97970 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -1,8 +1,8 @@ #include "StorageSystemRemoteDataPaths.h" #include #include -#include -#include +#include +#include #include #include #include From 6e61a4f49f14ac803e962c21c498ff60eb156ede Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Aug 2022 18:07:40 +0200 Subject: [PATCH 243/582] Update unit test --- src/Common/tests/gtest_lru_file_cache.cpp | 623 +++++++++++----------- 1 file changed, 314 insertions(+), 309 deletions(-) diff --git a/src/Common/tests/gtest_lru_file_cache.cpp b/src/Common/tests/gtest_lru_file_cache.cpp index bb454cc41eb..62a207d31f4 100644 --- a/src/Common/tests/gtest_lru_file_cache.cpp +++ b/src/Common/tests/gtest_lru_file_cache.cpp @@ -103,373 +103,376 @@ TEST(FileCache, get) DB::FileCacheSettings settings; settings.max_size = 30; settings.max_elements = 5; - auto cache = DB::FileCache(cache_base_path, settings); - cache.initialize(); - auto key = cache.hash("key1"); { - auto holder = cache.getOrSet(key, 0, 10, false); /// Add range [0, 9] - auto segments = fromHolder(holder); - /// Range was not present in cache. It should be added in cache as one while file segment. - ASSERT_EQ(segments.size(), 1); + auto cache = DB::FileCache(cache_base_path, settings); + cache.initialize(); + auto key = cache.hash("key1"); - assertRange(1, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::EMPTY); + { + auto holder = cache.getOrSet(key, 0, 10, false); /// Add range [0, 9] + auto segments = fromHolder(holder); + /// Range was not present in cache. It should be added in cache as one while file segment. + ASSERT_EQ(segments.size(), 1); - /// Exception because space not reserved. - /// EXPECT_THROW(download(segments[0]), DB::Exception); - /// Exception because space can be reserved only by downloader - /// EXPECT_THROW(segments[0]->reserve(segments[0]->range().size()), DB::Exception); + assertRange(1, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::EMPTY); - ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segments[0]->reserve(segments[0]->range().size())); - assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING); + /// Exception because space not reserved. + /// EXPECT_THROW(download(segments[0]), DB::Exception); + /// Exception because space can be reserved only by downloader + /// EXPECT_THROW(segments[0]->reserve(segments[0]->range().size()), DB::Exception); - download(segments[0]); - segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); - assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); - } + ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(segments[0]->reserve(segments[0]->range().size())); + assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING); - /// Current cache: [__________] - /// ^ ^ - /// 0 9 - ASSERT_EQ(cache.getFileSegmentsNum(), 1); - ASSERT_EQ(cache.getUsedCacheSize(), 10); + download(segments[0]); + segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); + assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); + } - { - /// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache. - auto holder = cache.getOrSet(key, 5, 10, false); - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 2); + /// Current cache: [__________] + /// ^ ^ + /// 0 9 + ASSERT_EQ(cache.getFileSegmentsNum(), 1); + ASSERT_EQ(cache.getUsedCacheSize(), 10); - assertRange(4, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); - assertRange(5, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::EMPTY); + { + /// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache. + auto holder = cache.getOrSet(key, 5, 10, false); + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 2); - ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[1]); - segments[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); - assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); - } + assertRange(4, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); + assertRange(5, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::EMPTY); - /// Current cache: [__________][_____] - /// ^ ^^ ^ - /// 0 910 14 - ASSERT_EQ(cache.getFileSegmentsNum(), 2); - ASSERT_EQ(cache.getUsedCacheSize(), 15); + ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(segments[1]); + segments[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); + assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); + } - { - auto holder = cache.getOrSet(key, 9, 1, false); /// Get [9, 9] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 1); - assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); - } + /// Current cache: [__________][_____] + /// ^ ^^ ^ + /// 0 910 14 + ASSERT_EQ(cache.getFileSegmentsNum(), 2); + ASSERT_EQ(cache.getUsedCacheSize(), 15); - { - auto holder = cache.getOrSet(key, 9, 2, false); /// Get [9, 10] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 2); - assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); - assertRange(9, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); - } + { + auto holder = cache.getOrSet(key, 9, 1, false); /// Get [9, 9] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 1); + assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); + } - { - auto holder = cache.getOrSet(key, 10, 1, false); /// Get [10, 10] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 1); - assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); - } + { + auto holder = cache.getOrSet(key, 9, 2, false); /// Get [9, 10] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 2); + assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); + assertRange(9, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); + } - complete(cache.getOrSet(key, 17, 4, false)); /// Get [17, 20] - complete(cache.getOrSet(key, 24, 3, false)); /// Get [24, 26] - /// complete(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] + { + auto holder = cache.getOrSet(key, 10, 1, false); /// Get [10, 10] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 1); + assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); + } - /// Current cache: [__________][_____] [____] [___][] - /// ^ ^^ ^ ^ ^ ^ ^^^ - /// 0 910 14 17 20 24 2627 - /// - ASSERT_EQ(cache.getFileSegmentsNum(), 4); - ASSERT_EQ(cache.getUsedCacheSize(), 22); + complete(cache.getOrSet(key, 17, 4, false)); /// Get [17, 20] + complete(cache.getOrSet(key, 24, 3, false)); /// Get [24, 26] + /// complete(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] - { - auto holder = cache.getOrSet(key, 0, 26, false); /// Get [0, 25] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 6); - - assertRange(11, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); - assertRange(12, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); - - /// Missing [15, 16] should be added in cache. - assertRange(13, segments[2], DB::FileSegment::Range(15, 16), DB::FileSegment::State::EMPTY); - - ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[2]); - - segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); - - assertRange(14, segments[3], DB::FileSegment::Range(17, 20), DB::FileSegment::State::DOWNLOADED); - - /// New [21, 23], but will not be added in cache because of elements limit (5) - assertRange(15, segments[4], DB::FileSegment::Range(21, 23), DB::FileSegment::State::EMPTY); - ASSERT_TRUE(segments[4]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_FALSE(segments[4]->reserve(1)); - - assertRange(16, segments[5], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - - /// Current cache: [__________][_____][ ][____] [___] - /// ^ ^ ^ - /// 0 20 24 + /// Current cache: [__________][_____] [____] [___][] + /// ^ ^^ ^ ^ ^ ^ ^^^ + /// 0 910 14 17 20 24 2627 /// + ASSERT_EQ(cache.getFileSegmentsNum(), 4); + ASSERT_EQ(cache.getUsedCacheSize(), 22); - /// Range [27, 27] must be evicted in previous getOrSet [0, 25]. - /// Let's not invalidate pointers to returned segments from range [0, 25] and - /// as max elements size is reached, next attempt to put something in cache should fail. - /// This will also check that [27, 27] was indeed evicted. + { + auto holder = cache.getOrSet(key, 0, 26, false); /// Get [0, 25] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 6); - auto holder1 = cache.getOrSet(key, 27, 1, false); - auto segments_1 = fromHolder(holder1); /// Get [27, 27] - ASSERT_EQ(segments_1.size(), 1); - assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); - } + assertRange(11, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); + assertRange(12, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); - { - auto holder = cache.getOrSet(key, 12, 10, false); /// Get [12, 21] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 4); + /// Missing [15, 16] should be added in cache. + assertRange(13, segments[2], DB::FileSegment::Range(15, 16), DB::FileSegment::State::EMPTY); - assertRange(18, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); - assertRange(19, segments[1], DB::FileSegment::Range(15, 16), DB::FileSegment::State::DOWNLOADED); - assertRange(20, segments[2], DB::FileSegment::Range(17, 20), DB::FileSegment::State::DOWNLOADED); + ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(segments[2]); - assertRange(21, segments[3], DB::FileSegment::Range(21, 21), DB::FileSegment::State::EMPTY); + segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); - ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[3]); + assertRange(14, segments[3], DB::FileSegment::Range(17, 20), DB::FileSegment::State::DOWNLOADED); - segments[3]->completeWithState(DB::FileSegment::State::DOWNLOADED); - ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED); - } + /// New [21, 23], but will not be added in cache because of elements limit (5) + assertRange(15, segments[4], DB::FileSegment::Range(21, 23), DB::FileSegment::State::EMPTY); + ASSERT_TRUE(segments[4]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_FALSE(segments[4]->reserve(1)); - /// Current cache: [_____][__][____][_] [___] - /// ^ ^ ^ ^ ^ - /// 10 17 21 24 26 + assertRange(16, segments[5], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - ASSERT_EQ(cache.getFileSegmentsNum(), 5); + /// Current cache: [__________][_____][ ][____] [___] + /// ^ ^ ^ + /// 0 20 24 + /// - { - auto holder = cache.getOrSet(key, 23, 5, false); /// Get [23, 28] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 3); + /// Range [27, 27] must be evicted in previous getOrSet [0, 25]. + /// Let's not invalidate pointers to returned segments from range [0, 25] and + /// as max elements size is reached, next attempt to put something in cache should fail. + /// This will also check that [27, 27] was indeed evicted. - assertRange(22, segments[0], DB::FileSegment::Range(23, 23), DB::FileSegment::State::EMPTY); - assertRange(23, segments[1], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - assertRange(24, segments[2], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); + auto holder1 = cache.getOrSet(key, 27, 1, false); + auto segments_1 = fromHolder(holder1); /// Get [27, 27] + ASSERT_EQ(segments_1.size(), 1); + assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); + } - ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[0]); - prepareAndDownload(segments[2]); - segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); - segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); - } + { + auto holder = cache.getOrSet(key, 12, 10, false); /// Get [12, 21] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 4); - /// Current cache: [____][_] [][___][__] - /// ^ ^ ^^^ ^^ ^ - /// 17 21 2324 26 28 + assertRange(18, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); + assertRange(19, segments[1], DB::FileSegment::Range(15, 16), DB::FileSegment::State::DOWNLOADED); + assertRange(20, segments[2], DB::FileSegment::Range(17, 20), DB::FileSegment::State::DOWNLOADED); - { - auto holder5 = cache.getOrSet(key, 2, 3,false); /// Get [2, 4] - auto s5 = fromHolder(holder5); - ASSERT_EQ(s5.size(), 1); - assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY); + assertRange(21, segments[3], DB::FileSegment::Range(21, 21), DB::FileSegment::State::EMPTY); - auto holder1 = cache.getOrSet(key, 30, 2, false); /// Get [30, 31] - auto s1 = fromHolder(holder1); - ASSERT_EQ(s1.size(), 1); - assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY); + ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(segments[3]); - ASSERT_TRUE(s5[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(s5[0]); - prepareAndDownload(s1[0]); - s5[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); - s1[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); + segments[3]->completeWithState(DB::FileSegment::State::DOWNLOADED); + ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED); + } + + /// Current cache: [_____][__][____][_] [___] + /// ^ ^ ^ ^ ^ + /// 10 17 21 24 26 + + ASSERT_EQ(cache.getFileSegmentsNum(), 5); + + { + auto holder = cache.getOrSet(key, 23, 5, false); /// Get [23, 28] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 3); + + assertRange(22, segments[0], DB::FileSegment::Range(23, 23), DB::FileSegment::State::EMPTY); + assertRange(23, segments[1], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); + assertRange(24, segments[2], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); + + ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(segments[0]); + prepareAndDownload(segments[2]); + segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); + segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); + } + + /// Current cache: [____][_] [][___][__] + /// ^ ^ ^^^ ^^ ^ + /// 17 21 2324 26 28 + + { + auto holder5 = cache.getOrSet(key, 2, 3,false); /// Get [2, 4] + auto s5 = fromHolder(holder5); + ASSERT_EQ(s5.size(), 1); + assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY); + + auto holder1 = cache.getOrSet(key, 30, 2, false); /// Get [30, 31] + auto s1 = fromHolder(holder1); + ASSERT_EQ(s1.size(), 1); + assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY); + + ASSERT_TRUE(s5[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(s5[0]); + prepareAndDownload(s1[0]); + s5[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); + s1[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); + + /// Current cache: [___] [_][___][_] [__] + /// ^ ^ ^ ^ ^ ^ ^ ^ + /// 2 4 23 24 26 27 30 31 + + auto holder2 = cache.getOrSet(key, 23, 1, false); /// Get [23, 23] + auto s2 = fromHolder(holder2); + ASSERT_EQ(s2.size(), 1); + + auto holder3 = cache.getOrSet(key, 24, 3, false); /// Get [24, 26] + auto s3 = fromHolder(holder3); + ASSERT_EQ(s3.size(), 1); + + auto holder4 = cache.getOrSet(key, 27, 1, false); /// Get [27, 27] + auto s4 = fromHolder(holder4); + ASSERT_EQ(s4.size(), 1); + + /// All cache is now unreleasable because pointers are still hold + auto holder6 = cache.getOrSet(key, 0, 40, false); + auto f = fromHolder(holder6); + ASSERT_EQ(f.size(), 9); + + assertRange(27, f[0], DB::FileSegment::Range(0, 1), DB::FileSegment::State::EMPTY); + assertRange(28, f[2], DB::FileSegment::Range(5, 22), DB::FileSegment::State::EMPTY); + assertRange(29, f[6], DB::FileSegment::Range(28, 29), DB::FileSegment::State::EMPTY); + assertRange(30, f[8], DB::FileSegment::Range(32, 39), DB::FileSegment::State::EMPTY); + + ASSERT_TRUE(f[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(f[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(f[6]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(f[8]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + + ASSERT_FALSE(f[0]->reserve(1)); + ASSERT_FALSE(f[2]->reserve(1)); + ASSERT_FALSE(f[6]->reserve(1)); + ASSERT_FALSE(f[8]->reserve(1)); + } + + { + auto holder = cache.getOrSet(key, 2, 3, false); /// Get [2, 4] + auto segments = fromHolder(holder); + ASSERT_EQ(segments.size(), 1); + assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); + } /// Current cache: [___] [_][___][_] [__] /// ^ ^ ^ ^ ^ ^ ^ ^ /// 2 4 23 24 26 27 30 31 - auto holder2 = cache.getOrSet(key, 23, 1, false); /// Get [23, 23] - auto s2 = fromHolder(holder2); - ASSERT_EQ(s2.size(), 1); - - auto holder3 = cache.getOrSet(key, 24, 3, false); /// Get [24, 26] - auto s3 = fromHolder(holder3); - ASSERT_EQ(s3.size(), 1); - - auto holder4 = cache.getOrSet(key, 27, 1, false); /// Get [27, 27] - auto s4 = fromHolder(holder4); - ASSERT_EQ(s4.size(), 1); - - /// All cache is now unreleasable because pointers are still hold - auto holder6 = cache.getOrSet(key, 0, 40, false); - auto f = fromHolder(holder6); - ASSERT_EQ(f.size(), 9); - - assertRange(27, f[0], DB::FileSegment::Range(0, 1), DB::FileSegment::State::EMPTY); - assertRange(28, f[2], DB::FileSegment::Range(5, 22), DB::FileSegment::State::EMPTY); - assertRange(29, f[6], DB::FileSegment::Range(28, 29), DB::FileSegment::State::EMPTY); - assertRange(30, f[8], DB::FileSegment::Range(32, 39), DB::FileSegment::State::EMPTY); - - ASSERT_TRUE(f[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(f[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(f[6]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(f[8]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - - ASSERT_FALSE(f[0]->reserve(1)); - ASSERT_FALSE(f[2]->reserve(1)); - ASSERT_FALSE(f[6]->reserve(1)); - ASSERT_FALSE(f[8]->reserve(1)); - } - - { - auto holder = cache.getOrSet(key, 2, 3, false); /// Get [2, 4] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 1); - assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); - } - - /// Current cache: [___] [_][___][_] [__] - /// ^ ^ ^ ^ ^ ^ ^ ^ - /// 2 4 23 24 26 27 30 31 - - { - auto holder = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] - auto segments = fromHolder(holder); - ASSERT_EQ(segments.size(), 3); - - assertRange(32, segments[0], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - assertRange(33, segments[1], DB::FileSegment::Range(27, 27), DB::FileSegment::State::DOWNLOADED); - - assertRange(34, segments[2], DB::FileSegment::Range(28, 29), DB::FileSegment::State::EMPTY); - ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADING); - - bool lets_start_download = false; - std::mutex mutex; - std::condition_variable cv; - - std::thread other_1([&] { - DB::ThreadStatus thread_status_1; - auto query_context_1 = DB::Context::createCopy(getContext().context); - query_context_1->makeQueryContext(); - query_context_1->setCurrentQueryId("query_id_1"); - DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - thread_status_1.attachQueryContext(query_context_1); - - auto holder_2 = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] once again. - auto segments_2 = fromHolder(holder_2); + auto holder = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] + auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); - assertRange(35, segments_2[0], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - assertRange(36, segments_2[1], DB::FileSegment::Range(27, 27), DB::FileSegment::State::DOWNLOADED); - assertRange(37, segments_2[2], DB::FileSegment::Range(28, 29), DB::FileSegment::State::DOWNLOADING); + assertRange(32, segments[0], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); + assertRange(33, segments[1], DB::FileSegment::Range(27, 27), DB::FileSegment::State::DOWNLOADED); - ASSERT_TRUE(segments[2]->getOrSetDownloader() != DB::FileSegment::getCallerId()); + assertRange(34, segments[2], DB::FileSegment::Range(28, 29), DB::FileSegment::State::EMPTY); + ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADING); + bool lets_start_download = false; + std::mutex mutex; + std::condition_variable cv; + + std::thread other_1([&] { - std::lock_guard lock(mutex); - lets_start_download = true; - } - cv.notify_one(); + DB::ThreadStatus thread_status_1; + auto query_context_1 = DB::Context::createCopy(getContext().context); + query_context_1->makeQueryContext(); + query_context_1->setCurrentQueryId("query_id_1"); + DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); + thread_status_1.attachQueryContext(query_context_1); - segments_2[2]->wait(); - ASSERT_TRUE(segments_2[2]->state() == DB::FileSegment::State::DOWNLOADED); - }); + auto holder_2 = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] once again. + auto segments_2 = fromHolder(holder_2); + ASSERT_EQ(segments.size(), 3); - { - std::unique_lock lock(mutex); - cv.wait(lock, [&]{ return lets_start_download; }); - } + assertRange(35, segments_2[0], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); + assertRange(36, segments_2[1], DB::FileSegment::Range(27, 27), DB::FileSegment::State::DOWNLOADED); + assertRange(37, segments_2[2], DB::FileSegment::Range(28, 29), DB::FileSegment::State::DOWNLOADING); - prepareAndDownload(segments[2]); - segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); - ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED); + ASSERT_TRUE(segments[2]->getOrSetDownloader() != DB::FileSegment::getCallerId()); + ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADING); - other_1.join(); - } + { + std::lock_guard lock(mutex); + lets_start_download = true; + } + cv.notify_one(); - /// Current cache: [___] [___][_][__][__] - /// ^ ^ ^ ^ ^^ ^^ ^ - /// 2 4 24 26 27 2930 31 - - { - /// Now let's check the similar case but getting ERROR state after segment->wait(), when - /// state is changed not manually via segment->complete(state) but from destructor of holder - /// and notify_all() is also called from destructor of holder. - - std::optional holder; - holder.emplace(cache.getOrSet(key, 3, 23, false)); /// Get [3, 25] - - auto segments = fromHolder(*holder); - ASSERT_EQ(segments.size(), 3); - - assertRange(38, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); - - assertRange(39, segments[1], DB::FileSegment::Range(5, 23), DB::FileSegment::State::EMPTY); - ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segments[1]->state() == DB::FileSegment::State::DOWNLOADING); - - assertRange(40, segments[2], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - - bool lets_start_download = false; - std::mutex mutex; - std::condition_variable cv; - - std::thread other_1([&] - { - DB::ThreadStatus thread_status_1; - auto query_context_1 = DB::Context::createCopy(getContext().context); - query_context_1->makeQueryContext(); - query_context_1->setCurrentQueryId("query_id_1"); - DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - thread_status_1.attachQueryContext(query_context_1); - - auto holder_2 = cache.getOrSet(key, 3, 23, false); /// Get [3, 25] once again - auto segments_2 = fromHolder(*holder); - ASSERT_EQ(segments_2.size(), 3); - - assertRange(41, segments_2[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); - assertRange(42, segments_2[1], DB::FileSegment::Range(5, 23), DB::FileSegment::State::DOWNLOADING); - assertRange(43, segments_2[2], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); - - ASSERT_TRUE(segments_2[1]->getDownloader() != DB::FileSegment::getCallerId()); - ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::DOWNLOADING); + segments_2[2]->wait(); + ASSERT_TRUE(segments_2[2]->state() == DB::FileSegment::State::DOWNLOADED); + }); { - std::lock_guard lock(mutex); - lets_start_download = true; + std::unique_lock lock(mutex); + cv.wait(lock, [&]{ return lets_start_download; }); } - cv.notify_one(); - segments_2[1]->wait(); - printRanges(segments_2); - ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); + prepareAndDownload(segments[2]); + segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); + ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED); - ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments_2[1]); - segments_2[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); - }); - - { - std::unique_lock lock(mutex); - cv.wait(lock, [&]{ return lets_start_download; }); + other_1.join(); } - holder.reset(); - other_1.join(); - printRanges(segments); - ASSERT_TRUE(segments[1]->state() == DB::FileSegment::State::DOWNLOADED); + /// Current cache: [___] [___][_][__][__] + /// ^ ^ ^ ^ ^^ ^^ ^ + /// 2 4 24 26 27 2930 31 + + { + /// Now let's check the similar case but getting ERROR state after segment->wait(), when + /// state is changed not manually via segment->complete(state) but from destructor of holder + /// and notify_all() is also called from destructor of holder. + + std::optional holder; + holder.emplace(cache.getOrSet(key, 3, 23, false)); /// Get [3, 25] + + auto segments = fromHolder(*holder); + ASSERT_EQ(segments.size(), 3); + + assertRange(38, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); + + assertRange(39, segments[1], DB::FileSegment::Range(5, 23), DB::FileSegment::State::EMPTY); + ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(segments[1]->state() == DB::FileSegment::State::DOWNLOADING); + + assertRange(40, segments[2], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); + + bool lets_start_download = false; + std::mutex mutex; + std::condition_variable cv; + + std::thread other_1([&] + { + DB::ThreadStatus thread_status_1; + auto query_context_1 = DB::Context::createCopy(getContext().context); + query_context_1->makeQueryContext(); + query_context_1->setCurrentQueryId("query_id_1"); + DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); + thread_status_1.attachQueryContext(query_context_1); + + auto holder_2 = cache.getOrSet(key, 3, 23, false); /// Get [3, 25] once again + auto segments_2 = fromHolder(*holder); + ASSERT_EQ(segments_2.size(), 3); + + assertRange(41, segments_2[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); + assertRange(42, segments_2[1], DB::FileSegment::Range(5, 23), DB::FileSegment::State::DOWNLOADING); + assertRange(43, segments_2[2], DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); + + ASSERT_TRUE(segments_2[1]->getDownloader() != DB::FileSegment::getCallerId()); + ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::DOWNLOADING); + + { + std::lock_guard lock(mutex); + lets_start_download = true; + } + cv.notify_one(); + + segments_2[1]->wait(); + printRanges(segments_2); + ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); + + ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(segments_2[1]); + segments_2[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); + }); + + { + std::unique_lock lock(mutex); + cv.wait(lock, [&]{ return lets_start_download; }); + } + + holder.reset(); + other_1.join(); + printRanges(segments); + ASSERT_TRUE(segments[1]->state() == DB::FileSegment::State::DOWNLOADED); + } } /// Current cache: [___][ ][___][_][__] @@ -481,6 +484,7 @@ TEST(FileCache, get) auto cache2 = DB::FileCache(cache_base_path, settings); cache2.initialize(); + auto key = cache2.hash("key1"); auto holder1 = cache2.getOrSet(key, 2, 28, false); /// Get [2, 29] @@ -501,6 +505,7 @@ TEST(FileCache, get) settings2.max_file_segment_size = 10; auto cache2 = DB::FileCache(caches_dir / "cache2", settings2); cache2.initialize(); + auto key = cache2.hash("key1"); auto holder1 = cache2.getOrSet(key, 0, 25, false); /// Get [0, 24] auto segments1 = fromHolder(holder1); From a78571de1175e5fae0b39843f85937a377c7a02b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Aug 2022 18:47:47 +0200 Subject: [PATCH 244/582] Fix --- src/Common/FileCache.cpp | 39 +++++++++++++++++++++------------------ src/Common/FileCache.h | 2 ++ 2 files changed, 23 insertions(+), 18 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 44ecac2cc02..d290ee30ed1 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -59,6 +59,24 @@ String FileCache::getPathInLocalCache(const Key & key) const return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str; } +void FileCache::removeKeyDirectoryIfExists(const Key & key, std::lock_guard & /* cache_lock */) const +{ + /// Note: it is guaranteed that there is no concurrency here with files deletion + /// because cache key directories are create only in FileCache class under cache_lock. + + auto key_str = key.toString(); + auto key_prefix_path = fs::path(cache_base_path) / key_str.substr(0, 3); + auto key_path = key_prefix_path / key_str; + + if (!fs::exists(key_path)) + return; + + fs::remove_all(key_path); + + if (fs::is_empty(key_prefix_path)) + fs::remove(key_prefix_path); +} + static bool isQueryInitialized() { return CurrentThread::isInitialized() @@ -174,15 +192,8 @@ FileSegments FileCache::getImpl( const auto & file_segments = it->second; if (file_segments.empty()) { - auto key_path = getPathInLocalCache(key); - files.erase(key); - - /// Note: it is guaranteed that there is no concurrency with files deletion, - /// because cache files are deleted only inside FileCache and under cache lock. - if (fs::exists(key_path)) - fs::remove_all(key_path); - + removeKeyDirectoryIfExists(key, cache_lock); return {}; } @@ -827,14 +838,10 @@ void FileCache::removeIfExists(const Key & key) } } - auto key_path = getPathInLocalCache(key); - if (!some_cells_were_skipped) { files.erase(key); - - if (fs::exists(key_path)) - fs::remove_all(key_path); + removeKeyDirectoryIfExists(key, cache_lock); } } @@ -924,12 +931,8 @@ void FileCache::remove( if (is_initialized && offsets.empty()) { - auto key_path = getPathInLocalCache(key); - files.erase(key); - - if (fs::exists(key_path)) - fs::remove_all(key_path); + removeKeyDirectoryIfExists(key, cache_lock); } } catch (...) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index b5b1e917e76..cdd0612cc00 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -261,6 +261,8 @@ private: void assertCacheCellsCorrectness(const FileSegmentsByOffset & cells_by_offset, std::lock_guard & cache_lock); + void removeKeyDirectoryIfExists(const Key & key, std::lock_guard & cache_lock) const; + /// Used to track and control the cache access of each query. /// Through it, we can realize the processing of different queries by the cache layer. struct QueryContext From 251bdab9a0eab95a2329bf55db3d5da2a6b5365c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Aug 2022 20:08:47 +0200 Subject: [PATCH 245/582] Better grep --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f745d11438f..a8babeb59c0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1599,7 +1599,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif /// We don't control the amount of refs for temporary parts so we cannot decide can we remove blobs /// or not. So we are not doing it bool keep_shared = false; - if (it->path().find("tmp-fetch") != std::string::npos) + if (it->path().find("fetch") != std::string::npos) { keep_shared = disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication; if (keep_shared) From e659b4553dac6e495dc99d7a17289851884c9132 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 Aug 2022 21:53:57 +0300 Subject: [PATCH 246/582] Update clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f40c93c6f5d..7e060a9f687 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -533,7 +533,7 @@ class TestCase: else: # If --database is not specified, we will create temporary database with # unique name and we will recreate and drop it for each test - def random_str(length=6): + def random_str(length=8): alphabet = string.ascii_lowercase + string.digits # NOTE: it is important not to use default random generator, since it shares state. return "".join( From 10a7e73d52ce232a5be15aa31ba9ee86a12b6913 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Aug 2022 21:07:39 +0200 Subject: [PATCH 247/582] Fix unit test --- src/{Common => Interpreters}/tests/gtest_lru_file_cache.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename src/{Common => Interpreters}/tests/gtest_lru_file_cache.cpp (99%) diff --git a/src/Common/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp similarity index 99% rename from src/Common/tests/gtest_lru_file_cache.cpp rename to src/Interpreters/tests/gtest_lru_file_cache.cpp index 6a5a54fc84d..a3fe3bbcba5 100644 --- a/src/Common/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -1,11 +1,11 @@ #include #include #include -#include +#include #include +#include #include #include -#include #include #include #include From e0597700ebe8a384431d276168f8faf218dc1ee2 Mon Sep 17 00:00:00 2001 From: Robert Coelho Date: Wed, 31 Aug 2022 20:34:14 -0400 Subject: [PATCH 248/582] Fix uncaught DNS_ERROR on failed connection to replicas DNS_ERROR would cause the replica to not be marked as unusable, resulting in the replica being repeatedly reattempted on subsequent queries and for connection failover to break. (This is common in Kubernetes setups where a replica has failed and it's DNS record is returning NXDOMAIN) On SELECT, would additionally result in an intermittent query error if the failed replica is chosen: "Code: 198. DB::Exception: Received from localhost:9000. DB::Exception: Not found address of host: chi-clickhouse-main-2-0: While executing Remote. (DNS_ERROR)" --- src/Client/ConnectionEstablisher.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 3ad9f6ba95c..e464ec464f1 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -90,6 +90,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: catch (const Exception & e) { if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT + && e.code() != ErrorCodes::DNS_ERROR && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) throw; From d9a516288c05b82857264df485c341aa4289264e Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 1 Sep 2022 11:56:10 +0800 Subject: [PATCH 249/582] Resolve comments --- src/Common/OpenTelemetryTraceContext.cpp | 9 +- src/Common/ThreadPool.cpp | 27 +++--- src/Common/ThreadPool.h | 109 +++++++---------------- 3 files changed, 44 insertions(+), 101 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index f00b6e5b873..c05d3385bc8 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -285,14 +285,7 @@ TracingContextHolder::TracingContextHolder( /// This object is created to initialize tracing context on a new thread, /// it's helpful to record the thread_id so that we know the thread switching from the span log - try - { - this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); - } - catch (...) - { - /// It's acceptable that this attribute is not recorded if any exception is raised - } + this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); /// set up trace context on current thread current_thread_trace_context = _parent_trace_context; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 739fd59ecd1..0b89139fa53 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -87,7 +87,7 @@ void ThreadPoolImpl::setQueueSize(size_t value) template template -ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool enable_tracing_context_propagation) +ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) { auto on_error = [&](const std::string & reason) { @@ -150,17 +150,10 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti } } - if (enable_tracing_context_propagation) - { - /// Tracing context on this thread is used as parent context for the sub-thread that runs the job - const auto ¤t_thread_context = DB::OpenTelemetry::CurrentContext(); - jobs.emplace(std::move(job), priority, current_thread_context); - } - else - { - DB::OpenTelemetry::TracingContextOnThread empty; - jobs.emplace(std::move(job), priority, empty); - } + jobs.emplace(std::move(job), + priority, + /// Tracing context on this thread is used as parent context for the sub-thread that runs the job + propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread()); ++scheduled_jobs; new_job_or_shutdown.notify_one(); @@ -182,9 +175,9 @@ bool ThreadPoolImpl::trySchedule(Job job, int priority, uint64_t wait_mi } template -void ThreadPoolImpl::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds, bool enable_tracing_context_propagation) +void ThreadPoolImpl::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) { - scheduleImpl(std::move(job), priority, wait_microseconds, enable_tracing_context_propagation); + scheduleImpl(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context); } template @@ -288,12 +281,13 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!need_shutdown) { + ALLOW_ALLOCATIONS_IN_SCOPE; + /// Set up tracing context for this thread by its parent context DB::OpenTelemetry::TracingContextHolder thread_trace_context("ThreadPool::worker()", parent_thead_trace_context); try { - ALLOW_ALLOCATIONS_IN_SCOPE; CurrentMetrics::Increment metric_active_threads( std::is_same_v ? CurrentMetrics::GlobalThreadActive : CurrentMetrics::LocalThreadActive); @@ -356,7 +350,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ template class ThreadPoolImpl; -template class ThreadPoolImpl; +template class ThreadPoolImpl>; +template class ThreadFromGlobalPoolImpl; std::unique_ptr GlobalThreadPool::the_instance; diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 5b834beeca1..fc5377b3783 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -56,7 +56,7 @@ public: bool trySchedule(Job job, int priority = 0, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. - void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0, bool enable_tracing_context_propagation = true); + void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitrary order. @@ -113,7 +113,7 @@ private: std::exception_ptr first_exception; template - ReturnType scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool enable_tracing_context_propagation = true); + ReturnType scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); void worker(typename std::list::iterator thread_it); @@ -156,14 +156,18 @@ public: /** Looks like std::thread but allocates threads in GlobalThreadPool. * Also holds ThreadStatus for ClickHouse. + * + * NOTE: User code should use 'ThreadFromGlobalPool' declared below instead of directly using this class. + * */ -class ThreadFromGlobalPool : boost::noncopyable +template +class ThreadFromGlobalPoolImpl : boost::noncopyable { public: - ThreadFromGlobalPool() = default; + ThreadFromGlobalPoolImpl() = default; template - explicit ThreadFromGlobalPool(Function && func, Args &&... args) + explicit ThreadFromGlobalPoolImpl(Function && func, Args &&... args) : state(std::make_shared()) { /// NOTE: @@ -187,15 +191,19 @@ public: /// before sending signal that permits to join this thread. DB::ThreadStatus thread_status; std::apply(function, arguments); - }); + }, + 0, // default priority + 0, // default wait_microseconds + propagate_opentelemetry_context + ); } - ThreadFromGlobalPool(ThreadFromGlobalPool && rhs) noexcept + ThreadFromGlobalPoolImpl(ThreadFromGlobalPoolImpl && rhs) noexcept { *this = std::move(rhs); } - ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs) noexcept + ThreadFromGlobalPoolImpl & operator=(ThreadFromGlobalPoolImpl && rhs) noexcept { if (initialized()) abort(); @@ -203,7 +211,7 @@ public: return *this; } - ~ThreadFromGlobalPool() + ~ThreadFromGlobalPoolImpl() { if (initialized()) abort(); @@ -256,72 +264,19 @@ protected: } }; -/// This class is used by ThreadPool only to allocate threads in GlobalThreadPool. -/// Any user code should use ThreadFromGlobalPool instead of this class to schedule a job in a thread. -/// -/// The difference between this class and ThreadFromGlobalPool is that this class disables the tracing context propagation to underlying thread. -/// If the context is propagated, not only the underlying worker will restore context but also the worker of ThreadPool. -/// -/// Since workers of ThreadPool won't exit until the ThreadPool is destroyed, the context restored by underlying worker won't be deleted for a very long time -/// which would cause wrong contexts for jobs for ThreadPool -/// -class Thread4ThreadPool : public ThreadFromGlobalPool -{ -public: - Thread4ThreadPool() = default; - - template - explicit Thread4ThreadPool(Function && func, Args &&... args) - { - state = std::make_shared(); - - /// NOTE: - /// - If this will throw an exception, the destructor won't be called - /// - this pointer cannot be passed in the lambda, since after detach() it will not be valid - GlobalThreadPool::instance().scheduleOrThrow([ - state = state, - func = std::forward(func), - args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture - { - SCOPE_EXIT(state->event.set()); - - state->thread_id = std::this_thread::get_id(); - - /// This moves are needed to destroy function and arguments before exit. - /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. - auto function = std::move(func); - auto arguments = std::move(args); - - /// Thread status holds raw pointer on query context, thus it always must be destroyed - /// before sending signal that permits to join this thread. - DB::ThreadStatus thread_status; - std::apply(function, arguments); - }, - - // default priority - 0, - - // default wait_microseconds - 0, - - /// Disable tracing context propagation on underlying thread pool because ThreadPool already has kept the context in its jobs. - false - ); - } - - Thread4ThreadPool(Thread4ThreadPool && rhs) noexcept - { - *this = std::move(rhs); - } - - Thread4ThreadPool & operator=(Thread4ThreadPool && rhs) noexcept - { - if (initialized()) - abort(); - state = std::move(rhs.state); - return *this; - } -}; - /// Recommended thread pool for the case when multiple thread pools are created and destroyed. -using ThreadPool = ThreadPoolImpl; +/// +/// The template parameter of ThreadFromGlobalPool is set to false to disable tracing context propagation to underlying worker. +/// Because ThreadFromGlobalPool schedules a job upon GlobalThreadPool, this means there will be two workers to schedule a job in 'ThreadPool', +/// one is at GlobalThreadPool level, the other is at ThreadPool level, so tracing context will be initialized on the same thread twice. +/// +/// Once the worker on ThreadPool gains the control of execution, it won't return until it's shutdown, +/// which means the tracing context initialized at underlying worker level won't be delete for a very long time. +/// This would cause wrong context for further jobs scheduled in ThreadPool. +/// +/// To make sure the tracing context are correctly propagated, we explicitly disable context propagation(including initialization and de-initialization) at underlying worker level. +/// +using ThreadPool = ThreadPoolImpl>; + +/// An alias for user code to execute a job in the global thread pool +using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl; From 19097c433d1518ab8d1f8a729b52d19b5ae7c6dd Mon Sep 17 00:00:00 2001 From: Robert Coelho Date: Thu, 1 Sep 2022 01:34:41 -0400 Subject: [PATCH 250/582] fix compile error --- src/Client/ConnectionEstablisher.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index e464ec464f1..757927d70bc 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; + extern const int DNS_ERROR; } ConnectionEstablisher::ConnectionEstablisher( From c7d1dd942c5d51847da553512993c454af403a2c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 07:56:06 +0000 Subject: [PATCH 251/582] Address PR comments --- src/Common/Base64.cpp | 33 ++++++++++++++++++ src/Common/Base64.h | 10 ++++++ src/Coordination/KeeperStorage.cpp | 18 +++------- src/Storages/StorageKeeperMap.cpp | 55 ++++++++---------------------- 4 files changed, 63 insertions(+), 53 deletions(-) create mode 100644 src/Common/Base64.cpp create mode 100644 src/Common/Base64.h diff --git a/src/Common/Base64.cpp b/src/Common/Base64.cpp new file mode 100644 index 00000000000..74ce979b5b1 --- /dev/null +++ b/src/Common/Base64.cpp @@ -0,0 +1,33 @@ +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +std::string base64Encode(const std::string & decoded, bool url_encoding) +{ + std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ostr.exceptions(std::ios::failbit); + Poco::Base64Encoder encoder(ostr, url_encoding ? Poco::BASE64_URL_ENCODING : 0); + encoder.rdbuf()->setLineLength(0); + encoder << decoded; + encoder.close(); + return ostr.str(); +} + +std::string base64Decode(const std::string & encoded, bool url_encoding) +{ + std::string decoded; + Poco::MemoryInputStream istr(encoded.data(), encoded.size()); + Poco::Base64Decoder decoder(istr, url_encoding ? Poco::BASE64_URL_ENCODING : 0); + Poco::StreamCopier::copyToString(decoder, decoded); + return decoded; +} + +} diff --git a/src/Common/Base64.h b/src/Common/Base64.h new file mode 100644 index 00000000000..39327cd5e08 --- /dev/null +++ b/src/Common/Base64.h @@ -0,0 +1,10 @@ +#pragma once + +namespace DB +{ + +std::string base64Encode(const std::string & decoded, bool url_encoding = false); + +std::string base64Decode(const std::string & encoded, bool url_encoding = false); + +} diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 693a1b16f0d..044afd34d5b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1,11 +1,11 @@ #include #include -#include #include #include #include -#include #include + +#include #include #include #include @@ -15,8 +15,11 @@ #include #include #include + #include #include +#include + #include #include #include @@ -36,17 +39,6 @@ namespace ErrorCodes namespace { -String base64Encode(const String & decoded) -{ - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); -} - String getSHA1(const String & userdata) { Poco::SHA1Engine engine; diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 50c34871410..51886d6da3b 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -23,20 +24,17 @@ #include #include -#include "Common/Exception.h" -#include "Common/ZooKeeper/IKeeper.h" +#include +#include +#include #include #include #include #include -#include "Core/UUID.h" -#include "base/types.h" + +#include #include -#include -#include -#include -#include namespace DB { @@ -53,33 +51,11 @@ namespace ErrorCodes namespace { -std::string base64Encode(const std::string & decoded) -{ - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr, Poco::BASE64_URL_ENCODING); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); -} - -std::string base64Decode(const std::string & encoded) -{ - std::string decoded; - Poco::MemoryInputStream istr(encoded.data(), encoded.size()); - Poco::Base64Decoder decoder(istr, Poco::BASE64_URL_ENCODING); - Poco::StreamCopier::copyToString(decoder, decoded); - return decoded; -} - -constexpr std::string_view default_host = "default"; - std::string_view getBaseName(const std::string_view path) { auto last_slash = path.find_last_of('/'); if (last_slash == std::string_view::npos) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); return path.substr(last_slash + 1); } @@ -98,7 +74,7 @@ public: { auto primary_key = storage.getPrimaryKey(); assert(primary_key.size() == 1); - primary_key_pos = getHeader().getPositionByName(storage.getPrimaryKey()[0]); + primary_key_pos = getHeader().getPositionByName(primary_key[0]); } std::string getName() const override { return "StorageKeeperMapSink"; } @@ -123,7 +99,7 @@ public: ++idx; } - auto key = base64Encode(wb_key.str()); + auto key = base64Encode(wb_key.str(), true); new_values[std::move(key)] = std::move(wb_value.str()); } } @@ -226,7 +202,7 @@ public: auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); for (auto & raw_key : raw_keys) - raw_key = base64Encode(raw_key); + raw_key = base64Encode(raw_key, true); return storage.getBySerializedKeys(raw_keys, nullptr); } @@ -257,7 +233,7 @@ StorageKeeperMap::StorageKeeperMap( , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { - if (table_id.uuid == UUIDHelpers::Nil) + if (!table_id.hasUUID()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); setInMemoryMetadata(metadata); @@ -572,7 +548,7 @@ Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPOD auto raw_keys = serializeKeysToRawString(keys[0]); if (raw_keys.size() != keys[0].column->size()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); return getBySerializedKeys(raw_keys, &null_map); } @@ -622,7 +598,7 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k if (code == Coordination::Error::ZOK) { - fillColumns(base64Decode(keys[i]), response.data, primary_key_pos, sample_block, columns); + fillColumns(base64Decode(keys[i], true), response.data, primary_key_pos, sample_block, columns); } else if (code == Coordination::Error::ZNONODE) { @@ -658,11 +634,10 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.empty() || engine_args.size() > 3) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-2 arguments:\n" + "Storage KeeperMap requires 1-3 arguments:\n" "root_path: path in the Keeper where the values will be stored (required)\n" "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)\n", - default_host); + "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); From d31148239157261ec7049790a51db7208aa6c131 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Sep 2022 07:59:01 +0000 Subject: [PATCH 252/582] More renamings for consistency AST Fuzzer (ASan) --> AST Fuzzer (asan) AST Fuzzer (TSan) --> AST Fuzzer (tsan) AST Fuzzer (MSan) --> AST Fuzzer (msan) AST Fuzzer (UBSan) --> AST Fuzzer (ubsan) --- .github/workflows/master.yml | 8 ++++---- .github/workflows/pull_request.yml | 8 ++++---- tests/ci/ci_config.py | 8 ++++---- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f0c616424d7..e8e3deceef5 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2550,7 +2550,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ASan) + CHECK_NAME=AST fuzzer (asan) REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse EOF - name: Download json reports @@ -2586,7 +2586,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (TSan) + CHECK_NAME=AST fuzzer (tsan) REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse EOF - name: Download json reports @@ -2622,7 +2622,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (UBSan) + CHECK_NAME=AST fuzzer (ubsan) REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse EOF - name: Download json reports @@ -2658,7 +2658,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (MSan) + CHECK_NAME=AST fuzzer (msan) REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c43369780a6..7901008a7db 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2302,7 +2302,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ASan) + CHECK_NAME=AST fuzzer (asan) REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse EOF - name: Download json reports @@ -2338,7 +2338,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (TSan) + CHECK_NAME=AST fuzzer (tsan) REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse EOF - name: Download json reports @@ -2374,7 +2374,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (UBSan) + CHECK_NAME=AST fuzzer (ubsan) REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse EOF - name: Download json reports @@ -2410,7 +2410,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (MSan) + CHECK_NAME=AST fuzzer (msan) REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index eb934b0a3dd..3d0513bca47 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -301,16 +301,16 @@ CI_CONFIG = { "AST fuzzer (debug)": { "required_build": "package_debug", }, - "AST fuzzer (ASan)": { + "AST fuzzer (asan)": { "required_build": "package_asan", }, - "AST fuzzer (MSan)": { + "AST fuzzer (msan)": { "required_build": "package_msan", }, - "AST fuzzer (TSan)": { + "AST fuzzer (tsan)": { "required_build": "package_tsan", }, - "AST fuzzer (UBSan)": { + "AST fuzzer (ubsan)": { "required_build": "package_ubsan", }, "Stateless tests flaky check (asan)": { From bfa67ffc8c4d9f4e566a87d64331b75747a9e7e0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Sep 2022 10:31:34 +0200 Subject: [PATCH 253/582] Update base/base/itoa.h Co-authored-by: Antonio Andelic --- base/base/itoa.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index 33457f73411..5e0b18d50c0 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -352,7 +352,7 @@ static inline char * writeUIntText(T x, char * p) static_assert(is_unsigned_v); int len = digits10(x); - auto *pp = p + len; + auto * pp = p + len; while (x >= 100) { const auto i = x % 100; From e10acfb1fac34a95c18c83cc011c1d11a6dfd7a9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 08:43:14 +0000 Subject: [PATCH 254/582] Fix stateless tests --- tests/queries/0_stateless/02416_keeper_map.sql | 2 +- tests/queries/0_stateless/02417_keeper_map_create_drop.sql | 2 +- tests/queries/0_stateless/02418_keeper_map_keys_limit.sql | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index 6688f2b4659..d4e6a80e66e 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02416_test SYNC; diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql index fff1d78d589..9d6b67456d9 100644 --- a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02417_test SYNC; diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql index aedeba661dd..59c13d83e60 100644 --- a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02418_test SYNC; From 56e4179cadceaa1ca8d8ed368c392bd74d876b40 Mon Sep 17 00:00:00 2001 From: luocongkai Date: Thu, 1 Sep 2022 17:10:32 +0800 Subject: [PATCH 255/582] fix bug when remove unneeded columns in subquery --- src/Interpreters/TreeRewriter.cpp | 11 ++++++++--- .../02131_remove_columns_in_subquery.reference | 1 + .../0_stateless/02131_remove_columns_in_subquery.sql | 3 ++- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9248e8eecb6..73410a39ffd 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -521,10 +521,15 @@ void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const ++new_elements_size; } /// removing aggregation can change number of rows, so `count()` result in outer sub-query would be wrong - if (func && AggregateUtils::isAggregateFunction(*func) && !select_query->groupBy()) + if (func && !select_query->groupBy()) { - new_elements[result_index] = elem; - ++new_elements_size; + GetAggregatesVisitor::Data data = {}; + GetAggregatesVisitor(data).visit(elem); + if (!data.aggregates.empty()) + { + new_elements[result_index] = elem; + ++new_elements_size; + } } } } diff --git a/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference b/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference index d00491fd7e5..6ed281c757a 100644 --- a/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference +++ b/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference @@ -1 +1,2 @@ 1 +1 diff --git a/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql b/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql index f9ca2269aad..c765d989d1d 100644 --- a/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql +++ b/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql @@ -1 +1,2 @@ -select count(1) from (SELECT 1 AS a, count(1) FROM numbers(5)) +select count(1) from (SELECT 1 AS a, count(1) FROM numbers(5)); +select count(1) from (SELECT 1 AS a, count(1) + 1 FROM numbers(5)); \ No newline at end of file From 018f4118bb11b199f1c5f632034f789b60e1bd16 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 1 Sep 2022 18:20:00 +0800 Subject: [PATCH 256/582] Fix view with merge table function (#40734) --- src/Storages/StorageView.cpp | 6 ++++-- .../0_stateless/02402_merge_engine_with_view.reference | 1 + .../queries/0_stateless/02402_merge_engine_with_view.sql | 8 ++++++-- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 3377af685f0..adaf1c4e404 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -179,12 +179,14 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ if (!table_expression->database_and_table_name) { - // If it's a view table function, add a fake db.table name. + // If it's a view or merge table function, add a fake db.table name. if (table_expression->table_function) { auto table_function_name = table_expression->table_function->as()->name; - if ((table_function_name == "view") || (table_function_name == "viewIfPermitted")) + if (table_function_name == "view" || table_function_name == "viewIfPermitted") table_expression->database_and_table_name = std::make_shared("__view"); + if (table_function_name == "merge") + table_expression->database_and_table_name = std::make_shared("__merge"); } if (!table_expression->database_and_table_name) throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); diff --git a/tests/queries/0_stateless/02402_merge_engine_with_view.reference b/tests/queries/0_stateless/02402_merge_engine_with_view.reference index 4a869286f82..6564c244b87 100644 --- a/tests/queries/0_stateless/02402_merge_engine_with_view.reference +++ b/tests/queries/0_stateless/02402_merge_engine_with_view.reference @@ -4,3 +4,4 @@ 3 4 4 +1 diff --git a/tests/queries/0_stateless/02402_merge_engine_with_view.sql b/tests/queries/0_stateless/02402_merge_engine_with_view.sql index 613f76e24c9..64822784845 100644 --- a/tests/queries/0_stateless/02402_merge_engine_with_view.sql +++ b/tests/queries/0_stateless/02402_merge_engine_with_view.sql @@ -1,4 +1,4 @@ - +-- #40014 CREATE TABLE m0 (id UInt64) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1; INSERT INTO m0 SELECT number FROM numbers(10); CREATE TABLE m1 (id UInt64, s String) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1; @@ -7,4 +7,8 @@ CREATE VIEW m1v AS SELECT id FROM m1; CREATE TABLE m2 (id UInt64) ENGINE=Merge(currentDatabase(),'m0|m1v'); -SELECT * FROM m2 WHERE id > 1 AND id < 5 ORDER BY id SETTINGS force_primary_key=1, max_bytes_to_read=64; \ No newline at end of file +SELECT * FROM m2 WHERE id > 1 AND id < 5 ORDER BY id SETTINGS force_primary_key=1, max_bytes_to_read=64; + +-- #40706 +CREATE VIEW v AS SELECT 1; +SELECT 1 FROM merge(currentDatabase(), '^v$'); \ No newline at end of file From 1da02adc8d013474417a4a9dcc8e10a515f7424b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Sep 2022 12:38:30 +0200 Subject: [PATCH 257/582] Assign PRs to an original merged_by too --- tests/ci/cherry_pick.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index b46dca93026..69dae5d5b70 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -242,7 +242,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise def _assign_new_pr(self, new_pr: PullRequest): # It looks there some race when multiple .add_to_assignees are executed, # so we'll add all at once - assignees = [self.pr.user] + assignees = [self.pr.user, self.pr.merged_by] if self.pr.assignees: assignees.extend(self.pr.assignees) logging.info( From 6442824da872838222703124e7c004f3084ee70d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 10:47:05 +0000 Subject: [PATCH 258/582] Fix KeeperMap with Replicated database --- src/Storages/StorageKeeperMap.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 51886d6da3b..8bbf4ba3f8b 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -233,8 +235,9 @@ StorageKeeperMap::StorageKeeperMap( , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (!table_id.hasUUID()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", table_id.getDatabaseName(), database->getEngineName()); setInMemoryMetadata(metadata); @@ -265,7 +268,13 @@ StorageKeeperMap::StorageKeeperMap( auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; - table_path = fs::path(tables_path) / toString(table_id.uuid); + + auto table_unique_id = toString(table_id.uuid); + if (const auto replicated_database = std::dynamic_pointer_cast(database)) + table_unique_id += replicated_database->getFullReplicaName(); + + table_path = fs::path(tables_path) / table_unique_id; + dropped_path = metadata_path_fs / "dropped"; dropped_lock_path = fs::path(dropped_path) / "lock"; From ca837e6755c53605290db9be2724e1f432016c38 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Sep 2022 13:05:10 +0200 Subject: [PATCH 259/582] Add a doc-string for ReleaseBranch._assign_new_pr --- tests/ci/cherry_pick.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 69dae5d5b70..064a0b3add1 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -240,6 +240,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self._assign_new_pr(self.backport_pr) def _assign_new_pr(self, new_pr: PullRequest): + """Assign `new_pr` to author, merger and assignees of an original PR""" # It looks there some race when multiple .add_to_assignees are executed, # so we'll add all at once assignees = [self.pr.user, self.pr.merged_by] From 9ae95f8c04c9e6f56fba0273bbeb7f169123def4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Sep 2022 13:09:14 +0200 Subject: [PATCH 260/582] Fix unit test --- src/Interpreters/Cache/FileCache.cpp | 3 +-- src/Interpreters/tests/gtest_lru_file_cache.cpp | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 19532534a8c..f0334dffe09 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -87,8 +87,6 @@ void FileCache::initialize() std::lock_guard cache_lock(mutex); if (!is_initialized) { - status_file = make_unique(fs::path(cache_base_path) / "status", StatusFile::write_full_info); - if (fs::exists(cache_base_path)) { try @@ -106,6 +104,7 @@ void FileCache::initialize() fs::create_directories(cache_base_path); } + status_file = make_unique(fs::path(cache_base_path) / "status", StatusFile::write_full_info); is_initialized = true; } } diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index bc71c9773fd..6460eeef8c5 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -89,7 +89,6 @@ TEST(FileCache, get) { if (fs::exists(cache_base_path)) fs::remove_all(cache_base_path); - fs::create_directories(cache_base_path); DB::ThreadStatus thread_status; From ecdea62cd5ab23254701d2a89c42c231d51a91da Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Sep 2022 14:26:50 +0300 Subject: [PATCH 261/582] Update clickhouse-test --- tests/clickhouse-test | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7e060a9f687..14cf4d0674a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -995,9 +995,12 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) try: + drop_database_query = "DROP DATABASE " + database + if args.replicated_database: + drop_database_query += " ON CLUSTER test_cluster_database_replicated" clickhouse_execute( args, - "DROP DATABASE " + database, + drop_database_query, timeout=seconds_left, settings={ "log_comment": args.testcase_basename, From c7b0b87574b36e36d5390ed49c1ed25016e845ee Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 1 Sep 2022 13:29:59 +0200 Subject: [PATCH 262/582] Better exception handling in DiskObjectStorageTransaction (#40813) Co-authored-by: alesapin --- .../DiskObjectStorageTransaction.cpp | 14 +++++++--- src/IO/S3/PocoHTTPClient.cpp | 26 ++++++++++++++++--- 2 files changed, 33 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 54d5a2f2368..6aabf300cb8 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -633,9 +634,11 @@ void DiskObjectStorageTransaction::commit() { operations_to_execute[i]->execute(metadata_transaction); } - catch (Exception & ex) + catch (...) { - ex.addMessage(fmt::format("While executing operation #{} ({})", i, operations_to_execute[i]->getInfoForLog())); + tryLogCurrentException( + &Poco::Logger::get("DiskObjectStorageTransaction"), + fmt::format("An error occurred while executing transaction's operation #{} ({})", i, operations_to_execute[i]->getInfoForLog())); for (int64_t j = i; j >= 0; --j) { @@ -643,9 +646,12 @@ void DiskObjectStorageTransaction::commit() { operations_to_execute[j]->undo(); } - catch (Exception & rollback_ex) + catch (...) { - rollback_ex.addMessage(fmt::format("While undoing operation #{}", i)); + tryLogCurrentException( + &Poco::Logger::get("DiskObjectStorageTransaction"), + fmt::format("An error occurred while undoing transaction's operation #{}", i)); + throw; } } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 089d89cd8a7..34e7a7f6eb4 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -121,9 +121,29 @@ std::shared_ptr PocoHTTPClient::MakeRequest( Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { - auto response = Aws::MakeShared("PocoHTTPClient", request); - makeRequestInternal(*request, response, readLimiter, writeLimiter); - return response; + try + { + auto response = Aws::MakeShared("PocoHTTPClient", request); + makeRequestInternal(*request, response, readLimiter, writeLimiter); + return response; + } + catch (const Exception &) + { + throw; + } + catch (const Poco::Exception & e) + { + throw Exception(Exception::CreateFromPocoTag{}, e); + } + catch (const std::exception & e) + { + throw Exception(Exception::CreateFromSTDTag{}, e); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; + } } namespace From acec516271f8f540048a9ddea56a64752d57004c Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Thu, 1 Sep 2022 19:31:01 +0800 Subject: [PATCH 263/582] add docs --- .../functions/uniqtheta-functions.md | 94 +++++++++++++++++++ .../UniqTheta/FunctionsUniqTheta.cpp | 55 ++++++++++- 2 files changed, 146 insertions(+), 3 deletions(-) create mode 100644 docs/en/sql-reference/functions/uniqtheta-functions.md diff --git a/docs/en/sql-reference/functions/uniqtheta-functions.md b/docs/en/sql-reference/functions/uniqtheta-functions.md new file mode 100644 index 00000000000..b2d3712abfc --- /dev/null +++ b/docs/en/sql-reference/functions/uniqtheta-functions.md @@ -0,0 +1,94 @@ +--- +slug: /en/sql-reference/functions/uniqtheta-functions +--- + +# uniqTheta Functions + +uniqTheta functions work for two uniqThetaSketch objects to do set operation calculations such as ∪ / ∩ / × (union/intersect/not), it is to return a new uniqThetaSketch object contain the result. + +A uniqThetaSketch object is to be constructed by aggregation function uniqTheta with -State. + +UniqThetaSketch is a data structure storage of approximate values set. +For more information on RoaringBitmap, see: [Theta Sketch Framework](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html). + +## uniqThetaUnion + +Two uniqThetaSketch objects to do union calculation(set operation ∪), the result is a new uniqThetaSketch. + +``` sql +uniqThetaUnion(uniqThetaSketch,uniqThetaSketch) +``` + +**Arguments** + +- `uniqThetaSketch` – uniqThetaSketch object. + +**Example** + +``` sql +select finalizeAggregation(uniqThetaUnion(a, b)) as a_union_b, finalizeAggregation(a) as a_cardinality, finalizeAggregation(b) as b_cardinality +from +(select arrayReduce('uniqThetaState',[1,2]) as a, arrayReduce('uniqThetaState',[2,3,4]) as b ); +``` + +``` text +┌─a_union_b─┬─a_cardinality─┬─b_cardinality─┐ +│ 4 │ 2 │ 3 │ +└───────────┴───────────────┴───────────────┘ +``` + +## uniqThetaIntersect + +Two uniqThetaSketch objects to do intersect calculation(set operation ∩), the result is a new uniqThetaSketch. + +``` sql +uniqThetaIntersect(uniqThetaSketch,uniqThetaSketch) +``` + +**Arguments** + +- `uniqThetaSketch` – uniqThetaSketch object. + +**Example** + +``` sql +select finalizeAggregation(uniqThetaIntersect(a, b)) as a_intersect_b, finalizeAggregation(a) as a_cardinality, finalizeAggregation(b) as b_cardinality +from +(select arrayReduce('uniqThetaState',[1,2]) as a, arrayReduce('uniqThetaState',[2,3,4]) as b ); +``` + +``` text +┌─a_intersect_b─┬─a_cardinality─┬─b_cardinality─┐ +│ 1 │ 2 │ 3 │ +└───────────────┴───────────────┴───────────────┘ +``` + +## uniqThetaNot + +Two uniqThetaSketch objects to do a_not_b calculation(set operation ×), the result is a new uniqThetaSketch. + +``` sql +uniqThetaNot(uniqThetaSketch,uniqThetaSketch) +``` + +**Arguments** + +- `uniqThetaSketch` – uniqThetaSketch object. + +**Example** + +``` sql +select finalizeAggregation(uniqThetaNot(a, b)) as a_not_b, finalizeAggregation(a) as a_cardinality, finalizeAggregation(b) as b_cardinality +from +(select arrayReduce('uniqThetaState',[2,3,4]) as a, arrayReduce('uniqThetaState',[1,2]) as b ); +``` + +``` text +┌─a_not_b─┬─a_cardinality─┬─b_cardinality─┐ +│ 2 │ 3 │ 2 │ +└─────────┴───────────────┴───────────────┘ +``` + +**See Also** + +- [uniqThetaSketch](../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch) diff --git a/src/Functions/UniqTheta/FunctionsUniqTheta.cpp b/src/Functions/UniqTheta/FunctionsUniqTheta.cpp index d3925952b00..aa280c0818e 100644 --- a/src/Functions/UniqTheta/FunctionsUniqTheta.cpp +++ b/src/Functions/UniqTheta/FunctionsUniqTheta.cpp @@ -9,9 +9,58 @@ namespace DB REGISTER_FUNCTION(UniqTheta) { - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction( + { + R"( +Two uniqThetaSketch objects to do intersect calculation(set operation ∩), the result is a new uniqThetaSketch. + +A uniqThetaSketch object is to be constructed by aggregation function uniqTheta with -State. + +UniqThetaSketch is a data structure storage of approximate values set. +For more information on RoaringBitmap, see: [Theta Sketch Framework](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html). + +Typical usage: +[example:typical] +)", + Documentation::Examples{ + {"typical", "select finalizeAggregation(uniqThetaIntersect(arrayReduce('uniqThetaState',[1,2]), arrayReduce('uniqThetaState',[2,3,4])));"}}, + Documentation::Categories{"uniqTheta"} + }); + + factory.registerFunction( + { + R"( +Two uniqThetaSketch objects to do union calculation(set operation ∪), the result is a new uniqThetaSketch. + +A uniqThetaSketch object is to be constructed by aggregation function uniqTheta with -State. + +UniqThetaSketch is a data structure storage of approximate values set. +For more information on RoaringBitmap, see: [Theta Sketch Framework](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html). + +Typical usage: +[example:typical] +)", + Documentation::Examples{ + {"typical", "select finalizeAggregation(uniqThetaUnion(arrayReduce('uniqThetaState',[1,2]), arrayReduce('uniqThetaState',[2,3,4])));"}}, + Documentation::Categories{"uniqTheta"} + }); + factory.registerFunction( + { + R"( +Two uniqThetaSketch objects to do a_not_b calculation(set operation ×), the result is a new uniqThetaSketch. + +A uniqThetaSketch object is to be constructed by aggregation function uniqTheta with -State. + +UniqThetaSketch is a data structure storage of approximate values set. +For more information on RoaringBitmap, see: [Theta Sketch Framework](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html). + +Typical usage: +[example:typical] +)", + Documentation::Examples{ + {"typical", "select finalizeAggregation(uniqThetaNot(arrayReduce('uniqThetaState',[1,2]), arrayReduce('uniqThetaState',[2,3,4])));"}}, + Documentation::Categories{"uniqTheta"} + }); } } From e3af5a7a11799d8c3afee28c585707d996abbf40 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 1 Sep 2022 08:31:22 -0300 Subject: [PATCH 264/582] Doc. Added ON CLUSTER cluster in couple places (#40874) --- docs/en/sql-reference/statements/alter/constraint.md | 4 ++-- docs/en/sql-reference/statements/alter/ttl.md | 4 ++-- docs/ru/sql-reference/statements/alter/constraint.md | 4 ++-- docs/ru/sql-reference/statements/alter/ttl.md | 6 +++--- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/constraint.md b/docs/en/sql-reference/statements/alter/constraint.md index 15bd27e1a95..844b24d7374 100644 --- a/docs/en/sql-reference/statements/alter/constraint.md +++ b/docs/en/sql-reference/statements/alter/constraint.md @@ -9,8 +9,8 @@ sidebar_label: CONSTRAINT Constraints could be added or deleted using following syntax: ``` sql -ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression; -ALTER TABLE [db].name DROP CONSTRAINT constraint_name; +ALTER TABLE [db].name [ON CLUSTER cluster] ADD CONSTRAINT constraint_name CHECK expression; +ALTER TABLE [db].name [ON CLUSTER cluster] DROP CONSTRAINT constraint_name; ``` See more on [constraints](../../../sql-reference/statements/create/table.md#constraints). diff --git a/docs/en/sql-reference/statements/alter/ttl.md b/docs/en/sql-reference/statements/alter/ttl.md index 3e9846ba1ab..a312e8cad91 100644 --- a/docs/en/sql-reference/statements/alter/ttl.md +++ b/docs/en/sql-reference/statements/alter/ttl.md @@ -11,7 +11,7 @@ sidebar_label: TTL You can change [table TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) with a request of the following form: ``` sql -ALTER TABLE table_name MODIFY TTL ttl_expression; +ALTER TABLE [db.]table_name [ON CLUSTER cluster] MODIFY TTL ttl_expression; ``` ## REMOVE TTL @@ -19,7 +19,7 @@ ALTER TABLE table_name MODIFY TTL ttl_expression; TTL-property can be removed from table with the following query: ```sql -ALTER TABLE table_name REMOVE TTL +ALTER TABLE [db.]table_name [ON CLUSTER cluster] REMOVE TTL ``` **Example** diff --git a/docs/ru/sql-reference/statements/alter/constraint.md b/docs/ru/sql-reference/statements/alter/constraint.md index cb380bc6a60..bc65b0bbf27 100644 --- a/docs/ru/sql-reference/statements/alter/constraint.md +++ b/docs/ru/sql-reference/statements/alter/constraint.md @@ -11,8 +11,8 @@ sidebar_label: "Манипуляции с ограничениями" Добавить или удалить ограничение можно с помощью запросов ``` sql -ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression; -ALTER TABLE [db].name DROP CONSTRAINT constraint_name; +ALTER TABLE [db].name [ON CLUSTER cluster] ADD CONSTRAINT constraint_name CHECK expression; +ALTER TABLE [db].name [ON CLUSTER cluster] DROP CONSTRAINT constraint_name; ``` Запросы выполняют добавление или удаление метаданных об ограничениях таблицы `[db].name`, поэтому выполняются мгновенно. diff --git a/docs/ru/sql-reference/statements/alter/ttl.md b/docs/ru/sql-reference/statements/alter/ttl.md index 855a651ffc2..2917e96fd34 100644 --- a/docs/ru/sql-reference/statements/alter/ttl.md +++ b/docs/ru/sql-reference/statements/alter/ttl.md @@ -11,7 +11,7 @@ sidebar_label: TTL Вы можете изменить [TTL для таблицы](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl) запросом следующего вида: ``` sql -ALTER TABLE table-name MODIFY TTL ttl-expression +ALTER TABLE [db.]table-name [ON CLUSTER cluster] MODIFY TTL ttl-expression ``` ## REMOVE TTL {#remove-ttl} @@ -19,7 +19,7 @@ ALTER TABLE table-name MODIFY TTL ttl-expression Удалить табличный TTL можно запросом следующего вида: ```sql -ALTER TABLE table_name REMOVE TTL +ALTER TABLE [db.]table_name [ON CLUSTER cluster] REMOVE TTL ``` **Пример** @@ -83,4 +83,4 @@ SELECT * FROM table_with_ttl; ### Смотрите также - Подробнее о [свойстве TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). -- Изменить столбец [с TTL](../../../sql-reference/statements/alter/column.md#alter_modify-column). \ No newline at end of file +- Изменить столбец [с TTL](../../../sql-reference/statements/alter/column.md#alter_modify-column). From 29ac78a92b9da91e39bac8e563596fa7f976a97c Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 1 Sep 2022 07:32:33 -0400 Subject: [PATCH 265/582] Update uk price paid (#40828) --- .../example-datasets/uk-price-paid.md | 122 +++++++++--------- 1 file changed, 60 insertions(+), 62 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index bb9046397a0..765aebb690f 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,28 +6,19 @@ title: "UK Property Price Paid" --- The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 278 MiB in ClickHouse. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 270 MiB in ClickHouse. -Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads +Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads
Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. -## Download the Dataset {#download-dataset} - -Run the command: - -```bash -wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv -``` - -Download will take about 2 minutes with good internet connection. - ## Create the Table {#create-table} ```sql CREATE TABLE uk_price_paid ( + uuid UUID, price UInt32, date Date, postcode1 LowCardinality(String), @@ -42,65 +33,68 @@ CREATE TABLE uk_price_paid town LowCardinality(String), district LowCardinality(String), county LowCardinality(String), - category UInt8 -) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); + category UInt8, + category2 UInt8 +) ORDER BY (postcode1, postcode2, addr1, addr2); ``` ## Preprocess and Import Data {#preprocess-import-data} -We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. - -In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. +In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with either `clickhouse-client` or the web based Play UI. The preprocessing is: -- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; +- splitting the postcode to two different columns `postcode1` and `postcode2` that are better for storage and queries; - coverting the `time` field to date as it only contains 00:00 time; - ignoring the [UUid](../../sql-reference/data-types/uuid.md) field because we don't need it for analysis; - transforming `type` and `duration` to more readable Enum fields with function [transform](../../sql-reference/functions/other-functions.md#transform); - transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to [UInt8](../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) field with 0 and 1. -Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. - ```bash -clickhouse-local --input-format CSV --structure ' - uuid String, - price UInt32, - time DateTime, - postcode String, - a String, - b String, - c String, - addr1 String, - addr2 String, - street String, - locality String, - town String, - district String, - county String, - d String, - e String -' --query " - WITH splitByChar(' ', postcode) AS p - SELECT - price, - toDate(time) AS date, - p[1] AS postcode1, - p[2] AS postcode2, - transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, - b = 'Y' AS is_new, - transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, - addr1, - addr2, - street, - locality, - town, - district, - county, - d = 'B' AS category - FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" +INSERT INTO uk_price_paid +WITH + splitByChar(' ', postcode) AS p +SELECT + replaceRegexpAll(uuid_string, '{|}','') AS uuid, + toUInt32(price_string) AS price, + parseDateTimeBestEffortUS(time) AS date, + p[1] AS postcode1, + p[2] AS postcode2, + transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, + b = 'Y' AS is_new, + transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, + addr1, + addr2, + street, + locality, + town, + district, + county, + d = 'B' AS category, + e = 'B' AS category2 +FROM url( + 'http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv', + 'CSV', + 'uuid_string String, + price_string String, + time String, + postcode String, + a String, + b String, + c String, + addr1 String, + addr2 String, + street String, + locality String, + town String, + district String, + county String, + d String, + e String' +) +SETTINGS max_http_get_redirects=1; ``` -It will take about 40 seconds. +It will take about 2 minutes depending on where you are in the world, and where your ClickHouse servers are. Almost all of the time is the download time of the CSV file from the UK government server. ## Validate the Data {#validate-data} @@ -112,13 +106,13 @@ SELECT count() FROM uk_price_paid; Result: -```text +```response ┌──count()─┐ -│ 26321785 │ +│ 27450499 │ └──────────┘ ``` -The size of dataset in ClickHouse is just 278 MiB, check it. +The size of dataset in ClickHouse is just 540 MiB, check it. Query: @@ -130,10 +124,14 @@ Result: ```text ┌─formatReadableSize(total_bytes)─┐ -│ 278.80 MiB │ +│ 545.04 MiB │ └─────────────────────────────────┘ ``` +:::note +The above size is for a replicated table, if you are using this dataset with a single instance the size will be half. +::: + ## Run Some Queries {#run-queries} ### Query 1. Average Price Per Year {#average-price} @@ -146,7 +144,7 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, Result: -```text +```response ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ From fb42afbbaca54360acc3d17becdc53c241acddf4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 1 Sep 2022 08:59:14 -0300 Subject: [PATCH 266/582] CaresPTRResolver small safety improvement Previous to #40769, only `hostent::h_aliases` was being accessed. After that PR got merged, `hostent::h_name` started being accessed as well. This PR moves the first `hostent::h_aliases != nullptr` check that could prevent `hostent::h_name` from being accessed. During debugging, I observed that even when there are not aliases, `hostent::h_aliases` is not null. That's why it hasn't caused any problems, but proposing this change to be on the safe side. --- src/Common/CaresPTRResolver.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index e5d48b864c8..a02909309b6 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -15,8 +15,8 @@ namespace DB static void callback(void * arg, int status, int, struct hostent * host) { - auto * ptr_records = reinterpret_cast*>(arg); - if (status == ARES_SUCCESS && host->h_aliases) + auto * ptr_records = static_cast*>(arg); + if (ptr_records && status == ARES_SUCCESS) { /* * In some cases (e.g /etc/hosts), hostent::h_name is filled and hostent::h_aliases is empty. @@ -28,11 +28,14 @@ namespace DB ptr_records->insert(ptr_record); } - int i = 0; - while (auto * ptr_record = host->h_aliases[i]) + if (host->h_aliases) { - ptr_records->insert(ptr_record); - i++; + int i = 0; + while (auto * ptr_record = host->h_aliases[i]) + { + ptr_records->insert(ptr_record); + i++; + } } } } From 62ff61404f3e2509dd8399cc28488f813212c33c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 12:16:13 +0000 Subject: [PATCH 267/582] Improve remove recursive --- src/Common/Base64.h | 2 ++ src/Common/ZooKeeper/ZooKeeper.cpp | 17 +++++++++++++---- src/Common/ZooKeeper/ZooKeeper.h | 19 ++++++++++++++++--- src/Interpreters/DDLWorker.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 4 ++-- 5 files changed, 34 insertions(+), 10 deletions(-) diff --git a/src/Common/Base64.h b/src/Common/Base64.h index 39327cd5e08..963d3acb48f 100644 --- a/src/Common/Base64.h +++ b/src/Common/Base64.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace DB { diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b95fa22ddb4..42d215318c9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -720,7 +720,7 @@ void ZooKeeper::removeChildren(const std::string & path) } -void ZooKeeper::removeChildrenRecursive(const std::string & path, const std::string_view keep_child_node) +void ZooKeeper::removeChildrenRecursive(const std::string & path, RemoveException keep_child) { Strings children = getChildren(path); while (!children.empty()) @@ -728,18 +728,23 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const std::str Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - if (likely(keep_child_node.empty() || keep_child_node != children.back())) + if (keep_child.path.empty() || keep_child.path != children.back()) [[likely]] { removeChildrenRecursive(fs::path(path) / children.back()); ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); } + else if (keep_child.remove_subtree) + { + removeChildrenRecursive(fs::path(path) / children.back()); + } + children.pop_back(); } multi(ops); } } -bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const std::string_view keep_child_node) +bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, RemoveException keep_child) { Strings children; if (tryGetChildren(path, children) != Coordination::Error::ZOK) @@ -756,7 +761,7 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); - if (likely(keep_child_node.empty() || keep_child_node != children.back())) + if (keep_child.path.empty() || keep_child.path != children.back()) [[likely]] { /// Will try to avoid recursive getChildren calls if child_path probably has no children. /// It may be extremely slow when path contain a lot of leaf children. @@ -766,6 +771,10 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } + else if (keep_child.remove_subtree && !probably_flat) + { + tryRemoveChildrenRecursive(child_path); + } children.pop_back(); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index d16f1f96271..65bede5abe2 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -58,6 +58,18 @@ struct ShuffleHost } }; +struct RemoveException +{ + explicit RemoveException(std::string_view path_ = "", bool remove_subtree_ = true) + : path(path_) + , remove_subtree(remove_subtree_) + {} + + std::string_view path; + // whether we should keep the child node and its subtree or just the child node + bool remove_subtree; +}; + using GetPriorityForLoadBalancing = DB::GetPriorityForLoadBalancing; /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. @@ -232,12 +244,13 @@ public: void tryRemoveRecursive(const std::string & path); /// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself. - /// If keep_child_node is not empty, this method will not remove path/keep_child_node and its subtree. - void removeChildrenRecursive(const std::string & path, std::string_view keep_child_node = {}); + /// Node defined as RemoveException will not be deleted. + void removeChildrenRecursive(const std::string & path, RemoveException keep_child = RemoveException{}); /// If probably_flat is true, this method will optimistically try to remove children non-recursive /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child. /// Returns true if no kind of fallback happened. - bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, std::string_view keep_child_node = {}); + /// Node defined as RemoveException will not be deleted. + bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, RemoveException keep_child= RemoveException{}); /// Remove all children nodes (non recursive). void removeChildren(const std::string & path); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 6ec20ab5f5f..c8878297c02 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -890,7 +890,7 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper) /// We recursively delete all nodes except node_path/finished to prevent staled hosts from /// creating node_path/active node (see createStatusDirs(...)) - zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, "finished"); + zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, zkutil::RemoveException{"finished"}); /// And then we remove node_path and node_path/finished in a single transaction Coordination::Requests ops; diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8bbf4ba3f8b..85ce8931ea7 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -411,12 +411,12 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { checkTable(); auto client = getClient(); - client->tryRemoveChildrenRecursive(root_path, true, getBaseName(metadata_path)); + client->tryRemoveChildrenRecursive(root_path, true, zkutil::RemoveException{getBaseName(metadata_path), /*remove_subtree_*/ false}); } bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { - zookeeper->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + zookeeper->removeChildrenRecursive(root_path, zkutil::RemoveException{getBaseName(metadata_path), /*remove_subtree_*/ false}); bool completely_removed = false; Coordination::Requests ops; From 36454a340d857f3a983b5cd818642f6f981796fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Sep 2022 17:12:16 +0300 Subject: [PATCH 268/582] Update src/Common/ZooKeeper/ZooKeeperArgs.cpp Co-authored-by: Antonio Andelic --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 1d644bb6a24..fe2f6957490 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -81,7 +81,8 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c { if (chroot.front() != '/') throw KeeperException( - std::string("Root path in config file should start with '/', but got ") + chroot, Coordination::Error::ZBADARGUMENTS); + Coordination::Error::ZBADARGUMENTS, + "Root path in config file should start with '/', but got {}", chroot); if (chroot.back() == '/') chroot.pop_back(); } From 04d431793bfaeb597ac70f7acc4d6316e3fb2b77 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Sep 2022 17:12:22 +0300 Subject: [PATCH 269/582] Update src/Common/ZooKeeper/ZooKeeperImpl.cpp Co-authored-by: Antonio Andelic --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9c323c41eab..d843e4f7896 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -644,7 +644,7 @@ void ZooKeeper::receiveThread() auto prev_bytes_received = in->count(); clock::time_point now = clock::now(); - UInt64 max_wait_us = args.operation_timeout_ms; + UInt64 max_wait_us = args.operation_timeout_ms * 1000; std::optional earliest_operation; { From 3134bdeb13c85d6895dc47fff061d1948374e76f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Sep 2022 16:26:05 +0200 Subject: [PATCH 270/582] Fix --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp | 5 +++-- src/Disks/IO/ReadIndirectBufferFromRemoteFS.h | 3 ++- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 2 +- src/IO/ReadBufferFromS3.cpp | 2 +- 8 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index b5347280497..96ae50bbbcf 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -29,7 +29,7 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( size_t max_single_download_retries_, bool use_external_buffer_, size_t read_until_position_) - : ReadBufferFromFileBase(read_settings_.remote_fs_buffer_size, nullptr, 0) + : ReadBufferFromFileBase(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size, nullptr, 0) , blob_container_client(blob_container_client_) , path(path_) , max_single_read_retries(max_single_read_retries_) diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index 3f7b378dee4..26947af23ec 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -1,6 +1,7 @@ #include "ReadIndirectBufferFromRemoteFS.h" #include +#include namespace DB @@ -13,8 +14,8 @@ namespace ErrorCodes ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( - std::shared_ptr impl_) - : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) + std::shared_ptr impl_, const ReadSettings & settings) + : ReadBufferFromFileBase(settings.remote_fs_buffer_size, nullptr, 0) , impl(impl_) { } diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h index fcd463a92c8..996e69296a6 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h @@ -9,6 +9,7 @@ namespace DB { class ReadBufferFromRemoteFSGather; +struct ReadSettings; /** * Reads data from S3/HDFS/Web using stored paths in metadata. @@ -18,7 +19,7 @@ class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { public: - explicit ReadIndirectBufferFromRemoteFS(std::shared_ptr impl_); + explicit ReadIndirectBufferFromRemoteFS(std::shared_ptr impl_, const ReadSettings & settings); off_t seek(off_t offset_, int whence) override; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 40f68b86e9d..09e5c3d32dc 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -112,7 +112,7 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL } else { - auto buf = std::make_unique(std::move(reader_impl)); + auto buf = std::make_unique(std::move(reader_impl), disk_read_settings); return std::make_unique(std::move(buf), settings_ptr->min_bytes_for_seek); } } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 2303401466d..b3e0e26c27e 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -74,7 +74,7 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI }; auto hdfs_impl = std::make_unique(std::move(read_buffer_creator), objects, disk_read_settings); - auto buf = std::make_unique(std::move(hdfs_impl)); + auto buf = std::make_unique(std::move(hdfs_impl), read_settings); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index ccde7d20778..33f8401ff78 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -157,7 +157,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT } else { - auto buf = std::make_unique(std::move(s3_impl)); + auto buf = std::make_unique(std::move(s3_impl), disk_read_settings); return std::make_unique(std::move(buf), settings_ptr->min_bytes_for_seek); } } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 3c7ce47340d..b0fed4e001b 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -188,7 +188,7 @@ std::unique_ptr WebObjectStorage::readObject( /// NOLINT } else { - auto buf = std::make_unique(std::move(web_impl)); + auto buf = std::make_unique(std::move(web_impl), read_settings); return std::make_unique(std::move(buf), min_bytes_for_seek); } } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 7fb432eab22..f7fd06cac08 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -48,7 +48,7 @@ ReadBufferFromS3::ReadBufferFromS3( size_t offset_, size_t read_until_position_, bool restricted_seek_) - : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) + : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , key(key_) From 9231084c57d18853e188ca278fc54c37dee5ae9c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 14:40:50 +0000 Subject: [PATCH 271/582] Style fix --- src/Interpreters/InterpreterDeleteQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index dcb8a2cb165..51fb6cfb948 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -21,7 +21,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int TABLE_IS_READ_ONLY; extern const int SUPPORT_IS_DISABLED; } From 4fa291cf597cfed2fe595bf64dad7b106b1cd2af Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Sep 2022 16:46:46 +0200 Subject: [PATCH 272/582] Fix hdfs read buffer too --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 3 ++- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 24 +++++++++++++++---- src/Storages/HDFS/ReadBufferFromHDFS.h | 4 +++- 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index b3e0e26c27e..2f82458ecd8 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -70,7 +70,8 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI auto hdfs_path = path.substr(begin_of_path); auto hdfs_uri = path.substr(0, begin_of_path); - return std::make_unique(hdfs_uri, hdfs_path, config, disk_read_settings); + return std::make_unique( + hdfs_uri, hdfs_path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); }; auto hdfs_impl = std::make_unique(std::move(read_buffer_creator), objects, disk_read_settings); diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index fab810a1e49..4aebcd6f6ab 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -41,8 +41,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(read_settings_.remote_fs_buffer_size) + size_t read_until_position_, + bool use_external_buffer_) + : BufferWithOwnMemory(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri_, config_)) @@ -132,10 +133,12 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_, - size_t read_until_position_) + size_t read_until_position_, + bool use_external_buffer_) : ReadBufferFromFileBase(read_settings_.remote_fs_buffer_size, nullptr, 0) , impl(std::make_unique( - hdfs_uri_, hdfs_file_path_, config_, read_settings_, read_until_position_)) + hdfs_uri_, hdfs_file_path_, config_, read_settings_, read_until_position_, use_external_buffer_)) + , use_external_buffer(use_external_buffer_) { } @@ -146,7 +149,18 @@ size_t ReadBufferFromHDFS::getFileSize() bool ReadBufferFromHDFS::nextImpl() { - impl->position() = impl->buffer().begin() + offset(); + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + else + { + impl->position() = impl->buffer().begin() + offset(); + assert(!impl->hasPendingData()); + } + auto result = impl->next(); if (result) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 41493c31882..c3b859f0566 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -29,7 +29,8 @@ public: const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_, - size_t read_until_position_ = 0); + size_t read_until_position_ = 0, + bool use_external_buffer = false); ~ReadBufferFromHDFS() override; @@ -49,6 +50,7 @@ public: private: std::unique_ptr impl; + bool use_external_buffer; }; } From ec55c90f36ae142cfa1ad149d55ca4562e88cf47 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Sep 2022 17:04:34 +0200 Subject: [PATCH 273/582] Fix --- src/Interpreters/Context.cpp | 13 ------------- src/Interpreters/Context.h | 1 - .../MergeTree/MergeTreeDataPartCompact.cpp | 5 +---- src/Storages/MergeTree/MergeTreeIndexReader.cpp | 5 +---- src/Storages/MergeTree/MergeTreeMarksLoader.cpp | 16 ++++++++++++---- src/Storages/MergeTree/MergeTreeMarksLoader.h | 4 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 2 -- src/Storages/MergeTree/MergeTreeReaderCompact.h | 1 - src/Storages/MergeTree/MergeTreeReaderStream.cpp | 6 ++---- src/Storages/MergeTree/MergeTreeReaderStream.h | 3 +-- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 5 +---- 11 files changed, 20 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d39c39cdb15..d6ee5aca1a8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -215,7 +215,6 @@ struct ContextSharedPart std::unique_ptr access_control; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. - mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. @@ -1712,18 +1711,6 @@ void Context::dropMarkCache() const shared->mark_cache->reset(); } -ThreadPool & Context::getLoadMarksThreadpool() const -{ - auto lock = getLock(); - if (!shared->load_marks_threadpool) - { - constexpr size_t pool_size = 50; - constexpr size_t queue_size = 1000000; - shared->load_marks_threadpool = std::make_unique(pool_size, pool_size, queue_size); - } - return *shared->load_marks_threadpool; -} - void Context::setIndexUncompressedCache(size_t max_size_in_bytes) { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 21378dc096c..cc54ce47f92 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -789,7 +789,6 @@ public: void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy); std::shared_ptr getMarkCache() const; void dropMarkCache() const; - ThreadPool & getLoadMarksThreadpool() const; /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 851153cd619..0002a4bb599 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -47,12 +47,9 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { auto ptr = std::static_pointer_cast(shared_from_this()); - auto context = storage.getContext(); - auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; - return std::make_unique( ptr, columns_to_read, metadata_snapshot, uncompressed_cache, - mark_cache, mark_ranges, reader_settings, load_marks_threadpool, + mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 33106f7ab64..2f9ea7c4ede 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -16,9 +16,6 @@ std::unique_ptr makeIndexReader( UncompressedCache * uncompressed_cache, MergeTreeReaderSettings settings) { - auto context = part->storage.getContext(); - auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; - return std::make_unique( part->data_part_storage, index->getFileName(), extension, marks_count, @@ -26,7 +23,7 @@ std::unique_ptr makeIndexReader( std::move(settings), mark_cache, uncompressed_cache, part->getFileSizeOrZero(index->getFileName() + extension), &part->index_granularity_info, - ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false, load_marks_threadpool); + ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false); } } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 7d8ee994ac9..67bc5deba49 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -37,7 +37,6 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( const MergeTreeIndexGranularityInfo & index_granularity_info_, bool save_marks_in_cache_, const ReadSettings & read_settings_, - ThreadPool * load_marks_threadpool_, size_t columns_in_mark_) : data_part_storage(std::move(data_part_storage_)) , mark_cache(mark_cache_) @@ -47,9 +46,8 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( , save_marks_in_cache(save_marks_in_cache_) , columns_in_mark(columns_in_mark_) , read_settings(read_settings_) - , load_marks_threadpool(load_marks_threadpool_) { - if (load_marks_threadpool) + if (read_settings_.load_marks_asynchronously) { future = loadMarksAsync(); } @@ -63,6 +61,16 @@ MergeTreeMarksLoader::~MergeTreeMarksLoader() } } +std::shared_ptr MergeTreeMarksLoader::getLoadMarksThreadpool() +{ + constexpr size_t pool_size = 50; + constexpr size_t queue_size = 1000000; + static std::shared_ptr load_marks_threadpool = std::make_shared(pool_size, pool_size, queue_size); + + return load_marks_threadpool; +} + + const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index) { if (!marks) @@ -193,7 +201,7 @@ std::future MergeTreeMarksLoader::loadMarksAsync() }); auto task_future = task->get_future(); - load_marks_threadpool->scheduleOrThrow([task]{ (*task)(); }); + getLoadMarksThreadpool()->scheduleOrThrow([task]{ (*task)(); }); return task_future; } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 60ccc953e9b..2474710f098 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -23,7 +23,6 @@ public: const MergeTreeIndexGranularityInfo & index_granularity_info_, bool save_marks_in_cache_, const ReadSettings & read_settings_, - ThreadPool * load_marks_threadpool_, size_t columns_in_mark_ = 1); ~MergeTreeMarksLoader(); @@ -31,6 +30,8 @@ public: const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0); private: + static std::shared_ptr getLoadMarksThreadpool(); + DataPartStoragePtr data_part_storage; MarkCache * mark_cache = nullptr; String mrk_path; @@ -46,7 +47,6 @@ private: MarkCache::MappedPtr loadMarksImpl(); std::future future; - ThreadPool * load_marks_threadpool; }; } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 88237091547..44b5fa1a11b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -22,7 +22,6 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( MarkCache * mark_cache_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_, - ThreadPool * load_marks_cache_threadpool_, ValueSizeMap avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) @@ -43,7 +42,6 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( data_part->index_granularity_info, settings.save_marks_in_cache, settings.read_settings, - load_marks_cache_threadpool_, data_part->getColumns().size()) { try diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 348e4802e2a..aa0eb949aa1 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -26,7 +26,6 @@ public: MarkCache * mark_cache_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_, - ThreadPool * load_marks_cache_threadpool_, ValueSizeMap avg_value_size_hints_ = {}, const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {}, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE); diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index 47f8b0f6008..7ee939c3f06 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -27,8 +27,7 @@ MergeTreeReaderStream::MergeTreeReaderStream( const MergeTreeIndexGranularityInfo * index_granularity_info_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_, - bool is_low_cardinality_dictionary_, - ThreadPool * load_marks_cache_threadpool_) + bool is_low_cardinality_dictionary_) : settings(settings_) , profile_callback(profile_callback_) , clock_type(clock_type_) @@ -50,8 +49,7 @@ MergeTreeReaderStream::MergeTreeReaderStream( marks_count, *index_granularity_info, save_marks_in_cache, - settings.read_settings, - load_marks_cache_threadpool_) + settings.read_settings) { } diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.h b/src/Storages/MergeTree/MergeTreeReaderStream.h index 83e314eef42..f480e355294 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.h +++ b/src/Storages/MergeTree/MergeTreeReaderStream.h @@ -31,8 +31,7 @@ public: const MergeTreeIndexGranularityInfo * index_granularity_info_, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type, - bool is_low_cardinality_dictionary_, - ThreadPool * load_marks_cache_threadpool_); + bool is_low_cardinality_dictionary_); void seekToMark(size_t index); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 9ac7b5a5c5a..4f21708fc32 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -178,15 +178,12 @@ void MergeTreeReaderWide::addStreams( bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; - auto context = data_part->storage.getContext(); - auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; - streams.emplace(stream_name, std::make_unique( data_part->data_part_storage, stream_name, DATA_FILE_EXTENSION, data_part->getMarksCount(), all_mark_ranges, settings, mark_cache, uncompressed_cache, data_part->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), &data_part->index_granularity_info, - profile_callback, clock_type, is_lc_dict, load_marks_threadpool)); + profile_callback, clock_type, is_lc_dict)); }; serialization->enumerateStreams(callback); From 94b74d46a6eb9b18732389322bbd439a357d1052 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 1 Sep 2022 17:05:04 +0200 Subject: [PATCH 274/582] Update tests --- .../02293_grouping_function.reference | 22 +++++++++------ .../0_stateless/02293_grouping_function.sql | 22 +++++++++------ ...02293_grouping_function_group_by.reference | 27 ++++++++++++------- .../02293_grouping_function_group_by.sql | 27 ++++++++++++------- 4 files changed, 64 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02293_grouping_function.reference b/tests/queries/0_stateless/02293_grouping_function.reference index e71d6812ab5..7d745a0e0fa 100644 --- a/tests/queries/0_stateless/02293_grouping_function.reference +++ b/tests/queries/0_stateless/02293_grouping_function.reference @@ -8,7 +8,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 1 0 1 0 2 @@ -30,7 +31,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 1 0 2 0 2 @@ -52,7 +54,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 1 0 1 @@ -73,7 +76,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, grouping(number, number % 2) = 1; +ORDER BY number, grouping(number, number % 2) = 1 +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 @@ -97,7 +101,8 @@ GROUP BY (number, number % 2), () ) -ORDER BY (gr, number); +ORDER BY (gr, number) +SETTINGS force_grouping_standard_compatibility=0; 0 10 0 0 1 2 1 1 2 @@ -129,7 +134,7 @@ GROUP BY ) HAVING grouping(number, number % 2) = 2 ORDER BY number -SETTINGS enable_optimize_predicate_expression = 0; +SETTINGS enable_optimize_predicate_expression = 0, force_grouping_standard_compatibility=0; 0 1 2 @@ -150,7 +155,7 @@ GROUP BY ) HAVING grouping(number, number % 2) = 1 ORDER BY number -SETTINGS enable_optimize_predicate_expression = 0; +SETTINGS enable_optimize_predicate_expression = 0, force_grouping_standard_compatibility=0; 0 0 SELECT @@ -161,7 +166,8 @@ GROUP BY GROUPING SETS ( (number), (number % 2)) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 1 0 1 diff --git a/tests/queries/0_stateless/02293_grouping_function.sql b/tests/queries/0_stateless/02293_grouping_function.sql index 169fc09c324..cf076c8e51c 100644 --- a/tests/queries/0_stateless/02293_grouping_function.sql +++ b/tests/queries/0_stateless/02293_grouping_function.sql @@ -19,7 +19,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -30,7 +31,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -41,7 +43,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number @@ -51,7 +54,8 @@ GROUP BY (number), (number % 2) ) -ORDER BY number, grouping(number, number % 2) = 1; +ORDER BY number, grouping(number, number % 2) = 1 +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -64,7 +68,8 @@ GROUP BY (number, number % 2), () ) -ORDER BY (gr, number); +ORDER BY (gr, number) +SETTINGS force_grouping_standard_compatibility=0; SELECT number @@ -76,7 +81,7 @@ GROUP BY ) HAVING grouping(number, number % 2) = 2 ORDER BY number -SETTINGS enable_optimize_predicate_expression = 0; +SETTINGS enable_optimize_predicate_expression = 0, force_grouping_standard_compatibility=0; SELECT number @@ -88,7 +93,7 @@ GROUP BY ) HAVING grouping(number, number % 2) = 1 ORDER BY number -SETTINGS enable_optimize_predicate_expression = 0; +SETTINGS enable_optimize_predicate_expression = 0, force_grouping_standard_compatibility=0; SELECT number, @@ -98,4 +103,5 @@ GROUP BY GROUPING SETS ( (number), (number % 2)) -ORDER BY number, gr; +ORDER BY number, gr +SETTINGS force_grouping_standard_compatibility=0; diff --git a/tests/queries/0_stateless/02293_grouping_function_group_by.reference b/tests/queries/0_stateless/02293_grouping_function_group_by.reference index 7f87aecd4bd..49cdca1411e 100644 --- a/tests/queries/0_stateless/02293_grouping_function_group_by.reference +++ b/tests/queries/0_stateless/02293_grouping_function_group_by.reference @@ -6,7 +6,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY number, number % 2 -ORDER BY number; +ORDER BY number +SETTINGS force_grouping_standard_compatibility=0; 0 1 1 1 2 1 @@ -25,7 +26,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY number, number % 2 -ORDER BY number; +ORDER BY number +SETTINGS force_grouping_standard_compatibility=0; 0 1 1 1 1 1 2 1 1 @@ -45,7 +47,8 @@ GROUP BY number % 2 WITH ROLLUP ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 2 0 3 @@ -74,7 +77,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY ROLLUP(number, number % 2) ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 2 0 3 @@ -105,7 +109,8 @@ GROUP BY number % 2 WITH CUBE ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 1 0 1 @@ -136,7 +141,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY CUBE(number, number % 2) ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 1 0 1 @@ -168,7 +174,8 @@ GROUP BY CUBE(number, number % 2) HAVING grouping(number) != 0 ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 5 0 6 1 5 @@ -205,7 +212,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY CUBE(number, number % 2) WITH TOTALS ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 1 0 1 @@ -247,7 +255,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY ROLLUP(number, number % 2) WITH TOTALS ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; 0 0 0 2 0 3 diff --git a/tests/queries/0_stateless/02293_grouping_function_group_by.sql b/tests/queries/0_stateless/02293_grouping_function_group_by.sql index 9bf9d43478b..d438a8a5277 100644 --- a/tests/queries/0_stateless/02293_grouping_function_group_by.sql +++ b/tests/queries/0_stateless/02293_grouping_function_group_by.sql @@ -15,7 +15,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY number, number % 2 -ORDER BY number; +ORDER BY number +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -25,7 +26,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY number, number % 2 -ORDER BY number; +ORDER BY number +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -36,7 +38,8 @@ GROUP BY number % 2 WITH ROLLUP ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -45,7 +48,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY ROLLUP(number, number % 2) ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -56,7 +60,8 @@ GROUP BY number % 2 WITH CUBE ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -65,7 +70,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY CUBE(number, number % 2) ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -75,7 +81,8 @@ GROUP BY CUBE(number, number % 2) HAVING grouping(number) != 0 ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -94,7 +101,8 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY CUBE(number, number % 2) WITH TOTALS ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; SELECT number, @@ -113,4 +121,5 @@ FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY ROLLUP(number, number % 2) WITH TOTALS ORDER BY - number, gr; + number, gr +SETTINGS force_grouping_standard_compatibility=0; From dbe3b777ec51b886a07dc3202a2579fb6a10bf7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Sep 2022 17:07:56 +0200 Subject: [PATCH 275/582] Fix build --- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index c21ef46753e..ecbd8cc9aa1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB From 304dc30f0f96750f89a9a00eb29efb3db2205209 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Thu, 1 Sep 2022 18:15:56 +0300 Subject: [PATCH 276/582] Remove remaining usage of // Y_IGNORE magic comment --- src/IO/S3Common.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index fb9cff5d109..1ff1c609952 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -24,7 +24,7 @@ # include # include # include -# include // Y_IGNORE +# include # include # include From 1da35a486c7b39edb110ad248b93a6ae3e1a0271 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 1 Sep 2022 23:16:05 +0800 Subject: [PATCH 277/582] Disable query_span for internal queries --- src/Interpreters/executeQuery.cpp | 59 +++++++++++++++++++------------ 1 file changed, 37 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ee7879bbbc7..96124c685a7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -291,11 +291,14 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr if (auto query_log = context->getQueryLog()) query_log->add(elem); - query_span->addAttribute("clickhouse.exception_code", elem.exception_code); - query_span->addAttribute("clickhouse.exception", elem.exception); - query_span->addAttribute("db.statement", elem.query); - query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); - query_span->finish(); + if (query_span) + { + query_span->addAttribute("clickhouse.exception_code", elem.exception_code); + query_span->addAttribute("clickhouse.exception", elem.exception); + query_span->addAttribute("db.statement", elem.query); + query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->finish(); + } ProfileEvents::increment(ProfileEvents::FailedQuery); @@ -345,7 +348,13 @@ static std::tuple executeQueryImpl( QueryProcessingStage::Enum stage, ReadBuffer * istr) { - std::shared_ptr query_span = std::make_shared("query"); + /// query_span is a special span, when this function exits, it's lifetime is not ended, but ends when the query finishes. + /// Some internal queries might call this function recursively by setting 'internal' parameter to 'true', + /// to make sure SpanHolders in current stack ends in correct order, we disable this span for these internal queries + /// + /// This does not have impact on the final span logs, because these internal queries are issued by external queries, + /// we still have enough span logs for the execution of external queries. + std::shared_ptr query_span = internal ? nullptr : std::make_shared("query"); const auto current_time = std::chrono::system_clock::now(); @@ -668,7 +677,7 @@ static std::tuple executeQueryImpl( { std::unique_ptr span; - if (query_span->isTraceEnabled()) + if (OpenTelemetry::CurrentContext().isTraceEnabled()) { auto * raw_interpreter_ptr = interpreter.get(); std::string class_name(demangle(typeid(*raw_interpreter_ptr).name())); @@ -928,16 +937,19 @@ static std::tuple executeQueryImpl( } } - query_span->addAttribute("db.statement", elem.query); - query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); - query_span->addAttribute("clickhouse.query_status", "QueryFinish"); - query_span->addAttributeIfNotEmpty("clickhouse.tracestate", OpenTelemetry::CurrentContext().tracestate); - query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); - query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); - query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); - query_span->addAttributeIfNotZero("clickhouse.written_bytes", elem.written_bytes); - query_span->addAttributeIfNotZero("clickhouse.memory_usage", elem.memory_usage); - query_span->finish(); + if (query_span) + { + query_span->addAttribute("db.statement", elem.query); + query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttribute("clickhouse.query_status", "QueryFinish"); + query_span->addAttributeIfNotEmpty("clickhouse.tracestate", OpenTelemetry::CurrentContext().tracestate); + query_span->addAttributeIfNotZero("clickhouse.read_rows", elem.read_rows); + query_span->addAttributeIfNotZero("clickhouse.read_bytes", elem.read_bytes); + query_span->addAttributeIfNotZero("clickhouse.written_rows", info.written_rows); + query_span->addAttributeIfNotZero("clickhouse.written_bytes", elem.written_bytes); + query_span->addAttributeIfNotZero("clickhouse.memory_usage", elem.memory_usage); + query_span->finish(); + } if (implicit_txn_control) { @@ -1023,11 +1035,14 @@ static std::tuple executeQueryImpl( ProfileEvents::increment(ProfileEvents::FailedInsertQuery); } - query_span->addAttribute("db.statement", elem.query); - query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); - query_span->addAttribute("clickhouse.exception", elem.exception); - query_span->addAttribute("clickhouse.exception_code", elem.exception_code); - query_span->finish(); + if (query_span) + { + query_span->addAttribute("db.statement", elem.query); + query_span->addAttribute("clickhouse.query_id", elem.client_info.current_query_id); + query_span->addAttribute("clickhouse.exception", elem.exception); + query_span->addAttribute("clickhouse.exception_code", elem.exception_code); + query_span->finish(); + } }; res.finish_callback = std::move(finish_callback); From a947487afeb0c0b1d0f843285caf81eb8927e812 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 1 Sep 2022 18:16:13 +0300 Subject: [PATCH 278/582] [docs] fix link markdown --- docs/en/getting-started/example-datasets/cell-towers.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index e74849a76e0..e31ce3de5ce 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -13,7 +13,7 @@ OpenCelliD Project is licensed under a Creative Commons Attribution-ShareAlike 4 ## Get the Dataset {#get-the-dataset} -1. Download the snapshot of the dataset from February 2021: [https://datasets.clickhouse.com/cell_towers.csv.xz] (729 MB). +1. Download the snapshot of the dataset from February 2021: [cell_towers.csv.xz](https://datasets.clickhouse.com/cell_towers.csv.xz) (729 MB). 2. Validate the integrity (optional step): ``` From f71a7e028688097d966cfbc7e29031b38307c7a0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 1 Sep 2022 17:19:29 +0200 Subject: [PATCH 279/582] Add test for compatibility --- ..._grouping_function_compatibility.reference | 29 +++++++++++++++++++ .../02416_grouping_function_compatibility.sql | 14 +++++++++ 2 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/02416_grouping_function_compatibility.reference create mode 100644 tests/queries/0_stateless/02416_grouping_function_compatibility.sql diff --git a/tests/queries/0_stateless/02416_grouping_function_compatibility.reference b/tests/queries/0_stateless/02416_grouping_function_compatibility.reference new file mode 100644 index 00000000000..c9a3ad2f593 --- /dev/null +++ b/tests/queries/0_stateless/02416_grouping_function_compatibility.reference @@ -0,0 +1,29 @@ +-- { echoOn } +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02416 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); +1 0 0 0 +1 0 2 0 +1 0 4 0 +1 0 6 0 +1 0 8 0 +1 1 1 0 +1 1 3 0 +1 1 5 0 +1 1 7 0 +1 1 9 0 +5 0 0 1 +5 1 0 1 +10 0 0 3 +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02416 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); +1 0 0 0 +1 0 2 0 +1 0 4 0 +1 0 6 0 +1 0 8 0 +1 1 1 0 +1 1 3 0 +1 1 5 0 +1 1 7 0 +1 1 9 0 +5 0 0 1 +5 1 0 1 +10 0 0 3 diff --git a/tests/queries/0_stateless/02416_grouping_function_compatibility.sql b/tests/queries/0_stateless/02416_grouping_function_compatibility.sql new file mode 100644 index 00000000000..ed21055ade5 --- /dev/null +++ b/tests/queries/0_stateless/02416_grouping_function_compatibility.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test02416; + +CREATE TABLE test02416(a UInt64, b UInt64) ENGINE=MergeTree() ORDER BY (a, b); + +INSERT INTO test02416 SELECT number % 2 as a, number as b FROM numbers(10); + +-- { echoOn } +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02416 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); + +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02416 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); + +-- { echoOff } +DROP TABLE test02416; + From 3e9225aafa28fb2e150270aa656d8d3068f7f7b7 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Sep 2022 12:24:03 -0400 Subject: [PATCH 280/582] move title to frontmatter, remove orig article link --- docs/en/operations/access-rights.md | 1 - docs/en/operations/quotas.md | 4 +--- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/docs/en/operations/access-rights.md b/docs/en/operations/access-rights.md index 1919aa49ab9..fc39d8b6dc7 100644 --- a/docs/en/operations/access-rights.md +++ b/docs/en/operations/access-rights.md @@ -151,4 +151,3 @@ Management queries: By default, SQL-driven access control and account management is disabled for all users. You need to configure at least one user in the `users.xml` configuration file and set the value of the [access_management](../operations/settings/settings-users.md#access_management-user-setting) setting to 1. -[Original article](https://clickhouse.com/docs/en/operations/access_rights/) diff --git a/docs/en/operations/quotas.md b/docs/en/operations/quotas.md index f35bf44fcd0..05355e615fd 100644 --- a/docs/en/operations/quotas.md +++ b/docs/en/operations/quotas.md @@ -2,10 +2,9 @@ slug: /en/operations/quotas sidebar_position: 51 sidebar_label: Quotas +title: Quotas --- -# Quotas - Quotas allow you to limit resource usage over a period of time or track the use of resources. Quotas are set up in the user config, which is usually ‘users.xml’. @@ -118,4 +117,3 @@ For distributed query processing, the accumulated amounts are stored on the requ When the server is restarted, quotas are reset. -[Original article](https://clickhouse.com/docs/en/operations/quotas/) From bcb6475c4ae70d6597367cd893b9d55da87a7e60 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 1 Sep 2022 18:30:55 +0200 Subject: [PATCH 281/582] add separate s3 profile events for disk s3 --- src/Common/ProfileEvents.cpp | 6 ++ src/Disks/ObjectStorages/S3/diskSettings.cpp | 3 +- src/IO/S3/PocoHTTPClient.cpp | 106 +++++++++++-------- src/IO/S3/PocoHTTPClient.h | 28 ++++- src/IO/S3/tests/gtest_aws_s3_client.cpp | 3 +- src/IO/S3Common.cpp | 9 +- src/IO/S3Common.h | 3 +- src/Storages/StorageS3.cpp | 3 +- 8 files changed, 106 insertions(+), 55 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 5f6de294c51..8f03cd987a3 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -286,6 +286,12 @@ The server successfully detected this situation and will download merged part fr M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ \ + M(DiskS3ReadRequestsCount, "Number of GET and HEAD requests to DiskS3 storage.") \ + M(DiskS3ReadRequestsErrors, "Number of non-throttling errors in GET and HEAD requests to DiskS3 storage.") \ + \ + M(DiskS3WriteRequestsCount, "Number of POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ + M(DiskS3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ + \ M(ReadBufferFromS3Microseconds, "Time spend in reading from S3.") \ M(ReadBufferFromS3Bytes, "Bytes read from S3.") \ M(ReadBufferFromS3RequestsErrors, "Number of exceptions while reading from S3.") \ diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4cdb867e48c..556d52fb481 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -118,7 +118,8 @@ std::unique_ptr getClient(const Poco::Util::AbstractConfigura S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), context->getRemoteHostFilter(), context->getGlobalContext()->getSettingsRef().s3_max_redirects, - context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ true); S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); if (uri.key.back() != '/') diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 089d89cd8a7..5fdfb918788 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -42,6 +42,12 @@ namespace ProfileEvents extern const Event S3WriteRequestsErrors; extern const Event S3WriteRequestsThrottling; extern const Event S3WriteRequestsRedirects; + + extern const Event DiskS3ReadRequestsCount; + extern const Event DiskS3ReadRequestsErrors; + + extern const Event DiskS3WriteRequestsCount; + extern const Event DiskS3WriteRequestsErrors; } namespace CurrentMetrics @@ -62,11 +68,13 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, - bool enable_s3_requests_logging_) + bool enable_s3_requests_logging_, + bool for_disk_s3_) : force_region(force_region_) , remote_host_filter(remote_host_filter_) , s3_max_redirects(s3_max_redirects_) , enable_s3_requests_logging(enable_s3_requests_logging_) + , for_disk_s3(for_disk_s3_) { } @@ -112,6 +120,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config , remote_host_filter(client_configuration.remote_host_filter) , s3_max_redirects(client_configuration.s3_max_redirects) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) + , for_disk_s3(client_configuration.for_disk_s3) , extra_headers(client_configuration.extra_headers) { } @@ -156,6 +165,49 @@ namespace } } +S3MetricKind PocoHTTPClient::getMetricKind(Aws::Http::HttpRequest & request) +{ + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + case Aws::Http::HttpMethod::HTTP_HEAD: + return S3MetricKind::Read; + case Aws::Http::HttpMethod::HTTP_POST: + case Aws::Http::HttpMethod::HTTP_DELETE: + case Aws::Http::HttpMethod::HTTP_PUT: + case Aws::Http::HttpMethod::HTTP_PATCH: + return S3MetricKind::Write; + } + throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED); +} + +void PocoHTTPClient::addMetric(Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount) +{ + constexpr ProfileEvents::Event NotUsed{-1}; + + const ProfileEvents::Event events_map[S3MetricType::EnumSize][S3MetricKind::EnumSize] = { + {ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds}, + {ProfileEvents::S3ReadRequestsCount, ProfileEvents::S3WriteRequestsCount}, + {ProfileEvents::S3ReadRequestsErrors, ProfileEvents::S3WriteRequestsErrors}, + {ProfileEvents::S3ReadRequestsThrottling, ProfileEvents::S3WriteRequestsThrottling}, + {ProfileEvents::S3ReadRequestsRedirects, ProfileEvents::S3WriteRequestsRedirects}, + }; + + const ProfileEvents::Event disk_s3_events_map[S3MetricType::EnumSize][S3MetricKind::EnumSize] = { + {NotUsed, NotUsed}, + {ProfileEvents::DiskS3ReadRequestsCount, ProfileEvents::DiskS3WriteRequestsCount}, + {ProfileEvents::DiskS3ReadRequestsErrors, ProfileEvents::DiskS3WriteRequestsErrors}, + {NotUsed, NotUsed}, + {NotUsed, NotUsed}, + }; + + S3MetricKind kind = getMetricKind(request); + + ProfileEvents::increment(events_map[static_cast(type)][static_cast(kind)], amount); + if (for_disk_s3) + if (ProfileEvents::Event event = disk_s3_events_map[static_cast(type)][static_cast(kind)]; event != NotUsed) + ProfileEvents::increment(event, amount); +} void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, @@ -169,45 +221,7 @@ void PocoHTTPClient::makeRequestInternal( if (enable_s3_requests_logging) LOG_TEST(log, "Make request to: {}", uri); - enum class S3MetricType - { - Microseconds, - Count, - Errors, - Throttling, - Redirects, - - EnumSize, - }; - - auto select_metric = [&request](S3MetricType type) - { - const ProfileEvents::Event events_map[][2] = { - {ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds}, - {ProfileEvents::S3ReadRequestsCount, ProfileEvents::S3WriteRequestsCount}, - {ProfileEvents::S3ReadRequestsErrors, ProfileEvents::S3WriteRequestsErrors}, - {ProfileEvents::S3ReadRequestsThrottling, ProfileEvents::S3WriteRequestsThrottling}, - {ProfileEvents::S3ReadRequestsRedirects, ProfileEvents::S3WriteRequestsRedirects}, - }; - - static_assert((sizeof(events_map) / sizeof(events_map[0])) == static_cast(S3MetricType::EnumSize)); - - switch (request.GetMethod()) - { - case Aws::Http::HttpMethod::HTTP_GET: - case Aws::Http::HttpMethod::HTTP_HEAD: - return events_map[static_cast(type)][0]; // Read - case Aws::Http::HttpMethod::HTTP_POST: - case Aws::Http::HttpMethod::HTTP_DELETE: - case Aws::Http::HttpMethod::HTTP_PUT: - case Aws::Http::HttpMethod::HTTP_PATCH: - return events_map[static_cast(type)][1]; // Write - } - - throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED); - }; - - ProfileEvents::increment(select_metric(S3MetricType::Count)); + addMetric(S3MetricType::Count); CurrentMetrics::Increment metric_increment{CurrentMetrics::S3Requests}; try @@ -314,7 +328,7 @@ void PocoHTTPClient::makeRequestInternal( auto & response_body_stream = session->receiveResponse(poco_response); watch.stop(); - ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds()); + addMetric(S3MetricType::Microseconds, watch.elapsedMicroseconds()); int status_code = static_cast(poco_response.getStatus()); @@ -329,7 +343,7 @@ void PocoHTTPClient::makeRequestInternal( if (enable_s3_requests_logging) LOG_TEST(log, "Redirecting request to new location: {}", location); - ProfileEvents::increment(select_metric(S3MetricType::Redirects)); + addMetric(S3MetricType::Redirects); continue; } @@ -367,7 +381,7 @@ void PocoHTTPClient::makeRequestInternal( LOG_WARNING(log, "Response for request contain tag in body, settings internal server error (500 code)"); response->SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); - ProfileEvents::increment(select_metric(S3MetricType::Errors)); + addMetric(S3MetricType::Errors); if (error_report) error_report(request_configuration); @@ -381,11 +395,11 @@ void PocoHTTPClient::makeRequestInternal( if (status_code == 429 || status_code == 503) { // API throttling - ProfileEvents::increment(select_metric(S3MetricType::Throttling)); + addMetric(S3MetricType::Throttling); } else if (status_code >= 300) { - ProfileEvents::increment(select_metric(S3MetricType::Errors)); + addMetric(S3MetricType::Errors); if (status_code >= 500 && error_report) error_report(request_configuration); } @@ -403,7 +417,7 @@ void PocoHTTPClient::makeRequestInternal( response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(false)); - ProfileEvents::increment(select_metric(S3MetricType::Errors)); + addMetric(S3MetricType::Errors); } } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 5fc8c9acc17..d7a46917f36 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -44,6 +44,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; bool enable_s3_requests_logging; + bool for_disk_s3; HeaderCollection extra_headers; void updateSchemeAndRegion(); @@ -55,7 +56,8 @@ private: const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, - bool enable_s3_requests_logging_ + bool enable_s3_requests_logging_, + bool for_disk_s3_ ); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. @@ -113,18 +115,42 @@ public: Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; private: + void makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + enum class S3MetricType + { + Microseconds, + Count, + Errors, + Throttling, + Redirects, + + EnumSize, + }; + + enum class S3MetricKind + { + Read, + Write, + + EnumSize, + }; + + S3MetricKind getMetricKind(Aws::Http::HttpRequest & request); + void addMetric(Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1); + std::function per_request_configuration; std::function error_report; ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; bool enable_s3_requests_logging; + bool for_disk_s3; const HeaderCollection extra_headers; }; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 44bdf436fae..21d421bb4f6 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -87,7 +87,8 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) region, remote_host_filter, s3_max_redirects, - enable_s3_requests_logging + enable_s3_requests_logging, + /* for_disk_s3 = */ false ); client_configuration.endpointOverride = uri.endpoint; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index fb9cff5d109..68b0d052473 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -543,7 +543,7 @@ public: /// AWS API tries credentials providers one by one. Some of providers (like ProfileConfigFileAWSCredentialsProvider) can be /// quite verbose even if nobody configured them. So we use our provider first and only after it use default providers. { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3); AddProvider(std::make_shared(aws_client_configuration)); } @@ -580,7 +580,7 @@ public: } else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true") { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3); /// See MakeDefaultHttpResourceClientConfiguration(). /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside @@ -700,9 +700,10 @@ namespace S3 const String & force_region, const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, - bool enable_s3_requests_logging) + bool enable_s3_requests_logging, + bool for_disk_s3) { - return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging); + return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging, for_disk_s3); } URI::URI(const Poco::URI & uri_) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 46a09ee8901..e532f70633b 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -45,7 +45,8 @@ public: const String & force_region, const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, - bool enable_s3_requests_logging); + bool enable_s3_requests_logging, + bool for_disk_s3); private: ClientFactory(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 1685de55b6e..627679d6779 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1076,7 +1076,8 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; client_configuration.maxConnections = upd.rw_settings.max_connections; From 66b5cf566c4737452148e1b57b86873594a219d0 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 1 Sep 2022 19:02:04 +0200 Subject: [PATCH 282/582] fix bugs --- src/IO/S3/PocoHTTPClient.cpp | 24 ++++++++++++------------ src/IO/S3/PocoHTTPClient.h | 4 ++-- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 7ed866eb658..9f9d17f0afa 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -185,7 +185,7 @@ namespace } } -S3MetricKind PocoHTTPClient::getMetricKind(Aws::Http::HttpRequest & request) +PocoHTTPClient::S3MetricKind PocoHTTPClient::getMetricKind(const Aws::Http::HttpRequest & request) const { switch (request.GetMethod()) { @@ -201,11 +201,11 @@ S3MetricKind PocoHTTPClient::getMetricKind(Aws::Http::HttpRequest & request) throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED); } -void PocoHTTPClient::addMetric(Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount) +void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount) const { - constexpr ProfileEvents::Event NotUsed{-1}; + constexpr ProfileEvents::Event NotUsed = 0; - const ProfileEvents::Event events_map[S3MetricType::EnumSize][S3MetricKind::EnumSize] = { + const ProfileEvents::Event events_map[static_cast(S3MetricType::EnumSize)][static_cast(S3MetricKind::EnumSize)] = { {ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds}, {ProfileEvents::S3ReadRequestsCount, ProfileEvents::S3WriteRequestsCount}, {ProfileEvents::S3ReadRequestsErrors, ProfileEvents::S3WriteRequestsErrors}, @@ -213,7 +213,7 @@ void PocoHTTPClient::addMetric(Aws::Http::HttpRequest & request, S3MetricType ty {ProfileEvents::S3ReadRequestsRedirects, ProfileEvents::S3WriteRequestsRedirects}, }; - const ProfileEvents::Event disk_s3_events_map[S3MetricType::EnumSize][S3MetricKind::EnumSize] = { + const ProfileEvents::Event disk_s3_events_map[static_cast(S3MetricType::EnumSize)][static_cast(S3MetricKind::EnumSize)] = { {NotUsed, NotUsed}, {ProfileEvents::DiskS3ReadRequestsCount, ProfileEvents::DiskS3WriteRequestsCount}, {ProfileEvents::DiskS3ReadRequestsErrors, ProfileEvents::DiskS3WriteRequestsErrors}, @@ -241,7 +241,7 @@ void PocoHTTPClient::makeRequestInternal( if (enable_s3_requests_logging) LOG_TEST(log, "Make request to: {}", uri); - addMetric(S3MetricType::Count); + addMetric(request, S3MetricType::Count); CurrentMetrics::Increment metric_increment{CurrentMetrics::S3Requests}; try @@ -348,7 +348,7 @@ void PocoHTTPClient::makeRequestInternal( auto & response_body_stream = session->receiveResponse(poco_response); watch.stop(); - addMetric(S3MetricType::Microseconds, watch.elapsedMicroseconds()); + addMetric(request, S3MetricType::Microseconds, watch.elapsedMicroseconds()); int status_code = static_cast(poco_response.getStatus()); @@ -363,7 +363,7 @@ void PocoHTTPClient::makeRequestInternal( if (enable_s3_requests_logging) LOG_TEST(log, "Redirecting request to new location: {}", location); - addMetric(S3MetricType::Redirects); + addMetric(request, S3MetricType::Redirects); continue; } @@ -401,7 +401,7 @@ void PocoHTTPClient::makeRequestInternal( LOG_WARNING(log, "Response for request contain tag in body, settings internal server error (500 code)"); response->SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); - addMetric(S3MetricType::Errors); + addMetric(request, S3MetricType::Errors); if (error_report) error_report(request_configuration); @@ -415,11 +415,11 @@ void PocoHTTPClient::makeRequestInternal( if (status_code == 429 || status_code == 503) { // API throttling - addMetric(S3MetricType::Throttling); + addMetric(request, S3MetricType::Throttling); } else if (status_code >= 300) { - addMetric(S3MetricType::Errors); + addMetric(request, S3MetricType::Errors); if (status_code >= 500 && error_report) error_report(request_configuration); } @@ -437,7 +437,7 @@ void PocoHTTPClient::makeRequestInternal( response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(false)); - addMetric(S3MetricType::Errors); + addMetric(request, S3MetricType::Errors); } } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index d7a46917f36..74d0d0e09ae 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -141,8 +141,8 @@ private: EnumSize, }; - S3MetricKind getMetricKind(Aws::Http::HttpRequest & request); - void addMetric(Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1); + S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request) const; + void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; std::function per_request_configuration; std::function error_report; From 8ba9d8857cbfc270e6920e0d29132909142cfff1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Sep 2022 18:31:34 +0200 Subject: [PATCH 283/582] Add test --- .../02417_load_marks_async.reference | 2 + .../0_stateless/02417_load_marks_async.sh | 51 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/02417_load_marks_async.reference create mode 100755 tests/queries/0_stateless/02417_load_marks_async.sh diff --git a/tests/queries/0_stateless/02417_load_marks_async.reference b/tests/queries/0_stateless/02417_load_marks_async.reference new file mode 100644 index 00000000000..541dab48def --- /dev/null +++ b/tests/queries/0_stateless/02417_load_marks_async.reference @@ -0,0 +1,2 @@ +Ok +Ok diff --git a/tests/queries/0_stateless/02417_load_marks_async.sh b/tests/queries/0_stateless/02417_load_marks_async.sh new file mode 100755 index 00000000000..71d7fdfd173 --- /dev/null +++ b/tests/queries/0_stateless/02417_load_marks_async.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test;" +${CLICKHOUSE_CLIENT} -n -q " +CREATE TABLE test +( +n0 UInt64, +n1 UInt64, +n2 UInt64, +n3 UInt64, +n4 UInt64, +n5 UInt64, +n6 UInt64, +n7 UInt64, +n8 UInt64, +n9 UInt64 +) +ENGINE = MergeTree +ORDER BY n0 SETTINGS min_bytes_for_wide_part = 1;" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)" +${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES test" + +function test +{ + QUERY_ID=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") + + ${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE" + ${CLICKHOUSE_CLIENT} --query_id "${QUERY_ID}" -q "SELECT * FROM test SETTINGS load_marks_asynchronously=$1 FORMAT Null" + ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + + result=$(${CLICKHOUSE_CLIENT} -q "SELECT ProfileEvents['WaitMarksLoadMicroseconds'] FROM system.query_log WHERE query_id = '${QUERY_ID}' AND type = 'QueryFinish' AND current_database = currentDatabase()") + if [[ $result -ne 0 ]]; then + echo 'Ok' + else + echo 'F' + fi + result=$(${CLICKHOUSE_CLIENT} -q "WITH CurrentMetric_BackgroundLoadingMarksTasks as a SELECT count(a) FROM system.metric_log WHERE a > 0") + if [[ $result -ne 0 ]]; then + echo 'Ok' + else + echo 'F' + fi +} + +test 1 From 7a4a65bc36eddabf1931878236fba3861e6c522a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 1 Sep 2022 20:36:08 +0200 Subject: [PATCH 284/582] Make better exception message in schema inference --- src/Processors/Formats/ISchemaReader.cpp | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index c7d8b87ab77..34b4adf3b93 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -120,17 +120,20 @@ NamesAndTypesList IRowSchemaReader::readSchema() for (size_t i = 0; i != data_types.size(); ++i) column_names.push_back("c" + std::to_string(i + 1)); } - /// If column names were set, check that the number of names match the number of types. - else if (column_names.size() != data_types.size()) - throw Exception( - ErrorCodes::INCORRECT_DATA, - "The number of column names {} differs with the number of types {}", column_names.size(), data_types.size()); - - for (size_t i = 0; i != column_names.size(); ++i) + else if (!data_types.empty()) { - auto hint_it = hints.find(column_names[i]); - if (hint_it != hints.end()) - data_types[i] = hint_it->second; + /// If column names were set, check that the number of names match the number of types. + if ((column_names.size() != data_types.size()) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "The number of column names {} differs with the number of types {}", column_names.size(), data_types.size()); + + for (size_t i = 0; i != column_names.size(); ++i) + { + auto hint_it = hints.find(column_names[i]); + if (hint_it != hints.end()) + data_types[i] = hint_it->second; + } } for (rows_read = 1; rows_read < max_rows_to_read; ++rows_read) From e9232fc4e62a9146a809d1bbee2cb220fdf2b709 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 1 Sep 2022 13:23:34 +0000 Subject: [PATCH 285/582] Better --- programs/obfuscator/Obfuscator.cpp | 53 ++++++++++++++---------------- 1 file changed, 24 insertions(+), 29 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 15a99c28890..95bf89b0255 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -1293,49 +1294,43 @@ try /// Create the header block SharedContextHolder shared_context = Context::createShared(); auto context = Context::createGlobal(shared_context.get()); + auto context_const = WithContext(context).getContext(); context->makeGlobalContext(); Block header; + ColumnsDescription schema_columns; + if (structure.empty()) { - ReadBufferIterator read_buffer_iterator = [&]() + ReadBufferIterator read_buffer_iterator = [&](ColumnsDescription &) { - return std::make_unique(STDIN_FILENO); + auto file = std::make_unique(STDIN_FILENO); + + /// stdin must be seekable + auto res = lseek(file->getFD(), 0, SEEK_SET); + if (-1 == res) + throwFromErrno("Input must be seekable file (it will be read twice).", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + + return file; }; - auto context_const = WithContext(context).getContext(); - - auto schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const); - auto schema_columns_info = schema_columns.getOrdinary(); - - for (auto & info : schema_columns_info) - { - ColumnWithTypeAndName column; - column.name = info.name; - column.type = info.type; - column.column = column.type->createColumn(); - header.insert(std::move(column)); - } + schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const); } else { - std::vector structure_vals; - boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on); + schema_columns = parseColumnsListFromString(structure, context_const); + } - if (structure_vals.size() % 2 != 0) - throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR); + auto schema_columns_info = schema_columns.getOrdinary(); - const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); - - for (size_t i = 0, size = structure_vals.size(); i < size; i += 2) - { - ColumnWithTypeAndName column; - column.name = structure_vals[i]; - column.type = data_type_factory.get(structure_vals[i + 1]); - column.column = column.type->createColumn(); - header.insert(std::move(column)); - } + for (auto & info : schema_columns_info) + { + ColumnWithTypeAndName column; + column.name = info.name; + column.type = info.type; + column.column = column.type->createColumn(); + header.insert(std::move(column)); } ReadBufferFromFileDescriptor file_in(STDIN_FILENO); From 29e922f05c51142a09697b8436d9aae2e67a07d1 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Sep 2022 02:22:52 +0200 Subject: [PATCH 286/582] support all profile events for disk s3 --- src/Common/ProfileEvents.cpp | 6 ++++++ src/IO/S3/PocoHTTPClient.cpp | 17 ++++++++++------- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 8f03cd987a3..519fd95a266 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -286,11 +286,17 @@ The server successfully detected this situation and will download merged part fr M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ \ + M(DiskS3ReadMicroseconds, "Time of GET and HEAD requests to DiskS3 storage.") \ M(DiskS3ReadRequestsCount, "Number of GET and HEAD requests to DiskS3 storage.") \ M(DiskS3ReadRequestsErrors, "Number of non-throttling errors in GET and HEAD requests to DiskS3 storage.") \ + M(DiskS3ReadRequestsThrottling, "Number of 429 and 503 errors in GET and HEAD requests to DiskS3 storage.") \ + M(DiskS3ReadRequestsRedirects, "Number of redirects in GET and HEAD requests to DiskS3 storage.") \ \ + M(DiskS3WriteMicroseconds, "Time of POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ M(DiskS3WriteRequestsCount, "Number of POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ M(DiskS3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ + M(DiskS3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ + M(DiskS3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \ \ M(ReadBufferFromS3Microseconds, "Time spend in reading from S3.") \ M(ReadBufferFromS3Bytes, "Bytes read from S3.") \ diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 9f9d17f0afa..fc5617ceff5 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -43,11 +43,17 @@ namespace ProfileEvents extern const Event S3WriteRequestsThrottling; extern const Event S3WriteRequestsRedirects; + extern const Event DiskS3ReadMicroseconds; extern const Event DiskS3ReadRequestsCount; extern const Event DiskS3ReadRequestsErrors; + extern const Event DiskS3ReadRequestsThrottling; + extern const Event DiskS3ReadRequestsRedirects; + extern const Event DiskS3WriteMicroseconds; extern const Event DiskS3WriteRequestsCount; extern const Event DiskS3WriteRequestsErrors; + extern const Event DiskS3WriteRequestsThrottling; + extern const Event DiskS3WriteRequestsRedirects; } namespace CurrentMetrics @@ -203,8 +209,6 @@ PocoHTTPClient::S3MetricKind PocoHTTPClient::getMetricKind(const Aws::Http::Http void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount) const { - constexpr ProfileEvents::Event NotUsed = 0; - const ProfileEvents::Event events_map[static_cast(S3MetricType::EnumSize)][static_cast(S3MetricKind::EnumSize)] = { {ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds}, {ProfileEvents::S3ReadRequestsCount, ProfileEvents::S3WriteRequestsCount}, @@ -214,19 +218,18 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT }; const ProfileEvents::Event disk_s3_events_map[static_cast(S3MetricType::EnumSize)][static_cast(S3MetricKind::EnumSize)] = { - {NotUsed, NotUsed}, + {ProfileEvents::DiskS3ReadMicroseconds, ProfileEvents::DiskS3WriteMicroseconds}, {ProfileEvents::DiskS3ReadRequestsCount, ProfileEvents::DiskS3WriteRequestsCount}, {ProfileEvents::DiskS3ReadRequestsErrors, ProfileEvents::DiskS3WriteRequestsErrors}, - {NotUsed, NotUsed}, - {NotUsed, NotUsed}, + {ProfileEvents::DiskS3ReadRequestsThrottling, ProfileEvents::DiskS3WriteRequestsThrottling}, + {ProfileEvents::DiskS3ReadRequestsRedirects, ProfileEvents::DiskS3WriteRequestsRedirects}, }; S3MetricKind kind = getMetricKind(request); ProfileEvents::increment(events_map[static_cast(type)][static_cast(kind)], amount); if (for_disk_s3) - if (ProfileEvents::Event event = disk_s3_events_map[static_cast(type)][static_cast(kind)]; event != NotUsed) - ProfileEvents::increment(event, amount); + ProfileEvents::increment(disk_s3_events_map[static_cast(type)][static_cast(kind)], amount); } void PocoHTTPClient::makeRequestInternal( From 662eed214f8f9e90cebc4a4273f1290f324b04ea Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Sep 2022 20:28:25 -0400 Subject: [PATCH 287/582] correct statements --- .../example-datasets/uk-price-paid.md | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 765aebb690f..09c726189f2 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,7 +6,6 @@ title: "UK Property Price Paid" --- The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 270 MiB in ClickHouse. Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads
Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data @@ -112,26 +111,6 @@ Result: └──────────┘ ``` -The size of dataset in ClickHouse is just 540 MiB, check it. - -Query: - -```sql -SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid'; -``` - -Result: - -```text -┌─formatReadableSize(total_bytes)─┐ -│ 545.04 MiB │ -└─────────────────────────────────┘ -``` - -:::note -The above size is for a replicated table, if you are using this dataset with a single instance the size will be half. -::: - ## Run Some Queries {#run-queries} ### Query 1. Average Price Per Year {#average-price} From 6efd07cbc8158ea27e3cae5e88cc97a46d6a82aa Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Sep 2022 20:32:06 -0400 Subject: [PATCH 288/582] correct statement --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 09c726189f2..5ba3dc0e38c 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -93,8 +93,6 @@ FROM url( SETTINGS max_http_get_redirects=1; ``` -It will take about 2 minutes depending on where you are in the world, and where your ClickHouse servers are. Almost all of the time is the download time of the CSV file from the UK government server. - ## Validate the Data {#validate-data} Query: From 59dccd6e4903f183c0319db8392017f0d9eb5ac0 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 26 Aug 2022 20:00:17 +0800 Subject: [PATCH 289/582] fix test Signed-off-by: Lloyd-Pottiger --- .../integrations/embedded-rocksdb.md | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 10 ++--- .../integration/test_rocksdb_options/test.py | 37 ++++++++++++++++--- .../02404_rocksdb_read_only.reference | 3 +- .../0_stateless/02404_rocksdb_read_only.sh | 17 ++++----- 5 files changed, 46 insertions(+), 23 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02404_rocksdb_read_only.sh diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 3c9cdf2ec67..64d1a0bf704 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -21,7 +21,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Engine parameters: - `ttl` - time to live for values. TTL is accepted in seconds. If TTL is 0, regular RocksDB instance is used (without TTL). -- `rocksdb_dir` - path to the directory of an existed RocksDB. Open the table with the specified `rocksdb_dir`. +- `rocksdb_dir` - path to the directory of an existed RocksDB or the destination path of the created RocksDB. Open the table with the specified `rocksdb_dir`. - `read_only` - when `read_only` is set to true, read-only mode is used. For storage with TTL, compaction will not be triggered (neither manual nor automatic), so no expired entries are removed. - `primary_key_name` – any column name in the column list. - `primary key` must be specified, it supports only one column in the primary key. The primary key will be serialized in binary as a `rocksdb key`. diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 34f39d8b5ad..0a435e558d2 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -29,6 +29,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -172,18 +173,15 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , primary_key{primary_key_} + , rocksdb_dir(std::move(rocksdb_dir_)) , ttl(ttl_) , read_only(read_only_) { setInMemoryMetadata(metadata_); - if (rocksdb_dir_.empty()) + if (rocksdb_dir.empty()) { rocksdb_dir = context_->getPath() + relative_data_path_; } - else - { - rocksdb_dir = rocksdb_dir_; - } if (!attach) { fs::create_directories(rocksdb_dir); @@ -397,7 +395,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); } - return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl, rocksdb_dir, read_only); + return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); } std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index 2964a6db20d..05713e6ab1b 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -54,10 +54,13 @@ def test_valid_options(start_cluster): DROP TABLE test; """ ) + + +def test_dirctory_missing_after_stop(start_cluster): + # for read_only = false node.query( """ - CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); - INSERT INTO test (key, value) VALUES (0, 'a'); + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only_missing') PRIMARY KEY(key); """ ) node.stop_clickhouse() @@ -65,16 +68,40 @@ def test_valid_options(start_cluster): [ "bash", "-c", - "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only", + "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only_missing", ] ) node.start_clickhouse() result = node.query( - """SELECT * FROM test; + """INSERT INTO test (key, value) VALUES (0, 'a'); + SELECT * FROM test; """ ) assert result.strip() == "0\ta" - result = node.query( + node.query( + """DROP TABLE test; + """ + ) + # for read_only = true + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only_missing', 1) PRIMARY KEY(key); + """ + ) + node.stop_clickhouse() + node.exec_in_container( + [ + "bash", + "-c", + "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only_missing", + ] + ) + node.start_clickhouse() + with pytest.raises(QueryRuntimeException): + node.query("""INSERT INTO test (key, value) VALUES (1, 'b');""") + result = node.query("""SELECT * FROM test;""") + assert result.strip() == "" + node.query( """DROP TABLE test; """ ) diff --git a/tests/queries/0_stateless/02404_rocksdb_read_only.reference b/tests/queries/0_stateless/02404_rocksdb_read_only.reference index c955fd6034f..85caf4188f1 100644 --- a/tests/queries/0_stateless/02404_rocksdb_read_only.reference +++ b/tests/queries/0_stateless/02404_rocksdb_read_only.reference @@ -9,8 +9,7 @@ FAIL OK OK ---- -3 FAIL OK ---- -FAIL \ No newline at end of file +FAIL diff --git a/tests/queries/0_stateless/02404_rocksdb_read_only.sh b/tests/queries/0_stateless/02404_rocksdb_read_only.sh old mode 100644 new mode 100755 index c2e723b26c3..0590ac542ce --- a/tests/queries/0_stateless/02404_rocksdb_read_only.sh +++ b/tests/queries/0_stateless/02404_rocksdb_read_only.sh @@ -1,26 +1,25 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-replicated-database +# Tags: no-ordinary-database, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -ROCKSDB_DIR="/var/lib/clickhouse/store/test_rocksdb_read_only" +ROCKSDB_DIR="$CLICKHOUSE_TMP/test_rocksdb_read_only" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_02404;" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_02404;" && echo 'OK' || echo 'FAIL'; echo "----" $CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; echo "----" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}') PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (0, 'a'), (1, 'b'), (3, 'c');" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}') PRIMARY KEY(key);" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (0, 'a'), (1, 'b'), (3, 'c');" && echo 'OK' || echo 'FAIL'; echo "----" $CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" && echo 'OK' || echo 'FAIL'; echo "----" -$CLICKHOUSE_CLIENT --query="SELECT * FROM test_02404;" 2>&1 | grep -F "rows in set." | awk '{print $1}'; $CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (4, 'd');" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" && echo 'OK' || echo 'FAIL'; echo "----" rm -r ${ROCKSDB_DIR} $CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; From 65b64010a0255836264c589fa516834b38b622e3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 2 Sep 2022 11:56:13 +0800 Subject: [PATCH 290/582] move out-of-bound check into set::areTypesEqual --- src/Interpreters/Set.cpp | 8 +++++++- src/Storages/MergeTree/KeyCondition.cpp | 3 --- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index af05b33c1f6..258731456ee 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -393,7 +393,13 @@ void Set::checkColumnsNumber(size_t num_key_columns) const bool Set::areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const { - return removeNullable(recursiveRemoveLowCardinality(data_types[set_type_idx]))->equals(*removeNullable(recursiveRemoveLowCardinality(other_type))); + /// Out-of-bound access can happen when same set expression built with different columns. + /// Caller may call this method to make sure that the set is indeed the one they want + /// without awaring data_types.size(). + if (set_type_idx >= data_types.size()) + return false; + return removeNullable(recursiveRemoveLowCardinality(data_types[set_type_idx])) + ->equals(*removeNullable(recursiveRemoveLowCardinality(other_type))); } void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7d6101cd795..f5eeb4ed35c 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -298,9 +298,6 @@ public: for (size_t i = 0; i < indexes_mapping.size(); ++i) { - /// Same set expression can be construct from different column(s) with different types - if (indexes_mapping[i].tuple_index >= candidate_set->getDataTypes().size()) - return false; if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) return false; } From dbf67f26d8923a20d5bfd1ff24842a350ef60745 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 14:39:57 +0000 Subject: [PATCH 291/582] Add prefix path --- src/Storages/StorageKeeperMap.cpp | 16 +++++++++++----- src/Storages/StorageKeeperMap.h | 2 +- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 85ce8931ea7..627ae4d4a93 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -235,6 +235,10 @@ StorageKeeperMap::StorageKeeperMap( , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { + std::string path_prefix = context_->getConfigRef().getString("keeper_map_path_prefix", ""); + if (path_prefix.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap is disabled because 'keeper_map_path_prefix' config is not defined"); + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (!table_id.hasUUID()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", table_id.getDatabaseName(), database->getEngineName()); @@ -246,7 +250,6 @@ StorageKeeperMap::StorageKeeperMap( if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); - auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); if (config_keys_limit != 0 && keys_limit > config_keys_limit) { @@ -264,7 +267,9 @@ StorageKeeperMap::StorageKeeperMap( LOG_INFO(log, "Keys limit for {} will be set to {}", getStorageID().getFullTableName(), keys_limit); } - std::filesystem::path root_path_fs{root_path}; + auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1); + root_path = root_path_fs.generic_string(); + auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; @@ -286,7 +291,7 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); - if (root_path != "/" && !client->exists(root_path)) + if (root_path_fs != "/" && !client->exists(root_path)) { if (!create_missing_root_path) { @@ -312,7 +317,7 @@ StorageKeeperMap::StorageKeeperMap( if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "Someone else removed leftovers"); + LOG_INFO(log, "Someone else removed leftover nodes"); } else if (code != Coordination::Error::ZOK) { @@ -648,7 +653,8 @@ StoragePtr create(const StorageFactory::Arguments & args) "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); - auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); + const auto root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); + auto root_path = checkAndGetLiteralArgument(root_path_node, "root_path"); bool create_missing_root_path = true; if (engine_args.size() > 1) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 3e47e5ee9c6..03360dd31dd 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -105,7 +105,7 @@ private: std::optional isTableValid() const; - const std::string root_path; + std::string root_path; std::string primary_key; std::string metadata_path; From c91369ec5a6c59e3098252e6c51251856d7ec4c2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:24:46 +0000 Subject: [PATCH 292/582] Use database in path to support parallel --- tests/queries/0_stateless/02416_keeper_map.sql | 12 ++++++------ .../0_stateless/02417_keeper_map_create_drop.sql | 4 ++-- .../0_stateless/02418_keeper_map_keys_limit.sql | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index d4e6a80e66e..caf44fc0ed6 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -2,13 +2,13 @@ DROP TABLE IF EXISTS 02416_test SYNC; -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } -CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416'); -- { serverError 36 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); DROP TABLE IF EXISTS 02416_test SYNC; -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); INSERT INTO 02416_test SELECT '1_1', number FROM numbers(10000); SELECT COUNT(1) == 1 FROM 02416_test; @@ -21,7 +21,7 @@ SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', ' DROP TABLE IF EXISTS 02416_test SYNC; DROP TABLE IF EXISTS 02416_test_memory; -CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); +CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(k); CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory; INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql index 9d6b67456d9..49340167eaa 100644 --- a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql @@ -2,12 +2,12 @@ DROP TABLE IF EXISTS 02417_test SYNC; -CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key); INSERT INTO 02417_test VALUES (1, 11); SELECT * FROM 02417_test ORDER BY key; SELECT '------'; -CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key); INSERT INTO 02417_test_another VALUES (2, 22); SELECT * FROM 02417_test_another ORDER BY key; SELECT '------'; diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql index 59c13d83e60..de84bde3aaf 100644 --- a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS 02418_test SYNC; -CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 3) PRIMARY KEY(key); +CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 3) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2); SELECT count() FROM 02418_test; @@ -12,7 +12,7 @@ INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3); SELECT count() FROM 02418_test; -CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 4) PRIMARY KEY(key); +CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 4) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test_another VALUES (4, 4.4); From b02bf49939f61ad6fed8006e80a7be6802ecb29c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:34:26 +0000 Subject: [PATCH 293/582] Use path prefix --- .../test_keeper_map/configs/enable_keeper_map.xml | 3 +++ tests/integration/test_keeper_map/test.py | 11 +++++------ 2 files changed, 8 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_keeper_map/configs/enable_keeper_map.xml diff --git a/tests/integration/test_keeper_map/configs/enable_keeper_map.xml b/tests/integration/test_keeper_map/configs/enable_keeper_map.xml new file mode 100644 index 00000000000..b4cbb6a954b --- /dev/null +++ b/tests/integration/test_keeper_map/configs/enable_keeper_map.xml @@ -0,0 +1,3 @@ + + /test_keeper_map + diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 025439f44db..8a4ae701599 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -17,7 +17,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=[], + main_configs=['configs/enable_keeper_map.xml'], with_zookeeper=True, stay_alive=True, ) @@ -34,7 +34,6 @@ def started_cluster(): def get_genuine_zk(): - print("Zoo1", cluster.get_instance_ip("zoo1")) return cluster.get_kazoo_client("zoo1") @@ -58,7 +57,7 @@ def test_create_keeper_map(started_cluster): zk_client = get_genuine_zk() def assert_children_size(expected_size): - assert len(zk_client.get_children("/test1")) == expected_size + assert len(zk_client.get_children("/test_keeper_map/test1")) == expected_size assert_children_size(1) @@ -91,7 +90,6 @@ def create_drop_loop(index, stop_event): if stop_event.is_set(): return - stdout.write(f"Trying with {i} for {index}\n") node.query( f"CREATE TABLE {table_name} (key UInt64, value UInt64) ENGINE = KeeperMap('/test') PRIMARY KEY(key);" ) @@ -127,7 +125,8 @@ def test_create_drop_keeper_map_concurrent(started_cluster): pool.close() client = get_genuine_zk() - assert len(client.get_children("/test")) == 0 + assert len(client.get_children("/test_keeper_map/test")) == 0 + client.stop() def test_keeper_map_without_zk(started_cluster): @@ -162,7 +161,7 @@ def test_keeper_map_without_zk(started_cluster): node.query("SELECT * FROM test_keeper_map") client = get_genuine_zk() - remove_children(client, "/test1") + remove_children(client, "/test_keeper_map/test1") node.restart_clickhouse(60) error = node.query_and_get_error("SELECT * FROM test_keeper_map") assert "Failed to activate table because of invalid metadata in ZooKeeper" in error From 1a48beec30f6947de1d8a3728c83cde2125a43d6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:40:39 +0000 Subject: [PATCH 294/582] Use server UUID --- src/Storages/StorageKeeperMap.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 627ae4d4a93..132e745ac8d 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -6,6 +6,7 @@ #include #include +#include #include @@ -274,10 +275,7 @@ StorageKeeperMap::StorageKeeperMap( metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; - auto table_unique_id = toString(table_id.uuid); - if (const auto replicated_database = std::dynamic_pointer_cast(database)) - table_unique_id += replicated_database->getFullReplicaName(); - + auto table_unique_id = toString(table_id.uuid) + toString(ServerUUID::get()); table_path = fs::path(tables_path) / table_unique_id; dropped_path = metadata_path_fs / "dropped"; From 41b431c8b4d423ad0040963158b2be269d5592ed Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:41:25 +0000 Subject: [PATCH 295/582] Enable keeper_map in tests --- tests/config/config.d/enable_keeper_map.xml | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 tests/config/config.d/enable_keeper_map.xml diff --git a/tests/config/config.d/enable_keeper_map.xml b/tests/config/config.d/enable_keeper_map.xml new file mode 100644 index 00000000000..b4cbb6a954b --- /dev/null +++ b/tests/config/config.d/enable_keeper_map.xml @@ -0,0 +1,3 @@ + + /test_keeper_map + From f5fc3db0dd0da8299d21d591dbe005ea0ed786f7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:42:22 +0000 Subject: [PATCH 296/582] Small fix --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 132e745ac8d..8f6f16a7098 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -289,7 +289,7 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); - if (root_path_fs != "/" && !client->exists(root_path)) + if (root_path != "/" && !client->exists(root_path)) { if (!create_missing_root_path) { From b7eebfc6260a408fb06fb65c67103ce1f4ca1d5c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:47:12 +0000 Subject: [PATCH 297/582] Correctly check if the node is using system path --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 15 +++++++++ src/Common/ZooKeeper/ZooKeeperCommon.h | 9 ++++++ src/Coordination/KeeperSnapshotManager.cpp | 37 ++++------------------ src/Coordination/KeeperStorage.cpp | 8 ++--- 4 files changed, 34 insertions(+), 35 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index b15126f5701..749052cbba3 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -898,4 +898,19 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); } +PathMatchResult matchPath(const std::string_view path, const std::string_view match_to) +{ + using enum PathMatchResult; + + auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), match_to.begin(), match_to.end()); + + if (second_it != match_to.end()) + return NOT_MATCH; + + if (first_it == path.end()) + return EXACT; + + return *first_it == '/' ? IS_CHILD : NOT_MATCH; +} + } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 53fabf651fa..9a9700b500b 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -554,4 +554,13 @@ private: ZooKeeperRequestFactory(); }; +enum class PathMatchResult +{ + NOT_MATCH, + EXACT, + IS_CHILD +}; + +PathMatchResult matchPath(std::string_view path, std::string_view match_to); + } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 1e3f37b617f..fe4050eb685 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -13,8 +13,10 @@ #include #include #include -#include "Coordination/KeeperContext.h" +#include #include +#include + namespace DB { @@ -146,33 +148,6 @@ namespace } } -namespace -{ - -enum class PathMatchResult -{ - NOT_MATCH, - EXACT, - IS_CHILD -}; - -PathMatchResult matchPath(const std::string_view path, const std::string_view match_to) -{ - using enum PathMatchResult; - - auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), match_to.begin(), match_to.end()); - - if (second_it != match_to.end()) - return NOT_MATCH; - - if (first_it == path.end()) - return EXACT; - - return *first_it == '/' ? IS_CHILD : NOT_MATCH; -} - -} - void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context) { writeBinary(static_cast(snapshot.version), out); @@ -217,7 +192,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr const auto & path = it->key; // write only the root system path because of digest - if (matchPath(path.toView(), keeper_system_path) == PathMatchResult::IS_CHILD) + if (Coordination::matchPath(path.toView(), keeper_system_path) == Coordination::PathMatchResult::IS_CHILD) { ++it; continue; @@ -365,8 +340,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial KeeperStorage::Node node{}; readNode(node, in, current_version, storage.acl_map); - using enum PathMatchResult; - auto match_result = matchPath(path, keeper_system_path); + using enum Coordination::PathMatchResult; + auto match_result = Coordination::matchPath(path, keeper_system_path); const std::string error_msg = fmt::format("Cannot read node on path {} from a snapshot because it is used as a system node", path); if (match_result == IS_CHILD) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 397cd2c0c71..1c8bc454c8b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -879,7 +879,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr path_created += seq_num_str.str(); } - if (path_created.starts_with(keeper_system_path)) + if (Coordination::matchPath(path_created, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to create a node inside the internal Keeper path ({}) which is not allowed. Path: {}", keeper_system_path, path_created); @@ -1049,7 +1049,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr std::vector new_deltas; - if (request.path.starts_with(keeper_system_path)) + if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); @@ -1203,7 +1203,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce std::vector new_deltas; - if (request.path.starts_with(keeper_system_path)) + if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); @@ -1472,7 +1472,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr { Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); - if (request.path.starts_with(keeper_system_path)) + if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); From 46f74fcbaffea708f8990b912da7f286e9d89901 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Sep 2022 07:49:09 +0000 Subject: [PATCH 298/582] Automatic style fix --- tests/integration/test_keeper_map/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 8a4ae701599..7dce77dec42 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -17,7 +17,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=['configs/enable_keeper_map.xml'], + main_configs=["configs/enable_keeper_map.xml"], with_zookeeper=True, stay_alive=True, ) From a0544b2a012fbeaeaa491e971ba926f67215fd80 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Sep 2022 07:26:23 +0000 Subject: [PATCH 299/582] test: Add test for '.' and '/' repeat commands in client interactive mode --- .../02417_repeat_input_commands.expect | 81 +++++++++++++++++++ .../02417_repeat_input_commands.reference | 0 2 files changed, 81 insertions(+) create mode 100755 tests/queries/0_stateless/02417_repeat_input_commands.expect create mode 100644 tests/queries/0_stateless/02417_repeat_input_commands.reference diff --git a/tests/queries/0_stateless/02417_repeat_input_commands.expect b/tests/queries/0_stateless/02417_repeat_input_commands.expect new file mode 100755 index 00000000000..119aac68645 --- /dev/null +++ b/tests/queries/0_stateless/02417_repeat_input_commands.expect @@ -0,0 +1,81 @@ +#!/usr/bin/expect -f + +set basedir [file dirname $argv0] +set basename [file tail $argv0] +exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 + +log_user 0 +set timeout 10 +match_max 100000 + +expect_after { + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } +} + +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +expect ":) " + +# ----------------------------------------- +# test . and / commands prior to the first query + +send -- ".\r" +expect "Empty query" +expect ":) " + +send -- "/\r" +expect "Empty query" +expect ":) " + +# ----------------------------------------- +# test . and / commands after first query + +send -- "SELECT 123\r" +expect "│ 123 │" +expect "1 row in set." +expect ":) " + +send -- ".\r" +expect "│ 123 │" +expect "1 row in set." +expect ":) " + +# test input of . more than once in a row +send -- ".\r" +expect "│ 123 │" +expect "1 row in set." +expect ":) " + +send -- "/\r" +expect "│ 123 │" +expect "1 row in set." +expect ":) " + +# test input of / more than once in a row +send -- "/\r" +expect "│ 123 │" +expect "1 row in set." +expect ":) " + +# ----------------------------------------- +# test . and / commands after another query + +send -- "SELECT 321\r" +expect "│ 321 │" +expect "1 row in set." +expect ":) " + +send -- ".\r" +expect "│ 321 │" +expect "1 row in set." +expect ":) " + +send -- "/\r" +expect "│ 321 │" +expect "1 row in set." +expect ":) " + +send -- "quit\r" +expect eof diff --git a/tests/queries/0_stateless/02417_repeat_input_commands.reference b/tests/queries/0_stateless/02417_repeat_input_commands.reference new file mode 100644 index 00000000000..e69de29bb2d From ad2196155c9dd21958dac0cc355c96be494bef2e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 08:14:06 +0000 Subject: [PATCH 300/582] Add test for system node modification --- src/Coordination/tests/gtest_coordination.cpp | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 493e76ee5fc..cdda6fc1f32 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2141,6 +2141,38 @@ TEST_P(CoordinationTest, TestCurrentApiVersion) EXPECT_EQ(keeper_version, static_cast(current_keeper_api_version)); } +TEST_P(CoordinationTest, TestSystemNodeModify) +{ + using namespace Coordination; + int64_t zxid{0}; + + // On INIT we abort when a system path is modified + keeper_context->server_state = KeeperContext::Phase::RUNNING; + KeeperStorage storage{500, "", keeper_context}; + const auto assert_create = [&](const std::string_view path, const auto expected_code) + { + auto request = std::make_shared(); + request->path = path; + storage.preprocessRequest(request, 0, 0, zxid); + auto responses = storage.processRequest(request, 0, zxid); + ASSERT_FALSE(responses.empty()); + + const auto & response = responses[0]; + ASSERT_EQ(response.response->error, expected_code) << "Unexpected error for path " << path; + + ++zxid; + }; + + assert_create("/keeper", Error::ZBADARGUMENTS); + assert_create("/keeper/with_child", Error::ZBADARGUMENTS); + assert_create(DB::keeper_api_version_path, Error::ZBADARGUMENTS); + + assert_create("/keeper_map", Error::ZOK); + assert_create("/keeper1", Error::ZOK); + assert_create("/keepe", Error::ZOK); + assert_create("/keeper1/test", Error::ZOK); +} + INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, CoordinationTest, ::testing::ValuesIn(std::initializer_list{ From e64436fef3065b0a55477f7e4d1eeab10f75417d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 08:54:48 +0000 Subject: [PATCH 301/582] Fix typos with new codespell --- base/base/find_symbols.h | 2 +- base/glibc-compatibility/memcpy/memcpy.h | 2 +- base/pcg-random/pcg_random.hpp | 2 +- base/pcg-random/pcg_uint128.hpp | 2 +- programs/client/Client.cpp | 2 +- programs/git-import/git-import.cpp | 2 +- src/Client/Connection.cpp | 2 +- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Common/IntervalKind.h | 2 +- src/Common/SLRUCachePolicy.h | 2 +- src/Common/Stopwatch.h | 2 +- src/Common/Volnitsky.h | 2 +- src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp | 2 +- src/Coordination/KeeperSnapshotManager.h | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/SessionExpiryQueue.h | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Core/SettingsEnums.h | 2 +- src/Dictionaries/HierarchyDictionariesUtils.h | 2 +- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/getDictionaryConfigurationFromAST.cpp | 2 +- src/Disks/ObjectStorages/IMetadataStorage.h | 2 +- src/Functions/FunctionHelpers.h | 2 +- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Parsers/InsertQuerySettingsPushDownVisitor.cpp | 2 +- src/Parsers/QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Processors/Sources/SQLiteSource.cpp | 2 +- src/Processors/Transforms/MergeJoinTransform.cpp | 2 +- src/Storages/AlterCommands.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/MergeTree/MergeTreeBackgroundExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/PartitionCommands.h | 2 +- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/System/StorageSystemDistributionQueue.cpp | 2 +- utils/check-style/check-typos | 2 +- utils/check-style/codespell-ignore-words.list | 5 +++++ 42 files changed, 46 insertions(+), 41 deletions(-) diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index b28749afda6..83f53773ae7 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -15,7 +15,7 @@ * * Allow to search for next character from the set of 'symbols...' in a string. * It is similar to 'strpbrk', 'strcspn' (and 'strchr', 'memchr' in the case of one symbol and '\0'), - * but with the following differencies: + * but with the following differences: * - works with any memory ranges, including containing zero bytes; * - doesn't require terminating zero byte: end of memory range is passed explicitly; * - if not found, returns pointer to end instead of nullptr; diff --git a/base/glibc-compatibility/memcpy/memcpy.h b/base/glibc-compatibility/memcpy/memcpy.h index 9bee26a3722..ff27c970bac 100644 --- a/base/glibc-compatibility/memcpy/memcpy.h +++ b/base/glibc-compatibility/memcpy/memcpy.h @@ -63,7 +63,7 @@ * Very large size of memcpy typically indicates suboptimal (not cache friendly) algorithms in code or unrealistic scenarios, * so we don't pay attention to using non-temporary stores. * - * On recent Intel CPUs, the presence of "erms" makes "rep movsb" the most benefitial, + * On recent Intel CPUs, the presence of "erms" makes "rep movsb" the most beneficial, * even comparing to non-temporary aligned unrolled stores even with the most wide registers. * * memcpy can be written in asm, C or C++. The latter can also use inline asm. diff --git a/base/pcg-random/pcg_random.hpp b/base/pcg-random/pcg_random.hpp index 94e43e1007b..db7c3d7f66c 100644 --- a/base/pcg-random/pcg_random.hpp +++ b/base/pcg-random/pcg_random.hpp @@ -101,7 +101,7 @@ #endif /* - * The pcg_extras namespace contains some support code that is likley to + * The pcg_extras namespace contains some support code that is likely to * be useful for a variety of RNGs, including: * - 128-bit int support for platforms where it isn't available natively * - bit twiddling operations diff --git a/base/pcg-random/pcg_uint128.hpp b/base/pcg-random/pcg_uint128.hpp index 1a1f61b9366..3452ba6f1c4 100644 --- a/base/pcg-random/pcg_uint128.hpp +++ b/base/pcg-random/pcg_uint128.hpp @@ -22,7 +22,7 @@ /* * This code provides a a C++ class that can provide 128-bit (or higher) * integers. To produce 2K-bit integers, it uses two K-bit integers, - * placed in a union that allowes the code to also see them as four K/2 bit + * placed in a union that allows the code to also see them as four K/2 bit * integers (and access them either directly name, or by index). * * It may seem like we're reinventing the wheel here, because several diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e27845de184..6506c23428a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -723,7 +723,7 @@ bool Client::processWithFuzzing(const String & full_query) // queries, for lack of a better solution. // There is also a problem that fuzzer substitutes positive Int64 // literals or Decimal literals, which are then parsed back as - // UInt64, and suddenly duplicate alias substitition starts or stops + // UInt64, and suddenly duplicate alias substitution starts or stops // working (ASTWithAlias::formatImpl) or something like that. // So we compare not even the first and second formatting of the // query, but second and third. diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index 54f5d7f75ea..030ddd263fa 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -67,7 +67,7 @@ Run this tool inside your git repository. It will create .tsv files that can be The tool can process large enough repositories in a reasonable time. It has been tested on: - ClickHouse: 31 seconds; 3 million rows; -- LLVM: 8 minues; 62 million rows; +- LLVM: 8 minutes; 62 million rows; - Linux - 12 minutes; 85 million rows; - Chromium - 67 minutes; 343 million rows; (the numbers as of Sep 2020) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 30f5af5a540..7a663195655 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -557,7 +557,7 @@ void Connection::sendQuery( /// Send correct hash only for !INITIAL_QUERY, due to: /// - this will avoid extra protocol complexity for simplest cases /// - there is no need in hash for the INITIAL_QUERY anyway - /// (since there is no secure/unsecure changes) + /// (since there is no secure/non-secure changes) if (client_info && !cluster_secret.empty() && client_info->query_kind != ClientInfo::QueryKind::INITIAL_QUERY) { #if USE_SSL diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index a1b816deecb..81067f51d29 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -41,7 +41,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( HedgedConnectionsFactory::~HedgedConnectionsFactory() { /// Stop anything that maybe in progress, - /// to avoid interfer with the subsequent connections. + /// to avoid interference with the subsequent connections. /// /// I.e. some replcas may be in the establishing state, /// this means that hedged connection is waiting for TablesStatusResponse, diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index 59e8d32f3e7..b46805655b1 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -64,7 +64,7 @@ struct IntervalKind const char * toNameOfFunctionExtractTimePart() const; /// Converts the string representation of an interval kind to its IntervalKind equivalent. - /// Returns false if the conversion unsucceeded. + /// Returns false if the conversion did not succeed. /// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`. static bool tryParseString(const std::string & kind, IntervalKind::Kind & result); }; diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 10b043ebaca..8d4709c66a7 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -33,7 +33,7 @@ public: * max_protected_size shows how many of the most frequently used entries will not be evicted after a sequential scan. * max_protected_size == 0 means that the default protected size is equal to half of the total max size. */ - /// TODO: construct from special struct with cache policy parametrs (also with max_protected_size). + /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). SLRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) : max_protected_size(max_size_ * std::min(1.0, size_ratio)) , max_size(max_size_) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 2b60bbde196..9e48c33f70d 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -31,7 +31,7 @@ inline UInt64 clock_gettime_ns_adjusted(UInt64 prev_time, clockid_t clock_type = } /** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday', - * returns nanoseconds instead of microseconds, and also by other minor differencies. + * returns nanoseconds instead of microseconds, and also by other minor differences. */ class Stopwatch { diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index d7ca7d35277..6f5948b6564 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -497,7 +497,7 @@ private: /// last index of offsets that was not processed size_t last; - /// limit for adding to hashtable. In worst case with case insentive search, the table will be filled at most as half + /// limit for adding to hashtable. In worst case with case insensitive search, the table will be filled at most as half static constexpr size_t small_limit = VolnitskyTraits::hash_size / 8; public: diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp index 9e4d4a3241f..3e3d0e164fe 100644 --- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -58,7 +58,7 @@ Fuzzing data consists of: else: read_key() if (7): - read_nonce (simillar to read_key) + read_nonce (similar to read_key) if (8): set current_key diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 4984e54f15f..c00ce9421e7 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -27,7 +27,7 @@ enum SnapshotVersion : uint8_t static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5; -/// What is stored in binary shapsnot +/// What is stored in binary snapshot struct SnapshotDeserializationResult { /// Storage diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 397cd2c0c71..711f3874868 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2192,7 +2192,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } catch (...) { - LOG_FATAL(&Poco::Logger::get("KeeperStorage"), "Failed to rollback log. Terminating to avoid incosistencies"); + LOG_FATAL(&Poco::Logger::get("KeeperStorage"), "Failed to rollback log. Terminating to avoid inconsistencies"); std::terminate(); } } diff --git a/src/Coordination/SessionExpiryQueue.h b/src/Coordination/SessionExpiryQueue.h index 8581800834d..862ec35e2f6 100644 --- a/src/Coordination/SessionExpiryQueue.h +++ b/src/Coordination/SessionExpiryQueue.h @@ -53,7 +53,7 @@ public: /// Session was actually removed bool remove(int64_t session_id); - /// Update session expiry time (must be called on hearbeats) + /// Update session expiry time (must be called on heartbeats) void addNewSessionOrUpdate(int64_t session_id, int64_t timeout_ms); /// Get all expired sessions diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 493e76ee5fc..ee7d0b8ec83 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1339,7 +1339,7 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint nuraft::async_result::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr &/*exception*/) { snapshot_created = ret; - std::cerr << "Snapshot finised\n"; + std::cerr << "Snapshot finished\n"; }; state_machine->create_snapshot(s, when_done); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2a987c840cd..baaeaccdae9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -149,7 +149,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard; if set to 1 - SELECT is executed on each shard; if set to 2 - SELECT and INSERT are executed on each shard", 0) \ M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ - M(UInt64, distributed_push_down_limit, 1, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ + M(UInt64, distributed_push_down_limit, 1, "If 1, LIMIT will be applied on each shard separately. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, true, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8d0e69f4b29..be2def2c01a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -89,7 +89,7 @@ static std::map sett {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature bu default"}}}, + {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 518c626fcc0..b5e908defc7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -153,7 +153,7 @@ enum class HandleKafkaErrorMode { DEFAULT = 0, // Ignore errors with threshold. STREAM, // Put errors to stream in the virtual column named ``_error. - /*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likey system.kafka_errors. This is not implemented now. */ + /*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likely system.kafka_errors. This is not implemented now. */ /*CUSTOM_SYSTEM_TABLE, Put errors to in a custom system table. This is not implemented now. */ }; diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index 621290f40f9..c7508ddd220 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -180,7 +180,7 @@ namespace detail /** Returns array with UInt8 represent if key from in_keys array is in hierarchy of key from keys column. * If value in result array is 1 that means key from in_keys array is in hierarchy of key from - * keys array with same index, 0 therwise. + * keys array with same index, 0 otherwise. * For getting hierarchy implementation uses getKeysHierarchy function. * * Not: keys size must be equal to in_keys_size. diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 1ede0ec5045..a735f426ec7 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -118,7 +118,7 @@ MongoDBDictionarySource::MongoDBDictionarySource( Poco::URI poco_uri(uri); // Parse database from URI. This is required for correctness -- the - // cursor is created using database name and colleciton name, so we have + // cursor is created using database name and collection name, so we have // to specify them properly. db = poco_uri.getPath(); // getPath() may return a leading slash, remove it. diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index edc3c34fe81..7d8253c47ce 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -244,7 +244,7 @@ void buildAttributeExpressionIfNeeded( root->appendChild(expression_element); } -/** Transofrms single dictionary attribute to configuration +/** Transforms single dictionary attribute to configuration * third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77 * to * diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 300d8ec59b9..3d6c772157d 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -124,7 +124,7 @@ public: virtual ~IMetadataStorage() = default; - /// ==== More specefic methods. Previous were almost general purpose. ==== + /// ==== More specific methods. Previous were almost general purpose. ==== /// Read multiple metadata files into strings and return mapping from file_path -> metadata virtual std::unordered_map getSerializedMetadata(const std::vector & file_paths) const = 0; diff --git a/src/Functions/FunctionHelpers.h b/src/Functions/FunctionHelpers.h index 8d33c820185..18a4e584080 100644 --- a/src/Functions/FunctionHelpers.h +++ b/src/Functions/FunctionHelpers.h @@ -134,7 +134,7 @@ using FunctionArgumentDescriptors = std::vector; * (e.g. depending on result type or other trait). * First, checks that number of arguments is as expected (including optional arguments). * Second, checks that mandatory args present and have valid type. - * Third, checks optional arguents types, skipping ones that are missing. + * Third, checks optional arguments types, skipping ones that are missing. * * Please note that if you have several optional arguments, like f([a, b, c]), * only these calls are considered valid: diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 07c7cd85d1b..eaf59731967 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -453,7 +453,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context return; /// Do not apply optimization for Distributed and Merge storages, - /// because we can't get the sorting key of their undelying tables + /// because we can't get the sorting key of their underlying tables /// and we can break the matching of the sorting key for `read_in_order` /// optimization by removing monotonous functions from the prefix of key. if (result.is_remote_storage || (result.storage && result.storage->getName() == "Merge")) diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp index a3bca76816f..1cebdfde957 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp @@ -55,7 +55,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS insert_settings.push_back(setting); else { - /// Do not ovewrite setting that was passed for INSERT + /// Do not overwrite setting that was passed for INSERT /// by settings that was passed for SELECT } } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index 2a7ed0125fa..fde8a07b555 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -11,7 +11,7 @@ struct SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: -/// (since settings after FORMAT parsed separatelly not in the ParserSelectQuery but in ParserQueryWithOutput) +/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput) /// /// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 -> /// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1 diff --git a/src/Processors/Sources/SQLiteSource.cpp b/src/Processors/Sources/SQLiteSource.cpp index 814480b63e3..60d39966659 100644 --- a/src/Processors/Sources/SQLiteSource.cpp +++ b/src/Processors/Sources/SQLiteSource.cpp @@ -39,7 +39,7 @@ SQLiteSource::SQLiteSource( if (status != SQLITE_OK) throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Cannot prepate sqlite statement. Status: {}. Message: {}", + "Cannot prepare sqlite statement. Status: {}. Message: {}", status, sqlite3_errstr(status)); compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 6f842bec939..315fc4810ba 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -513,7 +513,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) Columns lcols; if (!left_to_right_key_remap.empty()) { - /// If we have remapped columns, then we need to get values from right columns insead of defaults + /// If we have remapped columns, then we need to get values from right columns instead of defaults const auto & indices = idx_map[0]; const auto & left_src = cursors[0]->getCurrent().getColumns(); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7f99abf31fc..d370a67bfcc 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1274,7 +1274,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception{"Table doesn't have SAMPLE BY, cannot remove", ErrorCodes::BAD_ARGUMENTS}; } - /// Collect default expressions for MODIFY and ADD comands + /// Collect default expressions for MODIFY and ADD commands if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN) { if (command.default_expression) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index b51457ba5d5..96987b0f60e 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -810,7 +810,7 @@ void registerStorageKafka(StorageFactory & factory) /** Arguments of engine is following: * - Kafka broker list * - List of topics - * - Group ID (may be a constaint expression with a string result) + * - Group ID (may be a constraint expression with a string result) * - Message format (string) * - Row delimiter * - Schema (optional, if the format supports it) diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 6a079aa832f..b37dec6cbf1 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -382,7 +382,7 @@ bool StorageLiveView::getNewBlocks() BlocksMetadataPtr new_blocks_metadata = std::make_shared(); /// can't set mergeable_blocks here or anywhere else outside the writeIntoLiveView function - /// as there could be a race codition when the new block has been inserted into + /// as there could be a race condition when the new block has been inserted into /// the source table by the PushingToViews chain and this method /// called before writeIntoLiveView function is called which can lead to /// the same block added twice to the mergeable_blocks leading to diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index a2f97d6bbc6..0fc888dd6ad 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -155,7 +155,7 @@ private: * We use boost::circular_buffer as a container for queues not to do any allocations. * * Another nuisance that we faces with is than background operations always interact with an associated Storage. - * So, when a Storage want to shutdown, it must wait until all its background operaions are finished. + * So, when a Storage want to shutdown, it must wait until all its background operations are finished. */ template class MergeTreeBackgroundExecutor final : boost::noncopyable diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 71ef6edd7fb..3080e285ac8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -16,7 +16,7 @@ struct MergeTreeDataPartTTLInfo time_t max = 0; /// This TTL was computed on completely expired part. It doesn't make sense - /// to select such parts for TTL again. But make sense to recalcuate TTL + /// to select such parts for TTL again. But make sense to recalculate TTL /// again for merge with multiple parts. std::optional ttl_finished; bool finished() const { return ttl_finished.value_or(false); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index e8362e5cc6b..6d1a3efb01d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -279,7 +279,7 @@ private: /// Very large queue entries may appear occasionally. /// We cannot process MAX_MULTI_OPS at once because it will fail. /// But we have to process more than one entry at once because otherwise lagged replicas keep up slowly. - /// Let's start with one entry per transaction and icrease it exponentially towards MAX_MULTI_OPS. + /// Let's start with one entry per transaction and increase it exponentially towards MAX_MULTI_OPS. /// It will allow to make some progress before failing and remain operational even in extreme cases. size_t current_multi_batch_size = 1; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 9807c90bc23..4921cf8e53b 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -104,7 +104,7 @@ struct PartitionCommandResultInfo using PartitionCommandsResultInfo = std::vector; -/// Convert partition comands result to Source from single Chunk, which will be +/// Convert partition commands result to Source from single Chunk, which will be /// used to print info to the user. Tries to create narrowest table for given /// results. For example, if all commands were FREEZE commands, than /// old_part_name column will be absent. diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 91bf5eeccde..37caa66aae5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -126,7 +126,7 @@ private: static void assertCorrectInsertion(StorageData::Buffer & buffer, size_t column_idx); - /// lsn - log sequnce nuumber, like wal offset (64 bit). + /// lsn - log sequence number, like wal offset (64 bit). static Int64 getLSNValue(const std::string & lsn) { UInt32 upper_half, lower_half; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 507047751f3..5ce46ecffc8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -963,7 +963,7 @@ bool StorageMergeTree::merge( if (!merge_mutate_entry) return false; - /// Copying a vector of columns `deduplicate bu columns. + /// Copying a vector of columns `deduplicate by columns. IExecutableTask::TaskResultCallback f = [](bool) {}; auto task = std::make_shared( *this, metadata_snapshot, deduplicate, deduplicate_by_columns, merge_mutate_entry, table_lock_holder, f); diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index d8879c3655e..5297c4eb93c 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -57,7 +57,7 @@ std::string maskDataPath(const std::string & path) size_t user_pw_end = masked_path.find('@', node_pos); if (user_pw_end == std::string::npos) { - /// Likey new format (use_compact_format_in_distributed_parts_names=1) + /// Likely new format (use_compact_format_in_distributed_parts_names=1) return path; } diff --git a/utils/check-style/check-typos b/utils/check-style/check-typos index 3819b6785d7..9194a9464a7 100755 --- a/utils/check-style/check-typos +++ b/utils/check-style/check-typos @@ -5,7 +5,7 @@ ROOT_PATH=$(git rev-parse --show-toplevel) codespell \ - --skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,d3-*.js,*.min.js,${ROOT_PATH}/utils/check-style/aspell-ignore" \ + --skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,d3-*.js,*.min.js,*.sum,${ROOT_PATH}/utils/check-style/aspell-ignore" \ --ignore-words "${ROOT_PATH}/utils/check-style/codespell-ignore-words.list" \ --exclude-file "${ROOT_PATH}/utils/check-style/codespell-ignore-lines.list" \ --quiet-level 2 \ diff --git a/utils/check-style/codespell-ignore-words.list b/utils/check-style/codespell-ignore-words.list index 062e8a1622b..fc021920041 100644 --- a/utils/check-style/codespell-ignore-words.list +++ b/utils/check-style/codespell-ignore-words.list @@ -16,3 +16,8 @@ ot te fo ba +ro +rightt +iiterator +hastable +nam From f499381f5747938780532fe9b7f6edeef7591bb5 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 2 Sep 2022 18:45:37 +0800 Subject: [PATCH 302/582] move functional test to integration test Signed-off-by: Lloyd-Pottiger --- .../integration/test_rocksdb_options/test.py | 51 ------- .../test_rocksdb_read_only/__init__.py | 0 .../configs/rocksdb.xml | 22 +++ .../test_rocksdb_read_only/test.py | 138 ++++++++++++++++++ .../02404_rocksdb_read_only.reference | 15 -- .../0_stateless/02404_rocksdb_read_only.sh | 25 ---- 6 files changed, 160 insertions(+), 91 deletions(-) create mode 100644 tests/integration/test_rocksdb_read_only/__init__.py create mode 100644 tests/integration/test_rocksdb_read_only/configs/rocksdb.xml create mode 100644 tests/integration/test_rocksdb_read_only/test.py delete mode 100644 tests/queries/0_stateless/02404_rocksdb_read_only.reference delete mode 100755 tests/queries/0_stateless/02404_rocksdb_read_only.sh diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index 05713e6ab1b..c746d4d0042 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -56,57 +56,6 @@ def test_valid_options(start_cluster): ) -def test_dirctory_missing_after_stop(start_cluster): - # for read_only = false - node.query( - """ - CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only_missing') PRIMARY KEY(key); - """ - ) - node.stop_clickhouse() - node.exec_in_container( - [ - "bash", - "-c", - "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only_missing", - ] - ) - node.start_clickhouse() - result = node.query( - """INSERT INTO test (key, value) VALUES (0, 'a'); - SELECT * FROM test; - """ - ) - assert result.strip() == "0\ta" - node.query( - """DROP TABLE test; - """ - ) - # for read_only = true - node.query( - """ - CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only_missing', 1) PRIMARY KEY(key); - """ - ) - node.stop_clickhouse() - node.exec_in_container( - [ - "bash", - "-c", - "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only_missing", - ] - ) - node.start_clickhouse() - with pytest.raises(QueryRuntimeException): - node.query("""INSERT INTO test (key, value) VALUES (1, 'b');""") - result = node.query("""SELECT * FROM test;""") - assert result.strip() == "" - node.query( - """DROP TABLE test; - """ - ) - - def test_invalid_options(start_cluster): node.exec_in_container( [ diff --git a/tests/integration/test_rocksdb_read_only/__init__.py b/tests/integration/test_rocksdb_read_only/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_rocksdb_read_only/configs/rocksdb.xml b/tests/integration/test_rocksdb_read_only/configs/rocksdb.xml new file mode 100644 index 00000000000..c0ac49576fc --- /dev/null +++ b/tests/integration/test_rocksdb_read_only/configs/rocksdb.xml @@ -0,0 +1,22 @@ + + + + + 8 + + + 2 + + +

+ test + + 10000 + + + 14 + +
+ + + diff --git a/tests/integration/test_rocksdb_read_only/test.py b/tests/integration/test_rocksdb_read_only/test.py new file mode 100644 index 00000000000..a60cec474ad --- /dev/null +++ b/tests/integration/test_rocksdb_read_only/test.py @@ -0,0 +1,138 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +import pytest + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", main_configs=["configs/rocksdb.xml"], stay_alive=True +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_read_only(start_cluster): + # fail if read_only = true and directory does not exist. + with pytest.raises(QueryRuntimeException): + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + """ + ) + # create directory if read_only = false + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); + INSERT INTO test (key, value) VALUES (0, 'a'), (1, 'b'), (2, 'c'); + """ + ) + # fail if create multiple tables on the same directory + with pytest.raises(QueryRuntimeException): + node.query( + """ + CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + """ + ) + with pytest.raises(QueryRuntimeException): + node.query( + """ + CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); + """ + ) + with pytest.raises(QueryRuntimeException): + node.query( + """ + CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); + """ + ) + with pytest.raises(QueryRuntimeException): + node.query( + """ + CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + """ + ) + # success if create table on existing directory with no other tables on it + node.query( + """ + DROP TABLE test; + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + """ + ) + result = node.query("""SELECT count() FROM test;""") + assert result.strip() == "3" + # fail if insert into table with read_only = true + with pytest.raises(QueryRuntimeException): + node.query( + """INSERT INTO test (key, value) VALUES (4, 'd'); + """ + ) + node.query( + """ + DROP TABLE test; + """ + ) + + +def test_dirctory_missing_after_stop(start_cluster): + # for read_only = false + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only_missing') PRIMARY KEY(key); + """ + ) + node.stop_clickhouse() + node.exec_in_container( + [ + "bash", + "-c", + "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only_missing", + ] + ) + node.start_clickhouse() + result = node.query( + """INSERT INTO test (key, value) VALUES (0, 'a'); + SELECT * FROM test; + """ + ) + assert result.strip() == "0\ta" + node.query( + """DROP TABLE test; + """ + ) + # for read_only = true + node.query( + """ + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only_missing', 1) PRIMARY KEY(key); + """ + ) + node.stop_clickhouse() + node.exec_in_container( + [ + "bash", + "-c", + "rm -r /var/lib/clickhouse/store/test_rocksdb_read_only_missing", + ] + ) + node.start_clickhouse() + with pytest.raises(QueryRuntimeException): + node.query("""INSERT INTO test (key, value) VALUES (1, 'b');""") + result = node.query("""SELECT * FROM test;""") + assert result.strip() == "" + node.query( + """DROP TABLE test; + """ + ) + + diff --git a/tests/queries/0_stateless/02404_rocksdb_read_only.reference b/tests/queries/0_stateless/02404_rocksdb_read_only.reference deleted file mode 100644 index 85caf4188f1..00000000000 --- a/tests/queries/0_stateless/02404_rocksdb_read_only.reference +++ /dev/null @@ -1,15 +0,0 @@ -OK ----- -FAIL ----- -OK -OK ----- -FAIL -OK -OK ----- -FAIL -OK ----- -FAIL diff --git a/tests/queries/0_stateless/02404_rocksdb_read_only.sh b/tests/queries/0_stateless/02404_rocksdb_read_only.sh deleted file mode 100755 index 0590ac542ce..00000000000 --- a/tests/queries/0_stateless/02404_rocksdb_read_only.sh +++ /dev/null @@ -1,25 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-ordinary-database, no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ROCKSDB_DIR="$CLICKHOUSE_TMP/test_rocksdb_read_only" - -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_02404;" && echo 'OK' || echo 'FAIL'; -echo "----" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -echo "----" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '${ROCKSDB_DIR}') PRIMARY KEY(key);" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (0, 'a'), (1, 'b'), (3, 'c');" && echo 'OK' || echo 'FAIL'; -echo "----" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" && echo 'OK' || echo 'FAIL'; -echo "----" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_02404 (key, value) VALUES (4, 'd');" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="DROP TABLE test_02404;" && echo 'OK' || echo 'FAIL'; -echo "----" -rm -r ${ROCKSDB_DIR} -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_02404 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '${ROCKSDB_DIR}', 1) PRIMARY KEY(key);" 2>&1 | grep -F -q "OK" && echo 'OK' || echo 'FAIL'; From b76af9284e5f9881ec49329c0ac8b74d27e0ec51 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 2 Sep 2022 19:29:57 +0800 Subject: [PATCH 303/582] style check Signed-off-by: Lloyd-Pottiger --- tests/integration/test_rocksdb_read_only/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_rocksdb_read_only/test.py b/tests/integration/test_rocksdb_read_only/test.py index a60cec474ad..25cd3125e39 100644 --- a/tests/integration/test_rocksdb_read_only/test.py +++ b/tests/integration/test_rocksdb_read_only/test.py @@ -134,5 +134,3 @@ def test_dirctory_missing_after_stop(start_cluster): """DROP TABLE test; """ ) - - From 3f029951615acb6c2aed6a09581b31324f486f90 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Sep 2022 13:54:41 +0200 Subject: [PATCH 304/582] Better exceptions handling --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 27 ++++++++++++++++--- src/Storages/MergeTree/MergeTreeData.cpp | 26 +++++++++++++----- 2 files changed, 44 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 32c2c09a392..a52de88321c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -532,13 +532,34 @@ void IMergeTreeDataPart::removeIfNeeded() LOG_TRACE(storage.log, "Removed part from old location {}", path); } } - catch (...) + catch (const Exception & ex) { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while removing part {} with path {}", name, path)); + + /// In this case we want to avoid assertions, because such errors are unavoidable in setup + /// with zero-copy replication. + if (const auto * keeper_exception = dynamic_cast(&ex)) + { + if (Coordination::isHardwareError(keeper_exception->code)) + return; + } + /// FIXME If part it temporary, then directory will not be removed for 1 day (temporary_directories_lifetime). /// If it's tmp_merge_ or tmp_fetch_, /// then all future attempts to execute part producing operation will fail with "directory already exists". - /// Seems like it's especially important for remote disks, because removal may fail due to network issues. - tryLogCurrentException(__PRETTY_FUNCTION__, "while removiong path: " + path); + assert(!is_temp); + assert(state != MergeTreeDataPartState::DeleteOnDestroy); + assert(state != MergeTreeDataPartState::Temporary); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while removing part {} with path {}", name, path)); + + /// FIXME If part it temporary, then directory will not be removed for 1 day (temporary_directories_lifetime). + /// If it's tmp_merge_ or tmp_fetch_, + /// then all future attempts to execute part producing operation will fail with "directory already exists". + /// + /// For remote disks this issue is really frequent, so we don't about server here assert(!is_temp); assert(state != MergeTreeDataPartState::DeleteOnDestroy); assert(state != MergeTreeDataPartState::Temporary); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a8babeb59c0..5f7b455d974 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1046,29 +1046,43 @@ void MergeTreeData::loadDataPartsFromDisk( throw; broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while loading part {} on path {}", part->name, part_path)); } catch (...) { broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while loading part {} on path {}", part->name, part_path)); } /// Ignore broken parts that can appear as a result of hard server restart. if (broken) { - /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist - size_t size_of_part = data_part_storage->calculateTotalSizeOnDisk(); + std::optional size_of_part; + try + { + /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist + size_of_part = data_part_storage->calculateTotalSizeOnDisk(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while calculating part size {} on path {}", part->name, part_path)); + } + + std::string part_size_str = "failed to calculate size"; + if (size_of_part.has_value()) + part_size_str = formatReadableSizeWithBinarySuffix(*size_of_part); + LOG_ERROR(log, "Detaching broken part {}{} (size: {}). " "If it happened after update, it is likely because of backward incompability. " "You need to resolve this manually", - getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); + getFullPathOnDisk(part_disk_ptr), part_name, part_size_str); std::lock_guard loading_lock(mutex); broken_parts_to_detach.push_back(part); ++suspicious_broken_parts; - suspicious_broken_parts_bytes += size_of_part; + if (size_of_part.has_value()) + suspicious_broken_parts_bytes += *size_of_part; return; } if (!part->index_granularity_info.is_adaptive) From dd8478422a4d6c76b421569588efda00bee2022a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Sep 2022 14:15:18 +0200 Subject: [PATCH 305/582] Fix usage of generator expressions for libcxx cmake Before, if you store clickhouse sources in /src, there was a typo and it produce the following error: CMake Error in contrib/libcxx-cmake/CMakeLists.txt: Target "cxx" INTERFACE_INCLUDE_DIRECTORIES property contains path: "/src" which is prefixed in the source directory. Also move "src" into PRIVATE, since it is required only for libcxx itself. Signed-off-by: Azat Khuzhin --- contrib/libcxx-cmake/CMakeLists.txt | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index a501c4df64f..6f42a479588 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -54,9 +54,8 @@ set(SRCS add_library(cxx ${SRCS}) set_target_properties(cxx PROPERTIES FOLDER "contrib/libcxx-cmake") -target_include_directories(cxx SYSTEM BEFORE PUBLIC - $ - $/src) +target_include_directories(cxx SYSTEM BEFORE PRIVATE $) +target_include_directories(cxx SYSTEM BEFORE PUBLIC $) target_compile_definitions(cxx PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) # Enable capturing stack traces for all exceptions. From 53836bbeebfef9fa990add7b08fa949ee1b88358 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Sep 2022 12:49:50 +0000 Subject: [PATCH 306/582] Fix typo The system view is called 'role*_*grants', documented on page 'role*-*grants.md'. --- docs/en/sql-reference/statements/show.md | 2 +- docs/ru/sql-reference/statements/show.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 0721f17e9e2..00347d9cb5b 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -303,7 +303,7 @@ SHOW USERS ## SHOW ROLES -Returns a list of [roles](../../operations/access-rights.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role-grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). +Returns a list of [roles](../../operations/access-rights.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). ### Syntax diff --git a/docs/ru/sql-reference/statements/show.md b/docs/ru/sql-reference/statements/show.md index 1d072c9d5de..59f33c691ae 100644 --- a/docs/ru/sql-reference/statements/show.md +++ b/docs/ru/sql-reference/statements/show.md @@ -305,7 +305,7 @@ SHOW USERS ## SHOW ROLES {#show-roles-statement} -Выводит список [ролей](../../operations/access-rights.md#role-management). Для просмотра параметров ролей, см. системные таблицы [system.roles](../../operations/system-tables/roles.md#system_tables-roles) и [system.role-grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). +Выводит список [ролей](../../operations/access-rights.md#role-management). Для просмотра параметров ролей, см. системные таблицы [system.roles](../../operations/system-tables/roles.md#system_tables-roles) и [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). ### Синтаксис {#show-roles-syntax} From a326450ddf506d1e25913320c4a6a98f9465cef6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 13:12:02 +0000 Subject: [PATCH 307/582] Install enable_keeper_map config --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index e7d0f8e7acf..91be5286102 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -50,6 +50,7 @@ ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/session_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ From 396500aaa7c4e1d24f1c69f56bf244f75a6a0721 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 13:14:33 +0000 Subject: [PATCH 308/582] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index bdba298189e..1be805e7cb2 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit bdba298189e29995892de78dcecf64d127444e81 +Subproject commit 1be805e7cb2494aa8170015493474379b0362dfc From 0db38fcbc95483f577eee7f2ce40096f8fe456f6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 15:32:46 +0200 Subject: [PATCH 309/582] Fix --- src/IO/WriteBufferFromS3.cpp | 16 ++++++++++++---- src/IO/WriteBufferFromS3.h | 1 + 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 2f2479766d5..eda62f05446 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -431,7 +431,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() { if (schedule) { - std::lock_guard lock(bg_tasks_mutex); + std::unique_lock lock(bg_tasks_mutex); { while (!upload_object_tasks.empty() && upload_object_tasks.front().is_finised) { @@ -442,7 +442,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() if (exception) { - waitForAllBackGroundTasks(); + waitForAllBackGroundTasksUnlocked(lock); std::rethrow_exception(exception); } @@ -457,7 +457,15 @@ void WriteBufferFromS3::waitForAllBackGroundTasks() if (schedule) { std::unique_lock lock(bg_tasks_mutex); - bg_tasks_condvar.wait(lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; }); + waitForAllBackGroundTasksUnlocked(lock); + } +} + +void WriteBufferFromS3::waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock) +{ + if (schedule) + { + bg_tasks_condvar.wait(bg_tasks_lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; }); while (!upload_object_tasks.empty()) { @@ -472,7 +480,7 @@ void WriteBufferFromS3::waitForAllBackGroundTasks() if (put_object_task) { - bg_tasks_condvar.wait(lock, [this]() { return put_object_task->is_finised; }); + bg_tasks_condvar.wait(bg_tasks_lock, [this]() { return put_object_task->is_finised; }); if (put_object_task->exception) std::rethrow_exception(put_object_task->exception); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 99440654910..712044841d0 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -84,6 +84,7 @@ private: void waitForReadyBackGroundTasks(); void waitForAllBackGroundTasks(); + void waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock); String bucket; String key; From 0f3003e37bd41829203585cef2af11125e5fa725 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 2 Sep 2022 17:13:36 +0800 Subject: [PATCH 310/582] Add ccache `brew install` list --- docs/en/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index 97e4e4ddde1..12f74feb272 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -37,7 +37,7 @@ sudo xcode-select --install ``` bash brew update -brew install cmake ninja libtool gettext llvm gcc binutils grep findutils +brew install ccache cmake ninja libtool gettext llvm gcc binutils grep findutils ``` ## Checkout ClickHouse Sources {#checkout-clickhouse-sources} From c9b512e33eed1a825e5b64d66cf7291eb7a004f2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 15:32:46 +0200 Subject: [PATCH 311/582] Fix --- src/IO/WriteBufferFromS3.cpp | 16 ++++++++++++---- src/IO/WriteBufferFromS3.h | 1 + 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 6207ba53bd8..2510b86716c 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -431,7 +431,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() { if (schedule) { - std::lock_guard lock(bg_tasks_mutex); + std::unique_lock lock(bg_tasks_mutex); { while (!upload_object_tasks.empty() && upload_object_tasks.front().is_finised) { @@ -442,7 +442,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() if (exception) { - waitForAllBackGroundTasks(); + waitForAllBackGroundTasksUnlocked(lock); std::rethrow_exception(exception); } @@ -457,7 +457,15 @@ void WriteBufferFromS3::waitForAllBackGroundTasks() if (schedule) { std::unique_lock lock(bg_tasks_mutex); - bg_tasks_condvar.wait(lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; }); + waitForAllBackGroundTasksUnlocked(lock); + } +} + +void WriteBufferFromS3::waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock) +{ + if (schedule) + { + bg_tasks_condvar.wait(bg_tasks_lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; }); while (!upload_object_tasks.empty()) { @@ -472,7 +480,7 @@ void WriteBufferFromS3::waitForAllBackGroundTasks() if (put_object_task) { - bg_tasks_condvar.wait(lock, [this]() { return put_object_task->is_finised; }); + bg_tasks_condvar.wait(bg_tasks_lock, [this]() { return put_object_task->is_finised; }); if (put_object_task->exception) std::rethrow_exception(put_object_task->exception); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 99440654910..712044841d0 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -84,6 +84,7 @@ private: void waitForReadyBackGroundTasks(); void waitForAllBackGroundTasks(); + void waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock); String bucket; String key; From 77071381e4c26d07a4f56419605e244bdb087667 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 2 Sep 2022 16:37:33 +0200 Subject: [PATCH 312/582] fix build --- src/Processors/Formats/ISchemaReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 34b4adf3b93..648b0f47365 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -123,7 +123,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() else if (!data_types.empty()) { /// If column names were set, check that the number of names match the number of types. - if ((column_names.size() != data_types.size()) + if (column_names.size() != data_types.size()) throw Exception( ErrorCodes::INCORRECT_DATA, "The number of column names {} differs with the number of types {}", column_names.size(), data_types.size()); From 74561b40aed21e99eb87f1da61e2ba34c08a0ea4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 2 Sep 2022 17:42:18 +0300 Subject: [PATCH 313/582] stylecheck renewed --- base/glibc-compatibility/memcpy/memcpy.h | 1 - docker/test/style/Dockerfile | 2 +- src/Common/Stopwatch.h | 1 - src/Storages/Kafka/StorageKafka.cpp | 2 +- 4 files changed, 2 insertions(+), 4 deletions(-) diff --git a/base/glibc-compatibility/memcpy/memcpy.h b/base/glibc-compatibility/memcpy/memcpy.h index ff27c970bac..0930dfb5c67 100644 --- a/base/glibc-compatibility/memcpy/memcpy.h +++ b/base/glibc-compatibility/memcpy/memcpy.h @@ -214,4 +214,3 @@ tail: return ret; } - diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 0ec3f09ab7f..683124feaa0 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -17,7 +17,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black boto3 codespell dohq-artifactory PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 9e48c33f70d..cabc6d8ba1e 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -152,4 +152,3 @@ private: /// Most significant bit is a lock. When it is set, compareAndRestartDeferred method will return false. UInt64 nanoseconds(UInt64 prev_time) const { return clock_gettime_ns_adjusted(prev_time, clock_type) & 0x7FFFFFFFFFFFFFFFULL; } }; - diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 96987b0f60e..06ce4fb308d 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -810,7 +810,7 @@ void registerStorageKafka(StorageFactory & factory) /** Arguments of engine is following: * - Kafka broker list * - List of topics - * - Group ID (may be a constraint expression with a string result) + * - Group ID (may be a constant expression with a string result) * - Message format (string) * - Row delimiter * - Schema (optional, if the format supports it) From 3daf1f5c77190513761bedc986eae2db8a2ed207 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Sep 2022 16:57:51 +0200 Subject: [PATCH 314/582] fix build --- src/IO/S3/PocoHTTPClient.cpp | 2 +- src/IO/S3/PocoHTTPClient.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index fc5617ceff5..569bebb1ed1 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -191,7 +191,7 @@ namespace } } -PocoHTTPClient::S3MetricKind PocoHTTPClient::getMetricKind(const Aws::Http::HttpRequest & request) const +PocoHTTPClient::S3MetricKind PocoHTTPClient::getMetricKind(const Aws::Http::HttpRequest & request) { switch (request.GetMethod()) { diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 74d0d0e09ae..9005f132974 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -141,7 +141,7 @@ private: EnumSize, }; - S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request) const; + static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; std::function per_request_configuration; From 464818c142766787854d64fcad41869b71d37460 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 2 Sep 2022 15:05:58 +0000 Subject: [PATCH 315/582] try to fix filling of missed Nested columns with multiple levels --- src/Storages/MergeTree/IMergeTreeReader.cpp | 39 ++++++++--- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 68 ++++++++++--------- .../MergeTree/MergeTreeReaderCompact.h | 1 + .../MergeTree/MergeTreeReaderInMemory.cpp | 8 +-- 5 files changed, 74 insertions(+), 44 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 36a1e5e4b55..b77975ff9ad 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -204,17 +204,40 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const } } -IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const String & column_name) const +IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const NameAndTypePair & required_column) const { - String table_name = Nested::extractTableName(column_name); + auto get_offset_streams = [](const auto & serialization, const auto & name_in_storage) + { + NameSet offset_streams; + serialization->enumerateStreams([&](const auto & subpath) + { + if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) + return; + + auto subname = ISerialization::getSubcolumnNameForStream(subpath); + auto full_name = Nested::concatenateName(name_in_storage, subname); + offset_streams.insert(full_name); + }); + + return offset_streams; + }; + + auto required_name_in_storage = Nested::extractTableName(required_column.getNameInStorage()); + auto required_offset_streams = get_offset_streams(getSerializationInPart(required_column), required_name_in_storage); + for (const auto & part_column : data_part->getColumns()) { - if (typeid_cast(part_column.type.get())) - { - auto position = data_part->getColumnPosition(part_column.getNameInStorage()); - if (position && Nested::extractTableName(part_column.name) == table_name) - return position; - } + auto name_in_storage = Nested::extractTableName(part_column.name); + if (name_in_storage != required_name_in_storage) + continue; + + auto offset_streams = get_offset_streams(data_part->getSerialization(part_column.name), name_in_storage); + + bool has_all_offsets = std::all_of(required_offset_streams.begin(), required_offset_streams.end(), + [&](const auto & stream_name) { return offset_streams.contains(stream_name); }); + + if (has_all_offsets) + return data_part->getColumnPosition(part_column.name); } return {}; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 453563522a5..fa1bf988fbf 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -91,7 +91,7 @@ protected: MarkRanges all_mark_ranges; using ColumnPosition = std::optional; - ColumnPosition findColumnForOffsets(const String & column_name) const; + ColumnPosition findColumnForOffsets(const NameAndTypePair & column) const; private: /// Alter conversions, which must be applied on fly if required diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 28ec2eff56f..87e0d75a871 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -46,37 +46,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { try { - size_t columns_num = columns_to_read.size(); - - column_positions.resize(columns_num); - read_only_offsets.resize(columns_num); - - for (size_t i = 0; i < columns_num; ++i) - { - const auto & column_to_read = columns_to_read[i]; - - auto position = data_part->getColumnPosition(column_to_read.getNameInStorage()); - bool is_array = isArray(column_to_read.type); - - if (column_to_read.isSubcolumn()) - { - auto storage_column_from_part = getColumnInPart( - {column_to_read.getNameInStorage(), column_to_read.getTypeInStorage()}); - - auto subcolumn_name = column_to_read.getSubcolumnName(); - if (!storage_column_from_part.type->hasSubcolumn(subcolumn_name)) - position.reset(); - } - else if (!position && is_array) - { - /// If array of Nested column is missing in part, - /// we have to read its offsets if they exist. - position = findColumnForOffsets(column_to_read.name); - read_only_offsets[i] = (position != std::nullopt); - } - - column_positions[i] = std::move(position); - } + fillColumnPositions(); /// Do not use max_read_buffer_size, but try to lower buffer size with maximal size of granule to avoid reading much data. auto buffer_size = getReadBufferSize(data_part, marks_loader, column_positions, all_mark_ranges); @@ -139,6 +109,42 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } } +void MergeTreeReaderCompact::fillColumnPositions() +{ + size_t columns_num = columns_to_read.size(); + + column_positions.resize(columns_num); + read_only_offsets.resize(columns_num); + + for (size_t i = 0; i < columns_num; ++i) + { + const auto & column_to_read = columns_to_read[i]; + + auto position = data_part->getColumnPosition(column_to_read.getNameInStorage()); + bool is_array = isArray(column_to_read.type); + + if (column_to_read.isSubcolumn()) + { + auto storage_column_from_part = getColumnInPart( + {column_to_read.getNameInStorage(), column_to_read.getTypeInStorage()}); + + auto subcolumn_name = column_to_read.getSubcolumnName(); + if (!storage_column_from_part.type->hasSubcolumn(subcolumn_name)) + position.reset(); + } + + if (!position && is_array) + { + /// If array of Nested column is missing in part, + /// we have to read its offsets if they exist. + position = findColumnForOffsets(column_to_read); + read_only_offsets[i] = (position != std::nullopt); + } + + column_positions[i] = std::move(position); + } +} + size_t MergeTreeReaderCompact::readRows( size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index aa0eb949aa1..4c5f1231063 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -39,6 +39,7 @@ public: private: bool isContinuousReading(size_t mark, size_t column_position); + void fillColumnPositions(); ReadBuffer * data_buffer; CompressedReadBufferBase * compressed_data_buffer; diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 766c28c99b9..568e1cecf02 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -32,13 +32,13 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( {}) , part_in_memory(std::move(data_part_)) { - for (const auto & [name, type] : columns_to_read) + for (const auto & column_to_read : columns_to_read) { /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. - if (!part_in_memory->block.has(name) && typeid_cast(type.get())) - if (auto offset_position = findColumnForOffsets(name)) - positions_for_offsets[name] = *offset_position; + if (!part_in_memory->block.has(column_to_read.name) && typeid_cast(column_to_read.type.get())) + if (auto offset_position = findColumnForOffsets(column_to_read)) + positions_for_offsets[column_to_read.name] = *offset_position; } } From 69b9d34b10c5188f3b6b13ce8ef2de9f28235da6 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Fri, 2 Sep 2022 17:25:10 +0200 Subject: [PATCH 316/582] docs. optional params for GenerateRandom table --- docs/en/engines/table-engines/special/generate.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/generate.md b/docs/en/engines/table-engines/special/generate.md index d03d6dc9d13..e42429a1b10 100644 --- a/docs/en/engines/table-engines/special/generate.md +++ b/docs/en/engines/table-engines/special/generate.md @@ -15,7 +15,7 @@ Usage examples: ## Usage in ClickHouse Server {#usage-in-clickhouse-server} ``` sql -ENGINE = GenerateRandom(random_seed, max_string_length, max_array_length) +ENGINE = GenerateRandom([random_seed] [,max_string_length] [,max_array_length]) ``` The `max_array_length` and `max_string_length` parameters specify maximum length of all From dd776eb3d58c239a4b8c6db5817576c0ac446ca5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 2 Sep 2022 16:18:50 +0000 Subject: [PATCH 317/582] fix enumerateStreams --- src/DataTypes/Serializations/SerializationLowCardinality.cpp | 5 +++-- src/DataTypes/Serializations/SerializationNullable.cpp | 5 +++-- src/DataTypes/Serializations/SerializationSparse.cpp | 5 +++-- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 528cd13ddf3..dfe0188c8e7 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -48,12 +48,13 @@ void SerializationLowCardinality::enumerateStreams( const auto * column_lc = data.column ? &getColumnLowCardinality(*data.column) : nullptr; settings.path.push_back(Substream::DictionaryKeys); - settings.path.back().data = SubstreamData(dict_inner_serialization) + auto dict_data = SubstreamData(dict_inner_serialization) .withType(data.type ? dictionary_type : nullptr) .withColumn(column_lc ? column_lc->getDictionary().getNestedColumn() : nullptr) .withSerializationInfo(data.serialization_info); - dict_inner_serialization->enumerateStreams(settings, callback, settings.path.back().data); + settings.path.back().data = dict_data; + dict_inner_serialization->enumerateStreams(settings, callback, dict_data); settings.path.back() = Substream::DictionaryIndexes; settings.path.back().data = data; diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 47780f67800..560b73bc827 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -48,15 +48,16 @@ void SerializationNullable::enumerateStreams( auto null_map_serialization = std::make_shared(std::make_shared>(), "null", false); settings.path.push_back(Substream::NullMap); - settings.path.back().data = SubstreamData(null_map_serialization) + auto null_map_data = SubstreamData(null_map_serialization) .withType(type_nullable ? std::make_shared() : nullptr) .withColumn(column_nullable ? column_nullable->getNullMapColumnPtr() : nullptr) .withSerializationInfo(data.serialization_info); + settings.path.back().data = null_map_data; callback(settings.path); settings.path.back() = Substream::NullableElements; - settings.path.back().creator = std::make_shared(settings.path.back().data.column); + settings.path.back().creator = std::make_shared(null_map_data.column); settings.path.back().data = data; auto next_data = SubstreamData(nested) diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 70f9a03f510..855bdfa1b3e 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -156,15 +156,16 @@ void SerializationSparse::enumerateStreams( size_t column_size = column_sparse ? column_sparse->size() : 0; settings.path.push_back(Substream::SparseOffsets); - settings.path.back().data = SubstreamData(std::make_shared>()) + auto offsets_data = SubstreamData(std::make_shared>()) .withType(data.type ? std::make_shared() : nullptr) .withColumn(column_sparse ? column_sparse->getOffsetsPtr() : nullptr) .withSerializationInfo(data.serialization_info); + settings.path.back().data = offsets_data; callback(settings.path); settings.path.back() = Substream::SparseElements; - settings.path.back().creator = std::make_shared(settings.path.back().data.column, column_size); + settings.path.back().creator = std::make_shared(offsets_data.column, column_size); settings.path.back().data = data; auto next_data = SubstreamData(nested) From 239847ad169ab36f8736c7be1598603693d099b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 17:48:36 +0200 Subject: [PATCH 318/582] Fix races --- src/IO/WriteBufferFromS3.cpp | 71 +++++++++++++++++++++--------------- src/IO/WriteBufferFromS3.h | 31 ++++++++-------- 2 files changed, 58 insertions(+), 44 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 2510b86716c..c2e6ecc3e68 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -40,7 +40,7 @@ namespace ErrorCodes struct WriteBufferFromS3::UploadPartTask { Aws::S3::Model::UploadPartRequest req; - bool is_finised = false; + bool is_finished = false; std::string tag; std::exception_ptr exception; }; @@ -48,7 +48,7 @@ struct WriteBufferFromS3::UploadPartTask struct WriteBufferFromS3::PutObjectTask { Aws::S3::Model::PutObjectRequest req; - bool is_finised = false; + bool is_finished = false; std::exception_ptr exception; }; @@ -64,10 +64,10 @@ WriteBufferFromS3::WriteBufferFromS3( : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) , key(key_) - , client_ptr(std::move(client_ptr_)) - , upload_part_size(s3_settings_.min_upload_part_size) , s3_settings(s3_settings_) + , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) + , upload_part_size(s3_settings_.min_upload_part_size) , schedule(std::move(schedule_)) , write_settings(write_settings_) { @@ -218,7 +218,7 @@ void WriteBufferFromS3::writePart() return; } - if (part_tags.size() == S3_WARN_MAX_PARTS) + if (TSA_SUPPRESS_WARNING_FOR_READ(part_tags).size() == S3_WARN_MAX_PARTS) { // Don't throw exception here by ourselves but leave the decision to take by S3 server. LOG_WARNING(log, "Maximum part number in S3 protocol has reached (too many parts). Server may not accept this whole upload."); @@ -231,6 +231,7 @@ void WriteBufferFromS3::writePart() int part_number; { std::lock_guard lock(bg_tasks_mutex); + task = &upload_object_tasks.emplace_back(); ++num_added_bg_tasks; part_number = num_added_bg_tasks; @@ -240,7 +241,7 @@ void WriteBufferFromS3::writePart() auto task_finish_notify = [&, task]() { std::lock_guard lock(bg_tasks_mutex); - task->is_finised = true; + task->is_finished = true; ++num_finished_bg_tasks; /// Notification under mutex is important here. @@ -276,9 +277,11 @@ void WriteBufferFromS3::writePart() else { UploadPartTask task; - fillUploadRequest(task.req, part_tags.size() + 1); + auto & tags = TSA_SUPPRESS_WARNING_FOR_WRITE(part_tags); /// Suppress warning because schedule == false. + + fillUploadRequest(task.req, tags.size() + 1); processUploadRequest(task); - part_tags.push_back(task.tag); + tags.push_back(task.tag); } } @@ -302,6 +305,7 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) if (outcome.IsSuccess()) { task.tag = outcome.GetResult().GetETag(); + std::lock_guard lock(bg_tasks_mutex); /// Protect part_tags from race LOG_TRACE(log, "Writing part finished. Bucket: {}, Key: {}, Upload_id: {}, Etag: {}, Parts: {}", bucket, key, multipart_upload_id, task.tag, part_tags.size()); } else @@ -312,9 +316,11 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) void WriteBufferFromS3::completeMultipartUpload() { - LOG_TRACE(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, part_tags.size()); + const auto & tags = TSA_SUPPRESS_WARNING_FOR_READ(part_tags); - if (part_tags.empty()) + LOG_TRACE(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, tags.size()); + + if (tags.empty()) throw Exception("Failed to complete multipart upload. No parts have uploaded", ErrorCodes::S3_ERROR); Aws::S3::Model::CompleteMultipartUploadRequest req; @@ -323,10 +329,10 @@ void WriteBufferFromS3::completeMultipartUpload() req.SetUploadId(multipart_upload_id); Aws::S3::Model::CompletedMultipartUpload multipart_upload; - for (size_t i = 0; i < part_tags.size(); ++i) + for (size_t i = 0; i < tags.size(); ++i) { Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); + multipart_upload.AddParts(part.WithETag(tags[i]).WithPartNumber(i + 1)); } req.SetMultipartUpload(multipart_upload); @@ -334,12 +340,12 @@ void WriteBufferFromS3::completeMultipartUpload() auto outcome = client_ptr->CompleteMultipartUpload(req); if (outcome.IsSuccess()) - LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, part_tags.size()); + LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, tags.size()); else { throw Exception(ErrorCodes::S3_ERROR, "{} Tags:{}", outcome.GetError().GetMessage(), - fmt::join(part_tags.begin(), part_tags.end(), " ")); + fmt::join(tags.begin(), tags.end(), " ")); } } @@ -364,7 +370,7 @@ void WriteBufferFromS3::makeSinglepartUpload() auto task_notify_finish = [&]() { std::lock_guard lock(bg_tasks_mutex); - put_object_task->is_finised = true; + put_object_task->is_finished = true; /// Notification under mutex is important here. /// Othervies, WriteBuffer could be destroyed in between @@ -417,7 +423,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) req.SetContentType("binary/octet-stream"); } -void WriteBufferFromS3::processPutRequest(PutObjectTask & task) +void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) { auto outcome = client_ptr->PutObject(task.req); bool with_pool = static_cast(schedule); @@ -432,13 +438,15 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() if (schedule) { std::unique_lock lock(bg_tasks_mutex); + /// Suppress warnings because bg_tasks_mutex is actually hold, but tsa annotations do not understand std::unique_lock + auto & tasks = TSA_SUPPRESS_WARNING_FOR_WRITE(upload_object_tasks); { - while (!upload_object_tasks.empty() && upload_object_tasks.front().is_finised) + while (!tasks.empty() && tasks.front().is_finished) { - auto & task = upload_object_tasks.front(); + auto & task = tasks.front(); auto exception = task.exception; auto tag = std::move(task.tag); - upload_object_tasks.pop_front(); + tasks.pop_front(); if (exception) { @@ -446,7 +454,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() std::rethrow_exception(exception); } - part_tags.push_back(tag); + TSA_SUPPRESS_WARNING_FOR_WRITE(part_tags).push_back(tag); } } } @@ -465,24 +473,29 @@ void WriteBufferFromS3::waitForAllBackGroundTasksUnlocked(std::unique_lockis_finised; }); - if (put_object_task->exception) - std::rethrow_exception(put_object_task->exception); + bg_tasks_condvar.wait(bg_tasks_lock, [&task]() { return task->is_finished; }); + if (task->exception) + std::rethrow_exception(task->exception); } } } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 712044841d0..ae03299ffbd 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -80,38 +80,39 @@ private: struct PutObjectTask; void fillPutRequest(Aws::S3::Model::PutObjectRequest & req); - void processPutRequest(PutObjectTask & task); + void processPutRequest(const PutObjectTask & task); void waitForReadyBackGroundTasks(); void waitForAllBackGroundTasks(); void waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock); - String bucket; - String key; - std::shared_ptr client_ptr; - size_t upload_part_size = 0; - S3Settings::ReadWriteSettings s3_settings; - std::optional> object_metadata; + const String bucket; + const String key; + const S3Settings::ReadWriteSettings s3_settings; + const std::shared_ptr client_ptr; + const std::optional> object_metadata; - /// Buffer to accumulate data. - std::shared_ptr temporary_buffer; + size_t upload_part_size = 0; + std::shared_ptr temporary_buffer; /// Buffer to accumulate data. size_t last_part_size = 0; std::atomic total_parts_uploaded = 0; /// Upload in S3 is made in parts. /// We initiate upload, then upload each part and get ETag as a response, and then finalizeImpl() upload with listing all our parts. String multipart_upload_id; - std::vector part_tags; + std::vector TSA_GUARDED_BY(bg_tasks_mutex) part_tags; bool is_prefinalized = false; /// Following fields are for background uploads in thread pool (if specified). /// We use std::function to avoid dependency of Interpreters - ScheduleFunc schedule; - std::unique_ptr put_object_task; - std::list upload_object_tasks; - size_t num_added_bg_tasks = 0; - size_t num_finished_bg_tasks = 0; + const ScheduleFunc schedule; + + std::unique_ptr put_object_task; /// Does not need protection by mutex because of the logic around is_finished field. + std::list TSA_GUARDED_BY(bg_tasks_mutex) upload_object_tasks; + size_t num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; + size_t num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; + std::mutex bg_tasks_mutex; std::condition_variable bg_tasks_condvar; From 1e81e0ef678ea968ac1997c201e8a60c77ae53d8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 2 Sep 2022 18:54:34 +0200 Subject: [PATCH 319/582] Update WriteBufferFromS3.cpp --- src/IO/WriteBufferFromS3.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index c2e6ecc3e68..208ef6d015b 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -473,8 +473,7 @@ void WriteBufferFromS3::waitForAllBackGroundTasksUnlocked(std::unique_lock Date: Fri, 2 Sep 2022 10:11:57 -0700 Subject: [PATCH 320/582] re arrange sensitive data masker unit test scenario order --- .../tests/gtest_sensitive_data_masker.cpp | 102 +++++++++--------- 1 file changed, 52 insertions(+), 50 deletions(-) diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index b9ee9025c03..37f835e2884 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -101,56 +101,6 @@ TEST(Common, SensitiveDataMasker) EXPECT_EQ(maskerbad.rulesCount(), 0); EXPECT_EQ(maskerbad.wipeSensitiveData(x), 0); - { - std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - xml_isteam(R"END( - - - - hide SSN - [0-9]{3}-[0-9]{2}-[0-9]{4} - 000-00-0000 - - - hide root password - qwerty123 - - - (?i)Ivan - John - - - (?i)Petrov - Doe - - - hide email - (?i)[A-Z0-9._%+-]+@[A-Z0-9.-]+\.[A-Z]{2,4} - hidden@hidden.test - - - remove selects to bad_words table - ^.*bad_words.*$ - [QUERY IS CENSORED] - - -)END"); - - Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam); - DB::SensitiveDataMasker masker_xml_based(*xml_config, "query_masking_rules"); - std::string top_secret = "The e-mail of IVAN PETROV is kotik1902@sdsdf.test, and the password is qwerty123"; - EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 4); - EXPECT_EQ(top_secret, "The e-mail of John Doe is hidden@hidden.test, and the password is ******"); - - top_secret = "SELECT * FROM bad_words"; - EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 1); - EXPECT_EQ(top_secret, "[QUERY IS CENSORED]"); - -#ifndef NDEBUG - masker_xml_based.printStats(); -#endif - } - try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -228,4 +178,56 @@ TEST(Common, SensitiveDataMasker) EXPECT_EQ(e.code(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP); } + { + std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM + xml_isteam(R"END( + + + + hide SSN + [0-9]{3}-[0-9]{2}-[0-9]{4} + 000-00-0000 + + + hide root password + qwerty123 + + + (?i)Ivan + John + + + (?i)Petrov + Doe + + + hide email + (?i)[A-Z0-9._%+-]+@[A-Z0-9.-]+\.[A-Z]{2,4} + hidden@hidden.test + + + remove selects to bad_words table + ^.*bad_words.*$ + [QUERY IS CENSORED] + + +)END"); + + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam); + DB::SensitiveDataMasker masker_xml_based(*xml_config, "query_masking_rules"); + std::string top_secret = "The e-mail of IVAN PETROV is kotik1902@sdsdf.test, and the password is qwerty123"; + EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 4); + EXPECT_EQ(top_secret, "The e-mail of John Doe is hidden@hidden.test, and the password is ******"); + + top_secret = "SELECT * FROM bad_words"; + EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 1); + EXPECT_EQ(top_secret, "[QUERY IS CENSORED]"); + +#ifndef NDEBUG + masker_xml_based.printStats(); +#endif + } + + + } From 20444708955832f6bb1c6a648994f3e1d177611c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Sep 2022 19:18:44 +0200 Subject: [PATCH 321/582] review fixes --- src/Common/ZooKeeper/IKeeper.h | 6 ++++ src/Common/ZooKeeper/ZooKeeperArgs.h | 6 +--- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 50 +++++++++++++------------- 3 files changed, 32 insertions(+), 30 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index c6aa954688b..bd08aa2fc10 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -428,6 +428,12 @@ public: Exception(const Error code_, const std::string & path); /// NOLINT Exception(const Exception & exc); + template + Exception(const Error code_, fmt::format_string fmt, Args &&... args) + : Exception(fmt::format(fmt, std::forward(args)...), code_) + { + } + const char * name() const noexcept override { return "Coordination::Exception"; } const char * className() const noexcept override { return "Coordination::Exception"; } Exception * clone() const override { return new Exception(*this); } diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index 1fb9ca4bf56..b5c7b293506 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -3,13 +3,10 @@ #include #include -namespace Poco -{ -namespace Util +namespace Poco::Util { class AbstractConfiguration; } -} namespace zkutil { @@ -22,7 +19,6 @@ struct ZooKeeperArgs ZooKeeperArgs(const String & hosts_string); ZooKeeperArgs() = default; bool operator == (const ZooKeeperArgs &) const = default; - bool operator != (const ZooKeeperArgs &) const = default; String implementation = "zookeeper"; Strings hosts; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index d843e4f7896..ece6ce7513a 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -282,7 +282,7 @@ static void removeRootPath(String & path, const String & chroot) return; if (path.size() <= chroot.size()) - throw Exception("Received path is not longer than chroot", Error::ZDATAINCONSISTENCY); + throw Exception(Error::ZDATAINCONSISTENCY, "Received path is not longer than chroot"); path = path.substr(chroot.size()); } @@ -369,7 +369,7 @@ void ZooKeeper::connect( Poco::Timespan connection_timeout) { if (nodes.empty()) - throw Exception("No nodes passed to ZooKeeper constructor", Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); static constexpr size_t num_tries = 3; bool connected = false; @@ -458,7 +458,7 @@ void ZooKeeper::connect( } message << fail_reasons.str() << "\n"; - throw Exception(message.str(), Error::ZCONNECTIONLOSS); + throw Exception(Error::ZCONNECTIONLOSS, message.str()); } else { @@ -496,7 +496,7 @@ void ZooKeeper::receiveHandshake() read(handshake_length); if (handshake_length != SERVER_HANDSHAKE_LENGTH) - throw Exception("Unexpected handshake length received: " + DB::toString(handshake_length), Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Unexpected handshake length received: {}", handshake_length); read(protocol_version_read); if (protocol_version_read != ZOOKEEPER_PROTOCOL_VERSION) @@ -505,9 +505,9 @@ void ZooKeeper::receiveHandshake() /// It's better for faster failover than just connection drop. /// Implemented in clickhouse-keeper. if (protocol_version_read == KEEPER_PROTOCOL_VERSION_CONNECTION_REJECT) - throw Exception("Keeper server rejected the connection during the handshake. Possibly it's overloaded, doesn't see leader or stale", Error::ZCONNECTIONLOSS); + throw Exception(Error::ZCONNECTIONLOSS, "Keeper server rejected the connection during the handshake. Possibly it's overloaded, doesn't see leader or stale"); else - throw Exception("Unexpected protocol version: " + DB::toString(protocol_version_read), Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Unexpected protocol version: {}", protocol_version_read); } read(timeout); @@ -540,17 +540,15 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) read(err); if (read_xid != AUTH_XID) - throw Exception("Unexpected event received in reply to auth request: " + DB::toString(read_xid), - Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Unexpected event received in reply to auth request: {}", read_xid); int32_t actual_length = in->count() - count_before_event; if (length != actual_length) - throw Exception("Response length doesn't match. Expected: " + DB::toString(length) + ", actual: " + DB::toString(actual_length), - Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); if (err != Error::ZOK) - throw Exception("Error received in reply to auth request. Code: " + DB::toString(static_cast(err)) + ". Message: " + String(errorMessage(err)), - Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Error received in reply to auth request. Code: {}. Message: {}", + static_cast(err), errorMessage(err)); } @@ -655,7 +653,8 @@ void ZooKeeper::receiveThread() earliest_operation = operations.begin()->second; auto earliest_operation_deadline = earliest_operation->time + std::chrono::microseconds(args.operation_timeout_ms * 1000); if (now > earliest_operation_deadline) - throw Exception("Operation timeout (deadline already expired) for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); + throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (deadline already expired) for path: {}", + earliest_operation->request->getPath()); max_wait_us = std::chrono::duration_cast(earliest_operation_deadline - now).count(); } } @@ -672,11 +671,12 @@ void ZooKeeper::receiveThread() { if (earliest_operation) { - throw Exception("Operation timeout (no response) for request " + toString(earliest_operation->request->getOpNum()) + " for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); + throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response) for request {} for path: {}", + earliest_operation->request->getOpNum(), earliest_operation->request->getPath()); } waited_us += max_wait_us; if (waited_us >= args.session_timeout_ms * 1000) - throw Exception("Nothing is received in session timeout", Error::ZOPERATIONTIMEOUT); + throw Exception(Error::ZOPERATIONTIMEOUT, "Nothing is received in session timeout"); } @@ -709,12 +709,12 @@ void ZooKeeper::receiveEvent() UInt64 elapsed_ms = 0; if (unlikely(recv_inject_fault) && recv_inject_fault.value()(thread_local_rng)) - throw Exception("Session expired (fault injected)", Error::ZSESSIONEXPIRED); + throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); if (xid == PING_XID) { if (err != Error::ZOK) - throw Exception("Received error in heartbeat response: " + String(errorMessage(err)), Error::ZRUNTIMEINCONSISTENCY); + throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", errorMessage(err)); response = std::make_shared(); } @@ -819,7 +819,7 @@ void ZooKeeper::receiveEvent() int32_t actual_length = in->count() - count_before_event; if (length != actual_length) - throw Exception("Response length doesn't match. Expected: " + DB::toString(length) + ", actual: " + DB::toString(actual_length), Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_ms); //-V614 } @@ -1043,9 +1043,9 @@ void ZooKeeper::pushRequest(RequestInfo && info) { info.request->xid = next_xid.fetch_add(1); if (info.request->xid == CLOSE_XID) - throw Exception("xid equal to close_xid", Error::ZSESSIONEXPIRED); + throw Exception(Error::ZSESSIONEXPIRED, "xid equal to close_xid"); if (info.request->xid < 0) - throw Exception("XID overflow", Error::ZSESSIONEXPIRED); + throw Exception(Error::ZSESSIONEXPIRED, "XID overflow"); if (auto * multi_request = dynamic_cast(info.request.get())) { @@ -1055,14 +1055,14 @@ void ZooKeeper::pushRequest(RequestInfo && info) } if (unlikely(send_inject_fault) && send_inject_fault.value()(thread_local_rng)) - throw Exception("Session expired (fault injected)", Error::ZSESSIONEXPIRED); + throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on send)"); if (!requests_queue.tryPush(std::move(info), args.operation_timeout_ms)) { if (requests_queue.isFinished()) - throw Exception("Session expired", Error::ZSESSIONEXPIRED); + throw Exception(Error::ZSESSIONEXPIRED, "Session expired"); - throw Exception("Cannot push request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); + throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout"); } } catch (...) @@ -1231,7 +1231,7 @@ void ZooKeeper::list( if (keeper_api_version < Coordination::KeeperApiVersion::WITH_FILTERED_LIST) { if (list_request_type != ListRequestType::ALL) - throw Exception("Filtered list request type cannot be used because it's not supported by the server", Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); request = std::make_shared(); } @@ -1311,7 +1311,7 @@ void ZooKeeper::close() request_info.request = std::make_shared(std::move(request)); if (!requests_queue.tryPush(std::move(request_info), args.operation_timeout_ms)) - throw Exception("Cannot push close request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); + throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push close request to queue within operation timeout"); ProfileEvents::increment(ProfileEvents::ZooKeeperClose); } From 2dcf3fef94b47463d8f6e3aeebb8bf7e660f9e64 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 19:27:52 +0200 Subject: [PATCH 322/582] Fix tests --- programs/local/LocalServer.cpp | 4 ++++ src/Interpreters/Context.h | 1 - 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ce31600642a..83af6e1b5f0 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -367,6 +367,10 @@ try { UseSSL use_ssl; ThreadStatus thread_status; + SCOPE_EXIT_SAFE({ + if (connection) + connection.reset(); + }); StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true)); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index cc54ce47f92..ea03b8e6586 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -126,7 +126,6 @@ using PartUUIDsPtr = std::shared_ptr; class KeeperDispatcher; class Session; struct WriteSettings; -class Threadpool; class IInputFormat; class IOutputFormat; From 90baf74f13cb4e6e1d89475bb8f3657403d27822 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 2 Sep 2022 19:30:35 +0200 Subject: [PATCH 323/582] Added mkdir command --- programs/disks/CommandMkDir.cpp | 67 +++++++++++++++++++++++++++++++++ programs/disks/DisksApp.cpp | 3 +- programs/disks/DisksApp.h | 1 + programs/disks/ICommand.h | 1 + 4 files changed, 71 insertions(+), 1 deletion(-) create mode 100644 programs/disks/CommandMkDir.cpp diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp new file mode 100644 index 00000000000..11a940028a3 --- /dev/null +++ b/programs/disks/CommandMkDir.cpp @@ -0,0 +1,67 @@ +#pragma once + +#include "ICommand.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +class CommandMkDir : public ICommand +{ +public: + CommandMkDir() + { + command_name = "mkdir"; + command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); + description = "Create directory or directories recursively"; + usage = "mkdir [OPTION]... "; + command_option_description->add_options() + ("recursive", "recursively create directories") + ; + } + + void processOptions( + Poco::Util::LayeredConfiguration & config, + po::variables_map & options) const override + { + if (options.count("recursive")) + config.setBool("recursive", true); + } + + void execute( + const std::vector & command_arguments, + DB::ContextMutablePtr & global_context, + Poco::Util::LayeredConfiguration & config) override + { + if (command_arguments.size() != 1) + { + printHelpMessage(); + throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS); + } + + String disk_name = config.getString("disk", "default"); + + String path = command_arguments[0]; + + DiskPtr disk = global_context->getDisk(disk_name); + + String full_path = fullPathWithValidate(disk, path); + bool recursive = config.getBool("recursive", false); + + if (recursive) + disk->createDirectories(full_path); + else + disk->createDirectory(full_path); + } +}; +} + +std::unique_ptr makeCommandMkDir() +{ + return std::make_unique(); +} diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 08768386808..b662921a3b1 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -63,7 +63,7 @@ void DisksApp::addOptions( positional_options_description.add("command_name", 1); - supported_commands = {"list-disks", "list", "move", "remove", "link", "copy", "write", "read"}; + supported_commands = {"list-disks", "list", "move", "remove", "link", "copy", "write", "read", "mkdir"}; command_descriptions.emplace("list-disks", makeCommandListDisks()); command_descriptions.emplace("list", makeCommandList()); @@ -73,6 +73,7 @@ void DisksApp::addOptions( command_descriptions.emplace("copy", makeCommandCopy()); command_descriptions.emplace("write", makeCommandWrite()); command_descriptions.emplace("read", makeCommandRead()); + command_descriptions.emplace("mkdir", makeCommandMkDir()); } void DisksApp::processOptions() diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index cbb3a7dfcc9..24fa9c3d9eb 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -4,6 +4,7 @@ #include "CommandLink.cpp" #include "CommandList.cpp" #include "CommandListDisks.cpp" +#include "CommandMkDir.cpp" #include "CommandMove.cpp" #include "CommandRead.cpp" #include "CommandRemove.cpp" diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 9cde55dbb6b..f57f74a880e 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -65,3 +65,4 @@ std::unique_ptr makeCommandMove(); std::unique_ptr makeCommandRead(); std::unique_ptr makeCommandRemove(); std::unique_ptr makeCommandWrite(); +std::unique_ptr makeCommandMkDir(); From 2d3fb36a620ef755a3c9b6ed9993382f6e23d4b4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 31 Aug 2022 19:34:35 +0200 Subject: [PATCH 324/582] Fix debug symbols (Add a quirk to force clang emit .debug_aranges with ThinLTO) Wrap a linker into a script that will add some settings (`-mllvm -generate-arange-section`) in case of ThinLTO to emit `.debug_aranges` symbols. Dicussion in the LLVM can be found here [1]. [1]: https://discourse.llvm.org/t/clang-does-not-produce-full-debug-aranges-section-with-thinlto/64898 Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 2 ++ cmake/ld.lld.in | 17 +++++++++++++++++ cmake/tools.cmake | 9 +++++++-- .../02420_stracktrace_debug_symbols.reference | 1 + .../02420_stracktrace_debug_symbols.sh | 14 ++++++++++++++ 5 files changed, 41 insertions(+), 2 deletions(-) create mode 100755 cmake/ld.lld.in create mode 100644 tests/queries/0_stateless/02420_stracktrace_debug_symbols.reference create mode 100755 tests/queries/0_stateless/02420_stracktrace_debug_symbols.sh diff --git a/CMakeLists.txt b/CMakeLists.txt index dbbec2a600d..64fb870b61b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -143,6 +143,8 @@ include (cmake/add_warning.cmake) if (COMPILER_CLANG) # generate ranges for fast "addr2line" search if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") + # NOTE: that clang has a bug because of it does not emit .debug_aranges + # with ThinLTO, so custom ld.lld wrapper is shipped in docker images. set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") endif () diff --git a/cmake/ld.lld.in b/cmake/ld.lld.in new file mode 100755 index 00000000000..9736dab1bc3 --- /dev/null +++ b/cmake/ld.lld.in @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +# This is a workaround for bug in llvm/clang, +# that does not produce .debug_aranges with LTO +# +# NOTE: this is a temporary solution, that should be removed once [1] will be +# resolved. +# +# [1]: https://discourse.llvm.org/t/clang-does-not-produce-full-debug-aranges-section-with-thinlto/64898/8 + +# NOTE: only -flto=thin is supported. +# NOTE: it is not possible to check was there -gdwarf-aranges initially or not. +if [[ "$*" =~ -plugin-opt=thinlto ]]; then + exec "@LLD_PATH@" -mllvm -generate-arange-section "$@" +else + exec "@LLD_PATH@" "$@" +fi diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 5b005a6f1f9..57d39899a40 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -94,8 +94,13 @@ if (LINKER_NAME) if (NOT LLD_PATH) message (FATAL_ERROR "Using linker ${LINKER_NAME} but can't find its path.") endif () - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --ld-path=${LLD_PATH}") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LLD_PATH}") + + # This a temporary quirk to emit .debug_aranges with ThinLTO + set (LLD_WRAPPER "${CMAKE_CURRENT_BINARY_DIR}/ld.lld") + configure_file ("${CMAKE_CURRENT_SOURCE_DIR}/cmake/ld.lld.in" "${LLD_WRAPPER}" @ONLY) + + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --ld-path=${LLD_WRAPPER}") + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LLD_WRAPPER}") else () set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}") diff --git a/tests/queries/0_stateless/02420_stracktrace_debug_symbols.reference b/tests/queries/0_stateless/02420_stracktrace_debug_symbols.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02420_stracktrace_debug_symbols.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02420_stracktrace_debug_symbols.sh b/tests/queries/0_stateless/02420_stracktrace_debug_symbols.sh new file mode 100755 index 00000000000..9b647ec984b --- /dev/null +++ b/tests/queries/0_stateless/02420_stracktrace_debug_symbols.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# NOTE: that this test uses stacktrace instead of addressToLineWithInlines() or +# similar, since that code (use / might use) different code path in Dwarf +# parser. +# +# Also note, that to rely on this test one should assume that CI packages uses +# ThinLTO builds. + +$CLICKHOUSE_LOCAL --stacktrace -q 'select throwIf(1)' |& grep -c 'Common/Exception.cpp:[0-9]*: DB::Exception::Exception' From 45afaa6fb8b5166e0770e7e54525af0e810323fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Sep 2022 18:31:22 +0200 Subject: [PATCH 325/582] Fix loading external symbols Signed-off-by: Azat Khuzhin --- src/Common/SymbolIndex.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 46d9e8dbd5c..c02d17a381e 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -37,7 +37,7 @@ But because ClickHouse is linked with most of the symbols exported (-rdynamic fl It allows to get source file names and line numbers from addresses. Only available if you use -g option for compiler. It is also used by default for ClickHouse builds, but because of its weight (about two gigabytes) it is split to separate binary and provided in clickhouse-common-static-dbg package. -This separate binary is placed in /usr/lib/debug/usr/bin/clickhouse and is loaded automatically by tools like gdb, addr2line. +This separate binary is placed in /usr/lib/debug/usr/bin/clickhouse.debug and is loaded automatically by tools like gdb, addr2line. When you build ClickHouse by yourself, debug info is not split and present in a single huge binary. What ClickHouse is using to provide good stack traces? @@ -391,6 +391,7 @@ void collectSymbolsFromELF( std::filesystem::path local_debug_info_path = canonical_path.parent_path() / canonical_path.stem(); local_debug_info_path += ".debug"; std::filesystem::path debug_info_path = std::filesystem::path("/usr/lib/debug") / canonical_path.relative_path(); + debug_info_path += ".debug"; if (std::filesystem::exists(local_debug_info_path)) object_name = local_debug_info_path; From 26f1c1504a6291403181a17df9f1665585da3b6f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Sep 2022 18:52:48 +0200 Subject: [PATCH 326/582] Do not compress debug sections (internal DWARF parser cannot handle such) Although this increase debug symbol size from 510MB to 1.8GB, but it is not a problem for packages, since they are compressed anyway. Checked deb package, and size slightly increased though, 834M -> 962M. Signed-off-by: Azat Khuzhin --- cmake/split_debug_symbols.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/split_debug_symbols.cmake b/cmake/split_debug_symbols.cmake index 12182ed9c20..a9c2158359a 100644 --- a/cmake/split_debug_symbols.cmake +++ b/cmake/split_debug_symbols.cmake @@ -20,7 +20,7 @@ macro(clickhouse_split_debug_symbols) COMMAND mkdir -p "${STRIP_DESTINATION_DIR}/bin" COMMAND cp "${STRIP_BINARY_PATH}" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" # Splits debug symbols into separate file, leaves the binary untouched: - COMMAND "${OBJCOPY_PATH}" --only-keep-debug --compress-debug-sections "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" + COMMAND "${OBJCOPY_PATH}" --only-keep-debug "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check: COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" From 597197c01d0c7bd7c132133ebe2d210d9ab6609c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Sep 2022 09:03:39 +0200 Subject: [PATCH 327/582] Improve error messages in Elf loader Signed-off-by: Azat Khuzhin --- src/Common/Elf.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/Elf.cpp b/src/Common/Elf.cpp index b735367b179..0515cc0765a 100644 --- a/src/Common/Elf.cpp +++ b/src/Common/Elf.cpp @@ -22,13 +22,13 @@ Elf::Elf(const std::string & path) /// Check if it's an elf. elf_size = in.buffer().size(); if (elf_size < sizeof(ElfEhdr)) - throw Exception("The size of supposedly ELF file is too small", ErrorCodes::CANNOT_PARSE_ELF); + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The size of supposedly ELF file '{}' is too small", path); mapped = in.buffer().begin(); header = reinterpret_cast(mapped); if (memcmp(header->e_ident, "\x7F""ELF", 4) != 0) - throw Exception("The file is not ELF according to magic", ErrorCodes::CANNOT_PARSE_ELF); + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The file '{}' is not ELF according to magic", path); /// Get section header. ElfOff section_header_offset = header->e_shoff; @@ -37,7 +37,7 @@ Elf::Elf(const std::string & path) if (!section_header_offset || !section_header_num_entries || section_header_offset + section_header_num_entries * sizeof(ElfShdr) > elf_size) - throw Exception("The ELF is truncated (section header points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The ELF '{}' is truncated (section header points after end of file)", path); section_headers = reinterpret_cast(mapped + section_header_offset); @@ -48,11 +48,11 @@ Elf::Elf(const std::string & path) }); if (!section_names_strtab) - throw Exception("The ELF doesn't have string table with section names", ErrorCodes::CANNOT_PARSE_ELF); + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The ELF '{}' doesn't have string table with section names", path); ElfOff section_names_offset = section_names_strtab->header.sh_offset; if (section_names_offset >= elf_size) - throw Exception("The ELF is truncated (section names string table points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The ELF '{}' is truncated (section names string table points after end of file)", path); section_names = reinterpret_cast(mapped + section_names_offset); @@ -64,7 +64,7 @@ Elf::Elf(const std::string & path) if (!program_header_offset || !program_header_num_entries || program_header_offset + program_header_num_entries * sizeof(ElfPhdr) > elf_size) - throw Exception("The ELF is truncated (program header points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The ELF '{}' is truncated (program header points after end of file)", path); program_headers = reinterpret_cast(mapped + program_header_offset); } From 9479e2143fa42d6ec7fb6703b3483a054a58cf7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Sep 2022 14:37:54 +0200 Subject: [PATCH 328/582] Do not try to load empty debug files This will avoid CANNOT_PARSE_ELF error for builds that has empty debug file in clickhouse-common-static-dbg package, i.e. debug build. Signed-off-by: Azat Khuzhin --- src/Common/SymbolIndex.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index c02d17a381e..e217d23cc27 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -393,9 +393,20 @@ void collectSymbolsFromELF( std::filesystem::path debug_info_path = std::filesystem::path("/usr/lib/debug") / canonical_path.relative_path(); debug_info_path += ".debug"; - if (std::filesystem::exists(local_debug_info_path)) + /// NOTE: This is a workaround for current package system. + /// + /// Since nfpm cannot copy file only if it exists, + /// and so in cmake empty .debug file is created instead, + /// but if we will try to load empty Elf file, then the CANNOT_PARSE_ELF + /// exception will be thrown from the Elf::Elf. + auto exists_not_empty = [](const std::filesystem::path & path) + { + return std::filesystem::exists(path) && !std::filesystem::is_empty(path); + }; + + if (exists_not_empty(local_debug_info_path)) object_name = local_debug_info_path; - else if (std::filesystem::exists(debug_info_path)) + else if (exists_not_empty(debug_info_path)) object_name = debug_info_path; else if (build_id.size() >= 2) { @@ -413,7 +424,7 @@ void collectSymbolsFromELF( std::filesystem::path build_id_debug_info_path( fmt::format("/usr/lib/debug/.build-id/{}/{}.debug", build_id_hex.substr(0, 2), build_id_hex.substr(2))); - if (std::filesystem::exists(build_id_debug_info_path)) + if (exists_not_empty(build_id_debug_info_path)) object_name = build_id_debug_info_path; else object_name = canonical_path; From c6cbd981b6f97db2c9b5ffd42495ba2fa698ff3d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Sep 2022 23:20:51 +0200 Subject: [PATCH 329/582] tests: disable 02161_addressToLineWithInlines addressToLineWithInlines() may lead to the following error: Code: 465. DB::Exception: Received from localhost:9000. DB::Exception: could not find abbreviation code: while executing 'FUNCTION addressToLineWithInlines(arrayJoin(trace) :: 1) -> addressToLineWithInlines(arrayJoin(trace)) Array(String) : 0'. (CANNOT_PARSE_DWARF) CI: https://s3.amazonaws.com/clickhouse-test-reports/40873/45fd2bcb218ace3231a026eb91d688f0093c6407/stateless_tests__release_.html Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02161_addressToLineWithInlines.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql index b6b497b4b55..e4624fffd48 100644 --- a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql +++ b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql @@ -1,4 +1,5 @@ --- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64 +-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64, disabled +-- Tag disabled: Parsing inlines may lead to "could not find abbreviation code" (FIXME) SET allow_introspection_functions = 0; SELECT addressToLineWithInlines(1); -- { serverError 446 } From ec68ed8a9340873eb69df2c8100718125fe1b9da Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Sat, 3 Sep 2022 10:44:07 +0800 Subject: [PATCH 330/582] fix test Signed-off-by: Lloyd-Pottiger --- .../test_rocksdb_read_only/test.py | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_rocksdb_read_only/test.py b/tests/integration/test_rocksdb_read_only/test.py index 25cd3125e39..dcbfa417bff 100644 --- a/tests/integration/test_rocksdb_read_only/test.py +++ b/tests/integration/test_rocksdb_read_only/test.py @@ -38,13 +38,7 @@ def test_read_only(start_cluster): INSERT INTO test (key, value) VALUES (0, 'a'), (1, 'b'), (2, 'c'); """ ) - # fail if create multiple tables on the same directory - with pytest.raises(QueryRuntimeException): - node.query( - """ - CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); - """ - ) + # fail if create multiple non-read-only tables on the same directory with pytest.raises(QueryRuntimeException): node.query( """ @@ -57,12 +51,19 @@ def test_read_only(start_cluster): CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only') PRIMARY KEY(key); """ ) - with pytest.raises(QueryRuntimeException): - node.query( - """ - CREATE TABLE test_fail (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + # success if create multiple read-only tables on the same directory + node.query( """ - ) + CREATE TABLE test_1 (key UInt64, value String) Engine=EmbeddedRocksDB(0, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + DROP TABLE test_1; + """ + ) + node.query( + """ + CREATE TABLE test_2 (key UInt64, value String) Engine=EmbeddedRocksDB(10, '/var/lib/clickhouse/store/test_rocksdb_read_only', 1) PRIMARY KEY(key); + DROP TABLE test_2; + """ + ) # success if create table on existing directory with no other tables on it node.query( """ From 1768a82a53fa9a132d497492b21d51e1a0cb74bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Sep 2022 06:45:18 +0200 Subject: [PATCH 331/582] git checkout c4b8137d31e8b6093dba1ad42355a328bfdf84df docs/en/getting-started/example-datasets/uk-price-paid.md --- .../example-datasets/uk-price-paid.md | 129 +++++++++++------- 1 file changed, 77 insertions(+), 52 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 5ba3dc0e38c..bb9046397a0 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,18 +6,28 @@ title: "UK Property Price Paid" --- The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 278 MiB in ClickHouse. -Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads
+Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. +## Download the Dataset {#download-dataset} + +Run the command: + +```bash +wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv +``` + +Download will take about 2 minutes with good internet connection. + ## Create the Table {#create-table} ```sql CREATE TABLE uk_price_paid ( - uuid UUID, price UInt32, date Date, postcode1 LowCardinality(String), @@ -32,67 +42,66 @@ CREATE TABLE uk_price_paid town LowCardinality(String), district LowCardinality(String), county LowCardinality(String), - category UInt8, - category2 UInt8 -) ORDER BY (postcode1, postcode2, addr1, addr2); + category UInt8 +) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); ``` ## Preprocess and Import Data {#preprocess-import-data} -In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with either `clickhouse-client` or the web based Play UI. +We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. + +In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. The preprocessing is: -- splitting the postcode to two different columns `postcode1` and `postcode2` that are better for storage and queries; +- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; - coverting the `time` field to date as it only contains 00:00 time; - ignoring the [UUid](../../sql-reference/data-types/uuid.md) field because we don't need it for analysis; - transforming `type` and `duration` to more readable Enum fields with function [transform](../../sql-reference/functions/other-functions.md#transform); - transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to [UInt8](../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) field with 0 and 1. +Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. + ```bash -INSERT INTO uk_price_paid -WITH - splitByChar(' ', postcode) AS p -SELECT - replaceRegexpAll(uuid_string, '{|}','') AS uuid, - toUInt32(price_string) AS price, - parseDateTimeBestEffortUS(time) AS date, - p[1] AS postcode1, - p[2] AS postcode2, - transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, - b = 'Y' AS is_new, - transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, - addr1, - addr2, - street, - locality, - town, - district, - county, - d = 'B' AS category, - e = 'B' AS category2 -FROM url( - 'http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv', - 'CSV', - 'uuid_string String, - price_string String, - time String, - postcode String, - a String, - b String, - c String, - addr1 String, - addr2 String, - street String, - locality String, - town String, - district String, - county String, - d String, - e String' -) -SETTINGS max_http_get_redirects=1; +clickhouse-local --input-format CSV --structure ' + uuid String, + price UInt32, + time DateTime, + postcode String, + a String, + b String, + c String, + addr1 String, + addr2 String, + street String, + locality String, + town String, + district String, + county String, + d String, + e String +' --query " + WITH splitByChar(' ', postcode) AS p + SELECT + price, + toDate(time) AS date, + p[1] AS postcode1, + p[2] AS postcode2, + transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, + b = 'Y' AS is_new, + transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, + addr1, + addr2, + street, + locality, + town, + district, + county, + d = 'B' AS category + FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" ``` +It will take about 40 seconds. + ## Validate the Data {#validate-data} Query: @@ -103,12 +112,28 @@ SELECT count() FROM uk_price_paid; Result: -```response +```text ┌──count()─┐ -│ 27450499 │ +│ 26321785 │ └──────────┘ ``` +The size of dataset in ClickHouse is just 278 MiB, check it. + +Query: + +```sql +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid'; +``` + +Result: + +```text +┌─formatReadableSize(total_bytes)─┐ +│ 278.80 MiB │ +└─────────────────────────────────┘ +``` + ## Run Some Queries {#run-queries} ### Query 1. Average Price Per Year {#average-price} @@ -121,7 +146,7 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, Result: -```response +```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ From 385427ade826e5d408ff82fd569b76a92b6cf239 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 3 Sep 2022 17:20:02 +0000 Subject: [PATCH 332/582] Fix formatting of notes box in documentation Follow-up to PR #38435 --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/ru/sql-reference/functions/date-time-functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 52f9a06df72..3515e903adf 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -267,7 +267,7 @@ Result: └────────────────┘ ``` -:::Attention +:::note The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday` functions described below is `Date` or `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. In case argument is out of normal range: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 63386bb32b9..1c623cd1dab 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -267,7 +267,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; └────────────────┘ ``` -:::Attention +:::note Тип возвращаемого описанными далее функциями `toStartOf*`, `toMonday` значения - `Date` или `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. Возвращаемые значения для значений вне нормального диапазона: @@ -277,7 +277,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; * `2149-05-31` будет результатом функции `toLastDayOfMonth` при обработке аргумента больше `2149-05-31`. ::: -:::Attention +:::note Тип возвращаемого описанными далее функциями `toStartOf*`, `toLastDayOfMonth`, `toMonday` значения - `Date` или `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. Возвращаемые значения для значений вне нормального диапазона: From f2de8ff8ff7768127c95a5796bc750034f5aedc0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 3 Sep 2022 19:42:47 -0300 Subject: [PATCH 333/582] Doc. commpressions http.md (#40959) --- docs/en/interfaces/http.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 036fcde6d7a..c980bc65152 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -175,6 +175,10 @@ You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP - `br` - `deflate` - `xz` +- `zstd` +- `lz4` +- `bz2` +- `snappy` To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. In order for ClickHouse to compress the response, enable compression with [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting and append `Accept-Encoding: compression_method` header to the request. You can configure the data compression level in the [http_zlib_compression_level](../operations/settings/settings.md#settings-http_zlib_compression_level) setting for all compression methods. From dd19b0856ea6e1d039035b1b7053256c334b3775 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 3 Sep 2022 19:43:39 -0300 Subject: [PATCH 334/582] Doc. mapApply, mapFilter, mapUpdate (#40961) * Update tuple-map-functions.md * Update tuple-map-functions.md --- .../functions/tuple-map-functions.md | 114 ++++++++++++++++++ 1 file changed, 114 insertions(+) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index c555a838927..52023df4d72 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -430,5 +430,119 @@ Result: └────────────────────────────┘ ``` +## mapApply + +**Syntax** + +```sql +mapApply(func, map) +``` + +**Parameters** + +- `func` - [Lamda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `map` — [Map](../../sql-reference/data-types/map.md). + +**Returned value** + +- Returns a map obtained from the original map by application of `func(map1[i], …, mapN[i])` for each element. + +**Example** + +Query: + +```sql +SELECT mapApply((k, v) -> (k, v * 10), _map) AS r +FROM +( + SELECT map('key1', number, 'key2', number * 2) AS _map + FROM numbers(3) +) +``` + +Result: + +```text +┌─r─────────────────────┐ +│ {'key1':0,'key2':0} │ +│ {'key1':10,'key2':20} │ +│ {'key1':20,'key2':40} │ +└───────────────────────┘ +``` + +## mapFilter + +**Syntax** + +```sql +mapFilter(func, map) +``` + +**Parameters** + +- `func` - [Lamda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `map` — [Map](../../sql-reference/data-types/map.md). + +**Returned value** + +- Returns a map containing only the elements in `map` for which `func(map1[i], …, mapN[i])` returns something other than 0. + + +**Example** + +Query: + +```sql +SELECT mapFilter((k, v) -> ((v % 2) = 0), _map) AS r +FROM +( + SELECT map('key1', number, 'key2', number * 2) AS _map + FROM numbers(3) +) +``` + +Result: + +```text +┌─r───────────────────┐ +│ {'key1':0,'key2':0} │ +│ {'key2':2} │ +│ {'key1':2,'key2':4} │ +└─────────────────────┘ +``` + + +## mapUpdate + +**Syntax** + +```sql +mapUpdate(map1, map2) +``` + +**Parameters** + +- `map1` [Map](../../sql-reference/data-types/map.md). +- `map2` [Map](../../sql-reference/data-types/map.md). + +**Returned value** + +- Returns a map1 with values updated of values for the corresponding keys in map2. + +**Example** + +Query: + +```sql +SELECT mapUpdate(map('key1', 0, 'key3', 0), map('key1', 10, 'key2', 10)) AS map; +``` + +Result: + +```text +┌─map────────────────────────────┐ +│ {'key3':0,'key1':10,'key2':10} │ +└────────────────────────────────┘ +``` [Original article](https://clickhouse.com/docs/en/sql-reference/functions/tuple-map-functions/) From 22f7bfdcb5e7f686868a1329993e8f07cb148898 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 03:42:58 +0300 Subject: [PATCH 335/582] Update gtest_sensitive_data_masker.cpp --- src/Common/tests/gtest_sensitive_data_masker.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index 37f835e2884..b8eb270090b 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -227,7 +227,4 @@ TEST(Common, SensitiveDataMasker) masker_xml_based.printStats(); #endif } - - - } From 68bf3b70773b76b98eae3a35838031b09c5fd348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 02:53:14 +0200 Subject: [PATCH 336/582] Fix bad test --- .../0_stateless/02277_full_sort_join_misc.sql | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.sql b/tests/queries/0_stateless/02277_full_sort_join_misc.sql index b4e3882edaf..4297f532b98 100644 --- a/tests/queries/0_stateless/02277_full_sort_join_misc.sql +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.sql @@ -1,24 +1,24 @@ SET join_algorithm = 'full_sorting_merge'; -SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 USING key; +SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 USING key ORDER BY key; -SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key); +SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key) ORDER BY key; -SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key); +SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key) ORDER BY key; -SELECT * FROM (SELECT 1 :: Int32 as key) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 1 :: Int32 as key) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT -1 :: Nullable(Int32) as key) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT -1 :: Nullable(Int32) as key) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: Nullable(String) AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: Nullable(String) AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: LowCardinality(String) AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: LowCardinality(String) AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT 5 == count() FROM (SELECT number as a from numbers(5)) as t1 LEFT JOIN (SELECT number as b from numbers(5) WHERE number > 100) as t2 ON t1.a = t2.b; -SELECT 5 == count() FROM (SELECT number as a from numbers(5) WHERE number > 100) as t1 RIGHT JOIN (SELECT number as b from numbers(5)) as t2 ON t1.a = t2.b; +SELECT 5 == count() FROM (SELECT number as a from numbers(5)) as t1 LEFT JOIN (SELECT number as b from numbers(5) WHERE number > 100) as t2 ON t1.a = t2.b ORDER BY 1; +SELECT 5 == count() FROM (SELECT number as a from numbers(5) WHERE number > 100) as t1 RIGHT JOIN (SELECT number as b from numbers(5)) as t2 ON t1.a = t2.b ORDER BY 1; From 7c3c367e013b11764581a5d092c0d65fe1b5e06c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 04:18:57 +0300 Subject: [PATCH 337/582] Update tips.md --- docs/en/operations/tips.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 85927cd0e05..facf78c85bf 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -74,13 +74,16 @@ Make sure that [`fstrim`](https://en.wikipedia.org/wiki/Trim_(computing)) is ena ## File System {#file-system} -Ext4 is the most reliable option. Set the mount options `noatime`. -XFS should be avoided. It works mostly fine but there are some reports about lower performance. +Ext4 is the most reliable option. Set the mount options `noatime`. XFS works well too. Most other file systems should also work fine. +FAT-32 and exFAT are not supported due to lack of hard links. + Do not use compressed filesystems, because ClickHouse does compression on its own and better. It's not recommended to use encrypted filesystems, because you can use builtin encryption in ClickHouse, which is better. +While ClickHouse can work over NFS, it is not the best idea. + ## Linux Kernel {#linux-kernel} Don’t use an outdated Linux kernel. From 660c1439ecf3958c74173c3104f736653e13bc8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 03:27:55 +0200 Subject: [PATCH 338/582] Fix build --- .../tests/gtest_getMultipleValuesFromConfig.cpp | 2 +- src/Common/tests/gtest_sensitive_data_masker.cpp | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp index 11e9282a1c0..51c28ed43e9 100644 --- a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp +++ b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp @@ -10,7 +10,7 @@ using namespace DB; TEST(Common, getMultipleValuesFromConfig) { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( 0 1 diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index 4eedadff92c..0dc2f895145 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -103,7 +103,7 @@ TEST(Common, SensitiveDataMasker) { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( hide SSN @@ -153,7 +153,7 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( test @@ -166,7 +166,7 @@ TEST(Common, SensitiveDataMasker) )END"); - Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad); + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam); DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules"); ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl; @@ -182,13 +182,13 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( test )END"); - Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad); + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam); DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules"); ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl; @@ -204,13 +204,13 @@ TEST(Common, SensitiveDataMasker) try { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM - + xml_isteam(R"END( test())( )END"); - Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad); + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam); DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules"); ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl; From 1ae24b3e656024b2c633403918d77c6c640fee52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 04:07:10 +0200 Subject: [PATCH 339/582] Update spelling dictionary --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index fa178e764da..acb3fa0a0f0 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -251,6 +251,7 @@ dragonbox durations endian enum +exfat fastops fcoverage filesystem From 712de530335ed915b259100848ab931c503d9eb2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 04:15:45 +0200 Subject: [PATCH 340/582] Sharding s3 key names --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 15 ++++++++++++++- src/Disks/ObjectStorages/StoredObject.h | 1 + 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index ccde7d20778..369f6faddf8 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -31,6 +31,7 @@ #include #include + namespace DB { @@ -90,7 +91,19 @@ void logIfError(const Aws::Utils::Outcome & response, std::functi std::string S3ObjectStorage::generateBlobNameForPath(const std::string & /* path */) { - return getRandomASCIIString(32); + /// Path to store the new S3 object. + + /// Total length is 32 a-z characters for enough randomness. + /// First 3 characters are used as a prefix for + /// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/ + + constexpr size_t key_name_total_size = 32; + constexpr size_t key_name_prefix_size = 3; + + /// Path to store new S3 object. + return fmt::format("{}/{}", + getRandomASCIIString(key_name_prefix_size), + getRandomASCIIString(key_name_total_size - key_name_prefix_size)); } Aws::S3::Model::HeadObjectOutcome S3ObjectStorage::requestObjectHeadData(const std::string & bucket_from, const std::string & key) const diff --git a/src/Disks/ObjectStorages/StoredObject.h b/src/Disks/ObjectStorages/StoredObject.h index acb8a5fd127..d9faa766540 100644 --- a/src/Disks/ObjectStorages/StoredObject.h +++ b/src/Disks/ObjectStorages/StoredObject.h @@ -3,6 +3,7 @@ #include #include + namespace DB { From c4adc9ed8f837a9189f900479261bbeec50338c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 04:28:08 +0200 Subject: [PATCH 341/582] Remove trash --- docker/server/Dockerfile.ubuntu | 3 - docs/zh/development/tests.md | 341 ++++++++---------- .../data-types/lowcardinality.md | 3 +- docs/zh/sql-reference/statements/create.md | 2 - tests/ci/run_check.py | 2 - tests/ci/workflow_approve_rerun_lambda/app.py | 4 - tests/integration/test_ttl_move/test.py | 23 -- tests/integration/test_ttl_replicated/test.py | 10 - 8 files changed, 151 insertions(+), 237 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index b9e8b89cd92..f4102a6ccaf 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -31,9 +31,6 @@ ARG deb_location_url="" # set non-empty single_binary_location_url to create docker image # from a single binary url (useful for non-standard builds - with sanitizers, for arm64). -# for example (run on aarch64 server): -# docker build . --network host --build-arg single_binary_location_url="https://builds.clickhouse.com/master/aarch64/clickhouse" -t altinity/clickhouse-server:master-testing-arm -# note: clickhouse-odbc-bridge is not supported there. ARG single_binary_location_url="" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docs/zh/development/tests.md b/docs/zh/development/tests.md index ca9300597c7..e6d5cf66de9 100644 --- a/docs/zh/development/tests.md +++ b/docs/zh/development/tests.md @@ -1,338 +1,297 @@ --- -slug: /zh/development/tests +slug: /en/development/tests +sidebar_position: 70 +sidebar_label: Testing +title: ClickHouse Testing +description: Most of ClickHouse features can be tested with functional tests and they are mandatory to use for every change in ClickHouse code that can be tested that way. --- -# ClickHouse 测试 {#clickhouse-testing} -## 功能测试 {#functional-tests} +## Functional Tests -功能测试使用起来最简单方便. 大多数 ClickHouse 特性都可以通过功能测试进行测试, 并且对于可以通过功能测试进行测试的 ClickHouse 代码的每一个更改, 都必须使用这些特性 +Functional tests are the most simple and convenient to use. Most of ClickHouse features can be tested with functional tests and they are mandatory to use for every change in ClickHouse code that can be tested that way. -每个功能测试都会向正在运行的 ClickHouse 服务器发送一个或多个查询, 并将结果与参考进行比较. +Each functional test sends one or multiple queries to the running ClickHouse server and compares the result with reference. -测试位于 `查询` 目录中. 有两个子目录: `无状态` 和 `有状态`. 无状态测试在没有任何预加载测试数据的情况下运行查询 - 它们通常在测试本身内即时创建小型合成数据集. 状态测试需要来自 Yandex.Metrica 的预加载测试数据, 它对公众开放. +Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -每个测试可以是两种类型之一: `.sql` 和 `.sh`. `.sql` 测试是简单的 SQL 脚本, 它通过管道传输到 `clickhouse-client --multiquery --testmode`. `.sh` 测试是一个自己运行的脚本. SQL 测试通常比 `.sh` 测试更可取. 仅当您必须测试某些无法从纯 SQL 中执行的功能时才应使用 `.sh` 测试, 例如将一些输入数据传送到 `clickhouse-client` 或测试 `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. -### 在本地运行测试 {#functional-test-locally} +### Running a Test Locally {#functional-test-locally} -在本地启动ClickHouse服务器, 监听默认端口(9000). 例如, 要运行测试 `01428_hash_set_nan_key`, 请切换到存储库文件夹并运行以下命令: +Start the ClickHouse server locally, listening on the default port (9000). To +run, for example, the test `01428_hash_set_nan_key`, change to the repository +folder and run the following command: ``` PATH=$PATH: tests/clickhouse-test 01428_hash_set_nan_key ``` -有关更多选项, 请参阅`tests/clickhouse-test --help`. 您可以简单地运行所有测试或运行由测试名称中的子字符串过滤的测试子集:`./clickhouse-test substring`. 还有并行或随机顺序运行测试的选项. +For more options, see `tests/clickhouse-test --help`. You can simply run all tests or run subset of tests filtered by substring in test name: `./clickhouse-test substring`. There are also options to run tests in parallel or in randomized order. -### 添加新测试 {#adding-new-test} +### Adding a New Test -添加新的测试, 在 `queries/0_stateless` 目录下创建 `.sql` 或 `.sh` 文件, 手动检查, 然后通过以下方式生成`.reference`文件:`clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` 或 `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. -测试应仅使用(创建、删除等)`test` 数据库中假定已预先创建的表; 测试也可以使用临时表. +Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. -### 选择测试名称 {#choosing-test-name} +### Choosing the Test Name -测试名称以五位数前缀开头, 后跟描述性名称, 例如 `00422_hash_function_constexpr.sql`. 要选择前缀, 请找到目录中已存在的最大前缀, 并将其加一. 在此期间, 可能会添加一些具有相同数字前缀的其他测试, 但这没关系并且不会导致任何问题, 您以后不必更改它. +The name of the test starts with a five-digit prefix followed by a descriptive name, such as `00422_hash_function_constexpr.sql`. To choose the prefix, find the largest prefix already present in the directory, and increment it by one. In the meantime, some other tests might be added with the same numeric prefix, but this is OK and does not lead to any problems, you don't have to change it later. -一些测试的名称中标有 `zookeeper`、`shard` 或 `long` . `zookeeper` 用于使用 ZooKeeper 的测试. `shard` 用于需要服务器监听 `127.0.0.*` 的测试; `distributed` 或 `global` 具有相同的含义. `long` 用于运行时间稍长于一秒的测试. Yo你可以分别使用 `--no-zookeeper`、`--no-shard` 和 `--no-long` 选项禁用这些测试组. 如果需要 ZooKeeper 或分布式查询,请确保为您的测试名称添加适当的前缀. +Some tests are marked with `zookeeper`, `shard` or `long` in their names. `zookeeper` is for tests that are using ZooKeeper. `shard` is for tests that requires server to listen `127.0.0.*`; `distributed` or `global` have the same meaning. `long` is for tests that run slightly longer that one second. You can disable these groups of tests using `--no-zookeeper`, `--no-shard` and `--no-long` options, respectively. Make sure to add a proper prefix to your test name if it needs ZooKeeper or distributed queries. -### 检查必须发生的错误 {#checking-error-must-occur} +### Checking for an Error that Must Occur -有时您想测试是否因不正确的查询而发生服务器错误. 我们支持在 SQL 测试中对此进行特殊注释, 形式如下: +Sometimes you want to test that a server error occurs for an incorrect query. We support special annotations for this in SQL tests, in the following form: ``` select x; -- { serverError 49 } ``` -此测试确保服务器返回关于未知列“x”的错误代码为 49. 如果没有错误, 或者错误不同, 则测试失败. 如果您想确保错误发生在客户端, 请改用 `clientError` 注释. +This test ensures that the server returns an error with code 49 about unknown column `x`. If there is no error, or the error is different, the test will fail. If you want to ensure that an error occurs on the client side, use `clientError` annotation instead. -不要检查错误消息的特定措辞, 它将来可能会发生变化, 并且测试将不必要地中断. 只检查错误代码. 如果现有的错误代码不足以满足您的需求, 请考虑添加一个新的. +Do not check for a particular wording of error message, it may change in the future, and the test will needlessly break. Check only the error code. If the existing error code is not precise enough for your needs, consider adding a new one. -### 测试分布式查询 {#testing-distributed-query} +### Testing a Distributed Query -如果你想在功能测试中使用分布式查询, 你可以使用 `127.0.0.{1..2}` 的地址, 以便服务器查询自己; 或者您可以在服务器配置文件中使用预定义的测试集群, 例如`test_shard_localhost`. 请记住在测试名称中添加 `shard` 或 `distributed` 字样, 以便它以正确的配置在 CI 中运行, 其中服务器配置为支持分布式查询. +If you want to use distributed queries in functional tests, you can leverage `remote` table function with `127.0.0.{1..2}` addresses for the server to query itself; or you can use predefined test clusters in server configuration file like `test_shard_localhost`. Remember to add the words `shard` or `distributed` to the test name, so that it is run in CI in correct configurations, where the server is configured to support distributed queries. -## 已知错误 {#known-bugs} +## Known Bugs {#known-bugs} -如果我们知道一些可以通过功能测试轻松重现的错误, 我们将准备好的功能测试放在 `tests/queries/bugs` 目录中. 修复错误后, 这些测试将移至 `tests/queries/0_stateless` . +If we know some bugs that can be easily reproduced by functional tests, we place prepared functional tests in `tests/queries/bugs` directory. These tests will be moved to `tests/queries/0_stateless` when bugs are fixed. -## 集成测试 {#integration-tests} +## Integration Tests {#integration-tests} -集成测试允许在集群配置中测试 ClickHouse 以及 ClickHouse 与其他服务器(如 MySQL、Postgres、MongoDB)的交互. 它们可以用来模拟网络分裂、丢包等情况. 这些测试在Docker下运行, 并使用各种软件创建多个容器. +Integration tests allow testing ClickHouse in clustered configuration and ClickHouse interaction with other servers like MySQL, Postgres, MongoDB. They are useful to emulate network splits, packet drops, etc. These tests are run under Docker and create multiple containers with various software. -有关如何运行这些测试, 请参阅 `tests/integration/README.md` . +See `tests/integration/README.md` on how to run these tests. -注意, ClickHouse与第三方驱动程序的集成没有经过测试. 另外, 我们目前还没有JDBC和ODBC驱动程序的集成测试. +Note that integration of ClickHouse with third-party drivers is not tested. Also, we currently do not have integration tests with our JDBC and ODBC drivers. -## 单元测试 {#unit-tests} +## Unit Tests {#unit-tests} -当您想测试的不是 ClickHouse 整体, 而是单个独立库或类时,单元测试很有用. 您可以使用 `ENABLE_TESTS` CMake 选项启用或禁用测试构建. 单元测试(和其他测试程序)位于代码中的 `tests` 子目录中. 要运行单元测试, 请键入 `ninja test` 。有些测试使用 `gtest` , 但有些程序在测试失败时会返回非零退出码. +Unit tests are useful when you want to test not the ClickHouse as a whole, but a single isolated library or class. You can enable or disable build of tests with `ENABLE_TESTS` CMake option. Unit tests (and other test programs) are located in `tests` subdirectories across the code. To run unit tests, type `ninja test`. Some tests use `gtest`, but some are just programs that return non-zero exit code on test failure. -如果代码已经被功能测试覆盖了, 就没有必要进行单元测试(而且功能测试通常更易于使用). +It’s not necessary to have unit tests if the code is already covered by functional tests (and functional tests are usually much more simple to use). -例如, 您可以通过直接调用可执行文件来运行单独的 gtest 检查: +You can run individual gtest checks by calling the executable directly, for example: ```bash $ ./src/unit_tests_dbms --gtest_filter=LocalAddress* ``` -## 性能测试 {#performance-tests} +## Performance Tests {#performance-tests} -性能测试允许测量和比较 ClickHouse 的某些孤立部分在合成查询上的性能. 测试位于 `tests/performance`. 每个测试都由带有测试用例描述的 `.xml` 文件表示. 测试使用 `docker/tests/performance-comparison` 工具运行. 请参阅自述文件以进行调用. +Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Performance tests are located at `tests/performance/`. Each test is represented by an `.xml` file with a description of the test case. Tests are run with `docker/test/performance-comparison` tool . See the readme file for invocation. -每个测试在循环中运行一个或多个查询(可能带有参数组合). 一些测试可以包含预加载测试数据集的先决条件. +Each test run one or multiple queries (possibly with combinations of parameters) in a loop. -如果您希望在某些场景中提高ClickHouse的性能,并且如果可以在简单的查询中观察到改进,那么强烈建议编写性能测试。在测试期间使用 `perf top` 或其他perf工具总是有意义的. +If you want to improve performance of ClickHouse in some scenario, and if improvements can be observed on simple queries, it is highly recommended to write a performance test. Also, it is recommended to write performance tests when you add or modify SQL functions which are relatively isolated and not too obscure. It always makes sense to use `perf top` or other `perf` tools during your tests. -## 测试工具和脚本 {#test-tools-and-scripts} +## Test Tools and Scripts {#test-tools-and-scripts} - `tests` 目录中的一些程序不是准备好的测试,而是测试工具. 例如, 对于 `Lexer`, 有一个工具 `src/Parsers/tests/lexer` , 它只是对标准输入进行标记化并将着色结果写入标准输出. 您可以将这些类型的工具用作代码示例以及用于探索和手动测试. +Some programs in `tests` directory are not prepared tests, but are test tools. For example, for `Lexer` there is a tool `src/Parsers/tests/lexer` that just do tokenization of stdin and writes colorized result to stdout. You can use these kind of tools as a code examples and for exploration and manual testing. -## 其他测试 {#miscellaneous-tests} +## Miscellaneous Tests {#miscellaneous-tests} -在 `tests/external_models` 中有机器学习模型的测试. 这些测试不会更新, 必须转移到集成测试. +There are tests for machine learned models in `tests/external_models`. These tests are not updated and must be transferred to integration tests. -仲裁插入有单独的测试. 该测试在不同的服务器上运行 ClickHouse 集群并模拟各种故障情况:网络分裂、丢包(ClickHouse 节点之间、ClickHouse 和 ZooKeeper 之间、ClickHouse 服务器和客户端之间等)、`kill -9`、`kill -STOP` 和 `kill -CONT` , 比如 [Jepsen](https://aphyr.com/tags/Jepsen). 然后测试检查所有已确认的插入是否已写入并且所有被拒绝的插入均未写入. +There is separate test for quorum inserts. This test run ClickHouse cluster on separate servers and emulate various failure cases: network split, packet drop (between ClickHouse nodes, between ClickHouse and ZooKeeper, between ClickHouse server and client, etc.), `kill -9`, `kill -STOP` and `kill -CONT` , like [Jepsen](https://aphyr.com/tags/Jepsen). Then the test checks that all acknowledged inserts was written and all rejected inserts was not. -在 ClickHouse 开源之前, Quorum 测试是由单独的团队编写的. 这个团队不再与ClickHouse合作. 测试碰巧是用Java编写的. 由于这些原因, 必须重写仲裁测试并将其转移到集成测试. +Quorum test was written by separate team before ClickHouse was open-sourced. This team no longer work with ClickHouse. Test was accidentally written in Java. For these reasons, quorum test must be rewritten and moved to integration tests. -## 手动测试 {#manual-testing} +## Manual Testing {#manual-testing} -当您开发一个新特性时, 手动测试它也是合理的. 您可以按照以下步骤进行操作: +When you develop a new feature, it is reasonable to also test it manually. You can do it with the following steps: -构建 ClickHouse. 从终端运行 ClickHouse:将目录更改为 `programs/clickhouse-server` 并使用 `./clickhouse-server` 运行它. 默认情况下, 它将使用当前目录中的配置(`config.xml`、`users.xml` 和`config.d` 和`users.d` 目录中的文件). 要连接到 ClickHouse 服务器, 请运行 `programs/clickhouse-client/clickhouse-client` . +Build ClickHouse. Run ClickHouse from the terminal: change directory to `programs/clickhouse-server` and run it with `./clickhouse-server`. It will use configuration (`config.xml`, `users.xml` and files within `config.d` and `users.d` directories) from the current directory by default. To connect to ClickHouse server, run `programs/clickhouse-client/clickhouse-client`. -请注意, 所有 clickhouse 工具(服务器、客户端等)都只是指向名为 `clickhouse` 的单个二进制文件的符号链接. 你可以在 `programs/clickhouse` 找到这个二进制文件. 所有工具也可以作为 `clickhouse tool` 而不是 `clickhouse-tool` 调用. +Note that all clickhouse tools (server, client, etc) are just symlinks to a single binary named `clickhouse`. You can find this binary at `programs/clickhouse`. All tools can also be invoked as `clickhouse tool` instead of `clickhouse-tool`. -或者, 您可以安装 ClickHouse 包: 从 Yandex 存储库稳定发布, 或者您可以在 ClickHouse 源根目录中使用 `./release` 为自己构建包. 然后使用 `sudo service clickhouse-server start` 启动服务器(或停止以停止服务器). 在 `/etc/clickhouse-server/clickhouse-server.log` 中查找日志. +Alternatively you can install ClickHouse package: either stable release from ClickHouse repository or you can build package for yourself with `./release` in ClickHouse sources root. Then start the server with `sudo clickhouse start` (or stop to stop the server). Look for logs at `/etc/clickhouse-server/clickhouse-server.log`. -当您的系统上已经安装了 ClickHouse 时,您可以构建一个新的 `clickhouse` 二进制文件并替换现有的二进制文件: +When ClickHouse is already installed on your system, you can build a new `clickhouse` binary and replace the existing binary: ``` bash -$ sudo service clickhouse-server stop +$ sudo clickhouse stop $ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start +$ sudo clickhouse start ``` -您也可以停止系统 clickhouse-server 并使用相同的配置运行您自己的服务器, 但登录到终端: +Also you can stop system clickhouse-server and run your own with the same configuration but with logging to terminal: ``` bash -$ sudo service clickhouse-server stop +$ sudo clickhouse stop $ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml ``` -使用 gdb 的示例: +Example with gdb: ``` bash $ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml ``` -如果系统 clickhouse-server 已经在运行并且你不想停止它, 你可以在你的 `config.xml` 中更改端口号(或在 `config.d` 目录中的文件中覆盖它们), 提供适当的数据路径, 并运行它. +If the system clickhouse-server is already running and you do not want to stop it, you can change port numbers in your `config.xml` (or override them in a file in `config.d` directory), provide appropriate data path, and run it. -`clickhouse` 二进制文件几乎没有依赖关系, 可以在广泛的 Linux 发行版中使用. 要在服务器上快速而肮脏地测试您的更改, 您可以简单地将新构建的 `clickhouse` 二进制文件 `scp` 到您的服务器, 然后按照上面的示例运行它. +`clickhouse` binary has almost no dependencies and works across wide range of Linux distributions. To quick and dirty test your changes on a server, you can simply `scp` your fresh built `clickhouse` binary to your server and then run it as in examples above. -## 测试环境 {#testing-environment} +## Build Tests {#build-tests} -在发布稳定版之前, 我们将其部署在测试环境中.测试环境是一个集群,处理 [Yandex.Metrica](https://metrica.yandex.com/) 数据的 1/39 部分. 我们与 Yandex.Metrica 团队共享我们的测试环境. ClickHouse无需在现有数据上停机即可升级. 我们首先看到的是, 数据被成功地处理了, 没有滞后于实时, 复制继续工作, Yandex.Metrica 团队没有发现任何问题. 第一次检查可以通过以下方式进行: +Build tests allow to check that build is not broken on various alternative configurations and on some foreign systems. These tests are automated as well. -``` sql -SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; -``` +Examples: +- cross-compile for Darwin x86_64 (Mac OS X) +- cross-compile for FreeBSD x86_64 +- cross-compile for Linux AArch64 +- build on Ubuntu with libraries from system packages (discouraged) +- build with shared linking of libraries (discouraged) -在某些情况下, 我们还会部署到 Yandex 中我们朋友团队的测试环境:Market、Cloud 等. 此外, 我们还有一些用于开发目的的硬件服务器. +For example, build with system packages is bad practice, because we cannot guarantee what exact version of packages a system will have. But this is really needed by Debian maintainers. For this reason we at least have to support this variant of build. Another example: shared linking is a common source of trouble, but it is needed for some enthusiasts. -## 负载测试 {#load-testing} +Though we cannot run all tests on all variant of builds, we want to check at least that various build variants are not broken. For this purpose we use build tests. -部署到测试环境后, 我们使用来自生产集群的查询运行负载测试. 这是手动完成的. +We also test that there are no translation units that are too long to compile or require too much RAM. -确保您在生产集群上启用了 `query_log`. +We also test that there are no too large stack frames. -收集一天或更长时间的查询日志: +## Testing for Protocol Compatibility {#testing-for-protocol-compatibility} -``` bash -$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv -``` +When we extend ClickHouse network protocol, we test manually that old clickhouse-client works with new clickhouse-server and new clickhouse-client works with old clickhouse-server (simply by running binaries from corresponding packages). -这是一个复杂的例子. `type = 2` 将过滤成功执行的查询. `query LIKE '%ym:%'` 是从 Yandex.Metrica 中选择相关查询. `is_initial_query` 是只选择客户端发起的查询, 而不是 ClickHouse 本身(作为分布式查询处理的一部分). +We also test some cases automatically with integrational tests: +- if data written by old version of ClickHouse can be successfully read by the new version; +- do distributed queries work in a cluster with different ClickHouse versions. -`scp` 将此日志记录到您的测试集群并按如下方式运行它: +## Help from the Compiler {#help-from-the-compiler} -``` bash -$ clickhouse benchmark --concurrency 16 < queries.tsv -``` +Main ClickHouse code (that is located in `dbms` directory) is built with `-Wall -Wextra -Werror` and with some additional enabled warnings. Although these options are not enabled for third-party libraries. -(可能你还想指定一个 `--user`) +Clang has even more useful warnings - you can look for them with `-Weverything` and pick something to default build. -然后把它留到晚上或周末, 去休息一下. +For production builds, clang is used, but we also test make gcc builds. For development, clang is usually more convenient to use. You can build on your own machine with debug mode (to save battery of your laptop), but please note that compiler is able to generate more warnings with `-O3` due to better control flow and inter-procedure analysis. When building with clang in debug mode, debug version of `libc++` is used that allows to catch more errors at runtime. -您应该检查 `clickhouse-server` 没有崩溃, 内存占用是有限的, 且性能不会随着时间的推移而降低. +## Sanitizers {#sanitizers} -由于查询和环境的高度可变性, 没有记录和比较精确的查询执行时间. +### Address sanitizer +We run functional, integration, stress and unit tests under ASan on per-commit basis. -## 构建测试 {#build-tests} +### Thread sanitizer +We run functional, integration, stress and unit tests under TSan on per-commit basis. -构建测试允许检查在各种可选配置和一些外部系统上的构建是否被破坏. 这些测试也是自动化的. +### Memory sanitizer +We run functional, integration, stress and unit tests under MSan on per-commit basis. -示例: -- Darwin x86_64 (Mac OS X) 交叉编译 -- FreeBSD x86_64 交叉编译 -- Linux AArch64 交叉编译 -- 使用系统包中的库在 Ubuntu 上构建(不鼓励) -- 使用库的共享链接构建(不鼓励) - -例如, 使用系统包构建是不好的做法, 因为我们无法保证系统将拥有哪个确切版本的包. 但这确实是 Debian 维护者所需要的. 出于这个原因, 我们至少必须支持这种构建变体. 另一个例子: 共享链接是一个常见的麻烦来源, 但对于一些爱好者来说是需要的. - -虽然我们无法对所有构建变体运行所有测试, 但我们希望至少检查各种构建变体没有被破坏. 为此, 我们使用构建测试. - -我们还测试了那些太长而无法编译或需要太多RAM的没有翻译单元. - -我们还测试没有太大的堆栈帧. - -## 协议兼容性测试 {#testing-for-protocol-compatibility} - -当我们扩展 ClickHouse 网络协议时, 我们手动测试旧的 clickhouse-client 与新的 clickhouse-server 一起工作, 而新的 clickhouse-client 与旧的 clickhouse-server 一起工作(只需从相应的包中运行二进制文件). - -我们还使用集成测试自动测试一些案例: -- 旧版本ClickHouse写入的数据是否可以被新版本成功读取; -- 在具有不同 ClickHouse 版本的集群中执行分布式查询. - -## 编译器的帮助 {#help-from-the-compiler} - -主要的 ClickHouse 代码(位于 `dbms` 目录中)是用 `-Wall -Wextra -Werror` 和一些额外的启用警告构建的. 虽然没有为第三方库启用这些选项. - -Clang 有更多有用的警告 - 你可以用 `-Weverything` 寻找它们并选择一些东西来默认构建. - -对于生产构建, 使用 clang, 但我们也测试 make gcc 构建. 对于开发, clang 通常使用起来更方便. 您可以使用调试模式在自己的机器上构建(以节省笔记本电脑的电池), 但请注意, 由于更好的控制流和过程间分析, 编译器能够使用 `-O3` 生成更多警告. 在调试模式下使用 clang 构建时, 使用调试版本的 `libc++` 允许在运行时捕获更多错误. - -## 地址清理器 {#sanitizers} - -### 地址清理器 -我们在ASan上运行功能测试、集成测试、压力测试和单元测试. - -### 线程清理器 -我们在TSan下运行功能测试、集成测试、压力测试和单元测试. - -### 内存清理器 -我们在MSan上运行功能测试、集成测试、压力测试和单元测试. - -### 未定义的行为清理器 -我们在UBSan下运行功能测试、集成测试、压力测试和单元测试. 某些第三方库的代码未针对 UB 进行清理. +### Undefined behaviour sanitizer +We run functional, integration, stress and unit tests under UBSan on per-commit basis. The code of some third-party libraries is not sanitized for UB. ### Valgrind (Memcheck) -我们曾经在 Valgrind 下通宵运行功能测试, 但不再这样做了. 这需要几个小时. 目前在`re2`库中有一个已知的误报, 见[这篇文章](https://research.swtch.com/sparse). +We used to run functional tests under Valgrind overnight, but don't do it anymore. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). -## 模糊测试 {#fuzzing} +## Fuzzing {#fuzzing} -ClickHouse 模糊测试是使用 [libFuzzer](https://llvm.org/docs/LibFuzzer.html) 和随机 SQL 查询实现的. 所有模糊测试都应使用sanitizers(地址和未定义)进行. +ClickHouse fuzzing is implemented both using [libFuzzer](https://llvm.org/docs/LibFuzzer.html) and random SQL queries. +All the fuzz testing should be performed with sanitizers (Address and Undefined). -LibFuzzer 用于库代码的隔离模糊测试. Fuzzer 作为测试代码的一部分实现, 并具有 `_fuzzer` 名称后缀. -Fuzzer 示例可以在 `src/Parsers/tests/lexer_fuzzer.cpp` 中找到. LibFuzzer 特定的配置、字典和语料库存储在 `tests/fuzz` 中. -我们鼓励您为处理用户输入的每个功能编写模糊测试. +LibFuzzer is used for isolated fuzz testing of library code. Fuzzers are implemented as part of test code and have “_fuzzer” name postfixes. +Fuzzer example can be found at `src/Parsers/fuzzers/lexer_fuzzer.cpp`. LibFuzzer-specific configs, dictionaries and corpus are stored at `tests/fuzz`. +We encourage you to write fuzz tests for every functionality that handles user input. -默认情况下不构建模糊器. 要构建模糊器, 应设置` -DENABLE_FUZZING=1` 和 `-DENABLE_TESTS=1` 选项. -我们建议在构建模糊器时禁用 Jemalloc. 用于将 ClickHouse fuzzing 集成到 Google OSS-Fuzz 的配置可以在 `docker/fuzz` 中找到. +Fuzzers are not built by default. To build fuzzers both `-DENABLE_FUZZING=1` and `-DENABLE_TESTS=1` options should be set. +We recommend to disable Jemalloc while building fuzzers. Configuration used to integrate ClickHouse fuzzing to +Google OSS-Fuzz can be found at `docker/fuzz`. -我们还使用简单的模糊测试来生成随机SQL查询, 并检查服务器在执行这些查询时是否会死亡. -你可以在 `00746_sql_fuzzy.pl` 中找到它. 这个测试应该连续运行(通宵或更长时间). +We also use simple fuzz test to generate random SQL queries and to check that the server does not die executing them. +You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (overnight and longer). -我们还使用复杂的基于 AST 的查询模糊器, 它能够找到大量的极端情况. 它在查询 AST 中进行随机排列和替换. 它会记住先前测试中的 AST 节点, 以使用它们对后续测试进行模糊测试, 同时以随机顺序处理它们. 您可以在 [这篇博客文章](https://clickhouse.com/blog/en/2021/fuzzing-clickhouse/) 中了解有关此模糊器的更多信息. +We also use sophisticated AST-based query fuzzer that is able to find huge amount of corner cases. It does random permutations and substitutions in queries AST. It remembers AST nodes from previous tests to use them for fuzzing of subsequent tests while processing them in random order. You can learn more about this fuzzer in [this blog article](https://clickhouse.com/blog/en/2021/fuzzing-clickhouse/). -## 压力测试 {#stress-test} +## Stress test -压力测试是另一种模糊测试. 它使用单个服务器以随机顺序并行运行所有功能测试. 不检查测试结果. +Stress tests are another case of fuzzing. It runs all functional tests in parallel in random order with a single server. Results of the tests are not checked. -经检查: -- 服务器不会崩溃,不会触发调试或清理程序陷阱; -- 没有死锁; -- 数据库结构一致; -- 服务器可以在测试后成功停止并重新启动,没有异常; +It is checked that: +- server does not crash, no debug or sanitizer traps are triggered; +- there are no deadlocks; +- the database structure is consistent; +- server can successfully stop after the test and start again without exceptions. -有五种变体 (Debug, ASan, TSan, MSan, UBSan). +There are five variants (Debug, ASan, TSan, MSan, UBSan). -## 线程模糊器 {#thread-fuzzer} +## Thread Fuzzer -Thread Fuzzer(请不要与 Thread Sanitizer 混淆)是另一种允许随机化线程执行顺序的模糊测试. 它有助于找到更多特殊情况. +Thread Fuzzer (please don't mix up with Thread Sanitizer) is another kind of fuzzing that allows to randomize thread order of execution. It helps to find even more special cases. -## 安全审计 {#security-audit} +## Security Audit -Yandex安全团队的人员从安全的角度对ClickHouse的功能做了一些基本的概述. +Our Security Team did some basic overview of ClickHouse capabilities from the security standpoint. -## 静态分析仪 {#static-analyzers} +## Static Analyzers {#static-analyzers} -我们在每次提交的基础上运行 `clang-tidy`. `clang-static-analyzer` 检查也被启用. `clang-tidy` 也用于一些样式检查. +We run `clang-tidy` on per-commit basis. `clang-static-analyzer` checks are also enabled. `clang-tidy` is also used for some style checks. -我们已经评估了 `clang-tidy`、`Coverity`、`cppcheck`、`PVS-Studio`、`tscancode`、`CodeQL`. 您将在 `tests/instructions/` 目录中找到使用说明. 你也可以阅读[俄文文章](https://habr.com/company/yandex/blog/342018/). +We have evaluated `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`, `CodeQL`. You will find instructions for usage in `tests/instructions/` directory. -如果你使用 `CLion` 作为 IDE, 你可以利用一些开箱即用的 `clang-tidy` 检查 +If you use `CLion` as an IDE, you can leverage some `clang-tidy` checks out of the box. -我们还使用 `shellcheck` 对shell脚本进行静态分析. +We also use `shellcheck` for static analysis of shell scripts. -## 硬化 {#hardening} +## Hardening {#hardening} -在调试版本中, 我们使用自定义分配器执行用户级分配的 ASLR. +In debug build we are using custom allocator that does ASLR of user-level allocations. -我们还手动保护在分配后预期为只读的内存区域. +We also manually protect memory regions that are expected to be readonly after allocation. -在调试构建中, 我们还需要对libc进行自定义, 以确保不会调用 "有害的" (过时的、不安全的、非线程安全的)函数. +In debug build we also involve a customization of libc that ensures that no "harmful" (obsolete, insecure, not thread-safe) functions are called. -Debug 断言被广泛使用. +Debug assertions are used extensively. -在调试版本中,如果抛出带有 "逻辑错误" 代码(暗示错误)的异常, 则程序会过早终止. 它允许在发布版本中使用异常, 但在调试版本中使其成为断言. +In debug build, if exception with "logical error" code (implies a bug) is being thrown, the program is terminated prematurely. It allows to use exceptions in release build but make it an assertion in debug build. -jemalloc 的调试版本用于调试版本. -libc++ 的调试版本用于调试版本. +Debug version of jemalloc is used for debug builds. +Debug version of libc++ is used for debug builds. -## 运行时完整性检查 +## Runtime Integrity Checks -对存储在磁盘上的数据是校验和. MergeTree 表中的数据同时以三种方式进行校验和*(压缩数据块、未压缩数据块、跨块的总校验和). 客户端和服务器之间或服务器之间通过网络传输的数据也会进行校验和. 复制确保副本上的数据位相同. +Data stored on disk is checksummed. Data in MergeTree tables is checksummed in three ways simultaneously* (compressed data blocks, uncompressed data blocks, the total checksum across blocks). Data transferred over network between client and server or between servers is also checksummed. Replication ensures bit-identical data on replicas. -需要防止硬件故障(存储介质上的位腐烂、服务器上 RAM 中的位翻转、网络控制器 RAM 中的位翻转、网络交换机 RAM 中的位翻转、客户端 RAM 中的位翻转、线路上的位翻转). 请注意,比特位操作很常见, 即使对于 ECC RAM 和 TCP 校验和(如果您每天设法运行数千台处理 PB 数据的服务器, 也可能发生比特位操作. [观看视频(俄语)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). +It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client, bit flips on the wire). Note that bit flips are common and likely to occur even for ECC RAM and in presence of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See the video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). -ClickHouse 提供诊断功能, 可帮助运维工程师找到故障硬件. +ClickHouse provides diagnostics that will help ops engineers to find faulty hardware. -\* 它并不慢. +\* and it is not slow. -## 代码风格 {#code-style} +## Code Style {#code-style} -[此处](style.md)描述了代码样式规则. +Code style rules are described [here](style.md). -要检查一些常见的样式违规,您可以使用 `utils/check-style` 脚本. +To check for some common style violations, you can use `utils/check-style` script. -要强制使用正确的代码样式, 您可以使用 `clang-format`. 文件 `.clang-format` 位于源根目录. 它大多与我们的实际代码风格相对应. 但是不建议将 `clang-format` 应用于现有文件, 因为它会使格式变得更糟. 您可以使用可以在 clang 源代码库中找到的 `clang-format-diff` 工具. +To force proper style of your code, you can use `clang-format`. File `.clang-format` is located at the sources root. It mostly corresponding with our actual code style. But it’s not recommended to apply `clang-format` to existing files because it makes formatting worse. You can use `clang-format-diff` tool that you can find in clang source repository. -或者, 您可以尝试使用 `uncrustify` 工具来重新格式化您的代码. 配置位于源根目录中的 `uncrustify.cfg` 中. 它比 `clang-format` 测试更少. +Alternatively you can try `uncrustify` tool to reformat your code. Configuration is in `uncrustify.cfg` in the sources root. It is less tested than `clang-format`. -`CLion` 有自己的代码格式化程序, 必须根据我们的代码风格进行调整. +`CLion` has its own code formatter that has to be tuned for our code style. -我们还使用 `codespell` 来查找代码中的拼写错误.它也是自动化的. +We also use `codespell` to find typos in code. It is automated as well. -## Metrica B2B 测试 {#metrica-b2b-tests} +## Test Coverage {#test-coverage} -每个 ClickHouse 版本都使用 Yandex Metrica 和 AppMetrica 引擎进行测试. ClickHouse 的测试版和稳定版部署在 VM 上, 并使用 Metrica 引擎的小副本运行, 该引擎处理输入数据的固定样本. 然后将两个 Metrica 引擎实例的结果放在一起比较. - -这些测试由单独的团队自动化. 由于移动部件数量众多, 测试在大多数情况下都因完全不相关的原因而失败, 这些原因很难弄清楚. 这些测试很可能对我们有负面价值. 尽管如此, 这些测试在数百次中被证明是有用的. - -## 测试覆盖率 {#test-coverage} - -我们还跟踪测试覆盖率, 但仅针对功能测试和 clickhouse-server. 它每天进行. +We also track test coverage but only for functional tests and only for clickhouse-server. It is performed on daily basis. ## Tests for Tests -有自动检测薄片测试. 它运行所有新测试100次(用于功能测试)或10次(用于集成测试). 如果至少有一次测试失败,它就被认为是脆弱的. +There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky. ## Testflows -[Testflows](https://testflows.com/) 是一个企业级的测试框架. Altinity 使用它进行一些测试, 我们在 CI 中运行这些测试. +[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to test a subset of ClickHouse. -## Yandex 检查 (only for Yandex employees) +## Test Automation {#test-automation} -这些检查将ClickHouse代码导入到Yandex内部的单一存储库中, 所以ClickHouse代码库可以被Yandex的其他产品(YT和YDB)用作库. 请注意, clickhouse-server本身并不是由内部回购构建的, Yandex应用程序使用的是未经修改的开源构建的. +We run tests with [GitHub Actions](https://github.com/features/actions). -## 测试自动化 {#test-automation} +Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored for several months. When you send a pull request on GitHub, we tag it as “can be tested” and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. -我们使用 Yandex 内部 CI 和名为 "Sandbox" 的作业自动化系统运行测试. +We do not use Travis CI due to the limit on time and computational power. +We do not use Jenkins. It was used before and now we are happy we are not using Jenkins. -在每次提交的基础上, 构建作业和测试都在沙箱中运行. 生成的包和测试结果发布在GitHub上, 可以通过直接链接下载. 产物要保存几个月. 当你在GitHub上发送一个pull请求时, 我们会把它标记为 "可以测试" , 我们的CI系统会为你构建ClickHouse包(发布、调试、使用地址清理器等). - -由于时间和计算能力的限制, 我们不使用 Travis CI. -我们不用Jenkins. 以前用过, 现在我们很高兴不用Jenkins了. - -[原始文章](https://clickhouse.com/docs/en/development/tests/) +[Original article](https://clickhouse.com/docs/en/development/tests/) diff --git a/docs/zh/sql-reference/data-types/lowcardinality.md b/docs/zh/sql-reference/data-types/lowcardinality.md index e089a7f9d41..717c3c979a4 100644 --- a/docs/zh/sql-reference/data-types/lowcardinality.md +++ b/docs/zh/sql-reference/data-types/lowcardinality.md @@ -55,6 +55,5 @@ ORDER BY id ## 参考 -- [高效低基数类型](https://www.altinity.com/blog/2019/3/27/low-cardinality). - [使用低基数类型减少ClickHouse的存储成本 – 来自Instana工程师的分享](https://www.instana.com/blog/reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer/). -- [字符优化 (俄语视频分享)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [英语分享](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). \ No newline at end of file +- [字符优化 (俄语视频分享)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [英语分享](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). diff --git a/docs/zh/sql-reference/statements/create.md b/docs/zh/sql-reference/statements/create.md index aef21a704b5..af77f4750b5 100644 --- a/docs/zh/sql-reference/statements/create.md +++ b/docs/zh/sql-reference/statements/create.md @@ -121,8 +121,6 @@ ENGINE = ... ``` -如果指定了编解ec,则默认编解码器不适用。 编解码器可以组合在一个流水线中,例如, `CODEC(Delta, ZSTD)`. 要为您的项目选择最佳的编解码器组合,请通过类似于Altinity中描述的基准测试 [新编码提高ClickHouse效率](https://www.altinity.com/blog/2019/7/new-encodings-to-improve-clickhouse) 文章. - !!! warning "警告" 您无法使用外部实用程序解压缩ClickHouse数据库文件,如 `lz4`. 相反,使用特殊的 [ツ环板compressorョツ嘉ッツ偲](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) 实用程序。 diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 5e6542f6e4c..0a269c07642 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -20,8 +20,6 @@ from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS NAME = "Run Check" TRUSTED_ORG_IDS = { - 7409213, # yandex - 28471076, # altinity 54801242, # clickhouse } diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 29f3271a34c..39bd9cfb283 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -50,8 +50,6 @@ WorkflowDescription = namedtuple( # See https://api.github.com/orgs/{name} TRUSTED_ORG_IDS = { - 7409213, # yandex - 28471076, # altinity 54801242, # clickhouse } @@ -104,8 +102,6 @@ TRUSTED_CONTRIBUTORS = { "kreuzerkrieg", "lehasm", # DOCSUP "michon470", # DOCSUP - "MyroTk", # Tester in Altinity - "myrrc", # Michael Kot, Altinity "nikvas0", "nvartolomei", "olgarev", # DOCSUP diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 49d7ab4f2fc..99978cbf6dc 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1284,19 +1284,6 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): def test_alter_multiple_ttls(started_cluster, name, engine, positive): name = unique_table_name(name) - """Copyright 2019, Altinity LTD - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License.""" """Check that when multiple TTL expressions are set and before any parts are inserted the TTL expressions are changed with ALTER command then all old @@ -1664,16 +1651,6 @@ def test_double_move_while_select(started_cluster, name, positive): def test_alter_with_merge_work(started_cluster, name, engine, positive): name = unique_table_name(name) - """Copyright 2019, Altinity LTD -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License.""" """Check that TTL expressions are re-evaluated for existing parts after ALTER command changes TTL expressions and parts are merged. diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index bcdb2d25912..cacd9ef0c78 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -342,16 +342,6 @@ def optimize_with_retry(node, table_name, retry=20): ], ) def test_ttl_alter_delete(started_cluster, name, engine): - """Copyright 2019, Altinity LTD - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License.""" """Check compatibility with old TTL delete expressions to make sure that: * alter modify of column's TTL delete expression works From 74023efa66e7313e331f02d881248e05f436733c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 05:29:37 +0300 Subject: [PATCH 342/582] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index acb3fa0a0f0..0a4aecad50e 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -251,7 +251,7 @@ dragonbox durations endian enum -exfat +exFAT fastops fcoverage filesystem From 64f95710cd192d02811d94e94631caa8d449d09c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Sep 2022 14:02:46 +0200 Subject: [PATCH 343/582] ci: add symlink for llvm-symbolizer (to make symbolizer work) Simply installing llvm-symbolizer-$VER is not enough, since it does not contain proper symblink, while LLVM is looking only for "llvm-symbolizer" (without version) - [1]: [1]: https://github.com/llvm/llvm-project/blob/c444af1c20b35555f2fdb2c1ca38d3f23b2faebd/compiler-rt/lib/sanitizer_common/sanitizer_symbolizer_posix_libcdep.cpp#L454 Follow-up for: #40655 Signed-off-by: Azat Khuzhin --- docker/packager/binary/Dockerfile | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index ba0a27c9801..b9b0c5c2c6c 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -83,5 +83,8 @@ RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ --yes --no-install-recommends \ && apt-get clean +# for external_symbolizer_path +RUN ln -s /usr/bin/llvm-symbolizer-15 /usr/bin/llvm-symbolizer + COPY build.sh / CMD ["bash", "-c", "/build.sh 2>&1"] From 2e85f9f0ade45d62b12c82eee93f410744e0611e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Aug 2022 22:26:10 +0200 Subject: [PATCH 344/582] Remove completely processed WAL files Previously all WAL files had been stored, though with the time of use this can take too much space on disk, and also the startup time will be increased. But it is pretty easy to prune old WAL files (the one parts from which had been completely written to disk already). Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 102 +++++++++++------- src/Storages/MergeTree/MergeTreeData.h | 3 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 18 +++- .../MergeTree/MergeTreeWriteAheadLog.h | 5 +- .../02410_inmemory_wal_cleanup.reference | 35 ++++++ .../02410_inmemory_wal_cleanup.sql | 27 +++++ 6 files changed, 145 insertions(+), 45 deletions(-) create mode 100644 tests/queries/0_stateless/02410_inmemory_wal_cleanup.reference create mode 100644 tests/queries/0_stateless/02410_inmemory_wal_cleanup.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 239451a16bb..6474ec5b37e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1177,14 +1177,10 @@ void MergeTreeData::loadDataPartsFromDisk( void MergeTreeData::loadDataPartsFromWAL( DataPartsVector & /* broken_parts_to_detach */, DataPartsVector & duplicate_parts_to_remove, - MutableDataPartsVector & parts_from_wal, - DataPartsLock & part_lock) + MutableDataPartsVector & parts_from_wal) { for (auto & part : parts_from_wal) { - if (getActiveContainingPart(part->info, DataPartState::Active, part_lock)) - continue; - part->modification_time = time(nullptr); /// Assume that all parts are Active, covered parts will be detected and marked as Outdated later part->setState(DataPartState::Active); @@ -1212,7 +1208,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto metadata_snapshot = getInMemoryMetadataPtr(); const auto settings = getSettings(); - MutableDataPartsVector parts_from_wal; Strings part_file_names; auto disks = getStoragePolicy()->getDisks(); @@ -1269,16 +1264,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// Collect part names by disk. std::map>> disk_part_map; - std::map disk_wal_part_map; ThreadPool pool(disks.size()); - std::mutex wal_init_lock; + for (const auto & disk_ptr : disks) { if (disk_ptr->isBroken()) continue; auto & disk_parts = disk_part_map[disk_ptr->getName()]; - auto & disk_wal_parts = disk_wal_part_map[disk_ptr->getName()]; pool.scheduleOrThrowOnError([&, disk_ptr]() { @@ -1291,34 +1284,11 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (!startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) disk_parts.emplace_back(std::make_pair(it->name(), disk_ptr)); - else if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) - { - std::lock_guard lock(wal_init_lock); - if (write_ahead_log != nullptr) - throw Exception( - "There are multiple WAL files appeared in current storage policy. You need to resolve this manually", - ErrorCodes::CORRUPTED_DATA); - - write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); - for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext())) - disk_wal_parts.push_back(std::move(part)); - } - else if (settings->in_memory_parts_enable_wal) - { - MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); - for (auto && part : wal.restore(metadata_snapshot, getContext())) - disk_wal_parts.push_back(std::move(part)); - } } }); } - pool.wait(); - for (auto & [_, disk_wal_parts] : disk_wal_part_map) - parts_from_wal.insert( - parts_from_wal.end(), std::make_move_iterator(disk_wal_parts.begin()), std::make_move_iterator(disk_wal_parts.end())); - size_t num_parts = 0; std::queue>> parts_queue; for (auto & [_, disk_parts] : disk_part_map) @@ -1332,13 +1302,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto part_lock = lockParts(); data_parts_indexes.clear(); - if (num_parts == 0 && parts_from_wal.empty()) - { - resetObjectColumnsFromActiveParts(part_lock); - LOG_DEBUG(log, "There are no data parts"); - return; - } - DataPartsVector broken_parts_to_detach; DataPartsVector duplicate_parts_to_remove; @@ -1346,8 +1309,65 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) loadDataPartsFromDisk( broken_parts_to_detach, duplicate_parts_to_remove, pool, num_parts, parts_queue, skip_sanity_checks, settings); - if (!parts_from_wal.empty()) - loadDataPartsFromWAL(broken_parts_to_detach, duplicate_parts_to_remove, parts_from_wal, part_lock); + if (settings->in_memory_parts_enable_wal) + { + std::map disk_wal_part_map; + + std::mutex wal_init_lock; + for (const auto & disk_ptr : disks) + { + if (disk_ptr->isBroken()) + continue; + + auto & disk_wal_parts = disk_wal_part_map[disk_ptr->getName()]; + + pool.scheduleOrThrowOnError([&, disk_ptr]() + { + for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) + { + if (!startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) + continue; + + if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME) + { + std::lock_guard lock(wal_init_lock); + if (write_ahead_log != nullptr) + throw Exception( + "There are multiple WAL files appeared in current storage policy. You need to resolve this manually", + ErrorCodes::CORRUPTED_DATA); + + write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); + for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext(), part_lock)) + disk_wal_parts.push_back(std::move(part)); + } + else + { + MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); + for (auto && part : wal.restore(metadata_snapshot, getContext(), part_lock)) + disk_wal_parts.push_back(std::move(part)); + } + } + }); + } + + pool.wait(); + + MutableDataPartsVector parts_from_wal; + for (auto & [_, disk_wal_parts] : disk_wal_part_map) + parts_from_wal.insert( + parts_from_wal.end(), std::make_move_iterator(disk_wal_parts.begin()), std::make_move_iterator(disk_wal_parts.end())); + + loadDataPartsFromWAL(broken_parts_to_detach, duplicate_parts_to_remove, parts_from_wal); + + num_parts += parts_from_wal.size(); + } + + if (num_parts == 0) + { + resetObjectColumnsFromActiveParts(part_lock); + LOG_DEBUG(log, "There are no data parts"); + return; + } for (auto & part : broken_parts_to_detach) { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 68ec191412b..3403ff18c7f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1339,8 +1339,7 @@ private: void loadDataPartsFromWAL( DataPartsVector & broken_parts_to_detach, DataPartsVector & duplicate_parts_to_remove, - MutableDataPartsVector & parts_from_wal, - DataPartsLock & part_lock); + MutableDataPartsVector & parts_from_wal); void resetObjectColumnsFromActiveParts(const DataPartsLock & lock); void updateObjectColumns(const DataPartPtr & part, const DataPartsLock & lock); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 9b79f89ff98..c8b3349734e 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -122,7 +123,10 @@ void MergeTreeWriteAheadLog::rotate(const std::unique_lock &) init(); } -MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore( + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + std::unique_lock & parts_lock) { std::unique_lock lock(write_mutex); @@ -172,6 +176,9 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor part->uuid = metadata.part_uuid; block = block_in.read(); + + if (storage.getActiveContainingPart(part->info, MergeTreeDataPartState::Active, parts_lock)) + continue; } else { @@ -238,6 +245,15 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor std::copy_if(parts.begin(), parts.end(), std::back_inserter(result), [&dropped_parts](const auto & part) { return dropped_parts.count(part->name) == 0; }); + /// All parts in WAL had been already committed into the disk -> clear the WAL + if (result.empty()) + { + LOG_DEBUG(log, "WAL file '{}' had been completely processed. Removing.", path); + disk->removeFile(path); + init(); + return {}; + } + return result; } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index eb75d374cb1..b54161dbdaa 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -62,7 +62,10 @@ public: void addPart(DataPartInMemoryPtr & part); void dropPart(const String & part_name); - std::vector restore(const StorageMetadataPtr & metadata_snapshot, ContextPtr context); + std::vector restore( + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + std::unique_lock & parts_lock); using MinMaxBlockNumber = std::pair; static std::optional tryParseMinMaxBlockNumber(const String & filename); diff --git a/tests/queries/0_stateless/02410_inmemory_wal_cleanup.reference b/tests/queries/0_stateless/02410_inmemory_wal_cleanup.reference new file mode 100644 index 00000000000..6727d83a6f4 --- /dev/null +++ b/tests/queries/0_stateless/02410_inmemory_wal_cleanup.reference @@ -0,0 +1,35 @@ +-- { echo } + +DROP TABLE IF EXISTS in_memory; +CREATE TABLE in_memory (a UInt32) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_compact_part = 1000, min_bytes_for_wide_part = 10485760; +INSERT INTO in_memory VALUES (1); +INSERT INTO in_memory VALUES (2); +SELECT name, active, part_type FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory'; +all_1_1_0 1 InMemory +all_2_2_0 1 InMemory +SELECT * FROM in_memory ORDER BY a; +1 +2 +-- no WAL remove since parts are still in use +DETACH TABLE in_memory; +ATTACH TABLE in_memory; +SELECT name, active, part_type FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory'; +all_1_1_0 1 InMemory +all_2_2_0 1 InMemory +SELECT * FROM in_memory ORDER BY a; +1 +2 +-- WAL should be removed, since on disk part covers all parts in WAL +OPTIMIZE TABLE in_memory; +DETACH TABLE in_memory; +ATTACH TABLE in_memory; +SELECT name, active, part_type FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory'; +all_1_2_1 1 Compact +-- check that the WAL will be reinitialized after remove +INSERT INTO in_memory VALUES (3); +DETACH TABLE in_memory; +ATTACH TABLE in_memory; +SELECT * FROM in_memory ORDER BY a; +1 +2 +3 diff --git a/tests/queries/0_stateless/02410_inmemory_wal_cleanup.sql b/tests/queries/0_stateless/02410_inmemory_wal_cleanup.sql new file mode 100644 index 00000000000..0228852a115 --- /dev/null +++ b/tests/queries/0_stateless/02410_inmemory_wal_cleanup.sql @@ -0,0 +1,27 @@ +-- { echo } + +DROP TABLE IF EXISTS in_memory; + +CREATE TABLE in_memory (a UInt32) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_compact_part = 1000, min_bytes_for_wide_part = 10485760; +INSERT INTO in_memory VALUES (1); +INSERT INTO in_memory VALUES (2); +SELECT name, active, part_type FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory'; +SELECT * FROM in_memory ORDER BY a; + +-- no WAL remove since parts are still in use +DETACH TABLE in_memory; +ATTACH TABLE in_memory; +SELECT name, active, part_type FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory'; +SELECT * FROM in_memory ORDER BY a; + +-- WAL should be removed, since on disk part covers all parts in WAL +OPTIMIZE TABLE in_memory; +DETACH TABLE in_memory; +ATTACH TABLE in_memory; +SELECT name, active, part_type FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory'; + +-- check that the WAL will be reinitialized after remove +INSERT INTO in_memory VALUES (3); +DETACH TABLE in_memory; +ATTACH TABLE in_memory; +SELECT * FROM in_memory ORDER BY a; From 16af4aebc815d5af439ff0da3b960d810401b790 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 4 Sep 2022 16:26:31 +0200 Subject: [PATCH 345/582] impl (#40952) --- .../ReplaceQueryParameterVisitor.cpp | 20 ++++++++++++------- ...d_protocol_with_query_parameters.reference | 7 +++++++ ...7_extend_protocol_with_query_parameters.sh | 16 ++++++++++++++- 3 files changed, 35 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 03de8aecc92..664cda74522 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -1,16 +1,17 @@ -#include -#include #include -#include #include +#include #include #include -#include -#include -#include #include #include #include +#include +#include +#include +#include +#include +#include namespace DB @@ -30,7 +31,12 @@ void ReplaceQueryParameterVisitor::visit(ASTPtr & ast) else if (ast->as() || ast->as()) visitIdentifier(ast); else - visitChildren(ast); + { + if (auto * describe_query = dynamic_cast(ast.get()); describe_query && describe_query->table_expression) + visitChildren(describe_query->table_expression); + else + visitChildren(ast); + } } diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference index f46cdb6e5e3..1da5cd0b7b3 100644 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference @@ -7,3 +7,10 @@ UInt64 String DateTime Map(UUID, Array(Float32)) 13 str 2022-08-04 18:30:53 {'10':[11,12],'13':[14,15]} 1 1 +_CAST(42, \'Int64\') Int64 +_CAST([1, 2, 3], \'Array(UInt8)\') Array(UInt8) +_CAST(((\'abc\', 22), (\'def\', 33)), \'Map(String, UInt8)\') Map(String, UInt8) +_CAST([[4, 5, 6], [7], [8, 9]], \'Array(Array(UInt8))\') Array(Array(UInt8)) +_CAST(((10, [11, 12]), (13, [14, 15])), \'Map(UInt8, Array(UInt8))\') Map(UInt8, Array(UInt8)) +_CAST(((\'ghj\', ((\'klm\', [16, 17]))), (\'nop\', ((\'rst\', [18])))), \'Map(String, Map(String, Array(UInt8)))\') Map(String, Map(String, Array(UInt8))) +a Int8 diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh index 335af1bb6e6..e61dc337d2a 100755 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh @@ -68,13 +68,27 @@ $CLICKHOUSE_CLIENT -n -q "select {n: UInt8} -- { serverError 456 }" $CLICKHOUSE_CLIENT -n -q "set param_n = 12; set param_n = 13; select {n: UInt8}" -# but multiple different parameters could be defined within each session +# multiple different parameters could be defined within each session $CLICKHOUSE_CLIENT -n -q " set param_a = 13, param_b = 'str'; set param_c = '2022-08-04 18:30:53'; set param_d = '{\'10\': [11, 12], \'13\': [14, 15]}'; select {a: UInt32}, {b: String}, {c: DateTime}, {d: Map(String, Array(UInt8))}" + # empty parameter name is not allowed $CLICKHOUSE_CLIENT --param_="" -q "select 1" 2>&1 | grep -c 'Code: 36' $CLICKHOUSE_CLIENT -q "set param_ = ''" 2>&1 | grep -c 'Code: 36' + + +# parameters are also supported for DESCRIBE TABLE queries +$CLICKHOUSE_CLIENT \ + --param_id="42" \ + --param_arr="[1, 2, 3]" \ + --param_map="{'abc': 22, 'def': 33}" \ + --param_mul_arr="[[4, 5, 6], [7], [8, 9]]" \ + --param_map_arr="{10: [11, 12], 13: [14, 15]}" \ + --param_map_map_arr="{'ghj': {'klm': [16, 17]}, 'nop': {'rst': [18]}}" \ + -q "describe table(select {id: Int64}, {arr: Array(UInt8)}, {map: Map(String, UInt8)}, {mul_arr: Array(Array(UInt8))}, {map_arr: Map(UInt8, Array(UInt8))}, {map_map_arr: Map(String, Map(String, Array(UInt8)))})" + +$CLICKHOUSE_CLIENT --param_p=42 -q "describe table (select * from (select {p:Int8} as a group by a) order by a)" From 6c164905ca446f5a3bd69ca259f69decd33cd0ad Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Sep 2022 18:57:51 +0200 Subject: [PATCH 346/582] Add test. --- tests/integration/test_storage_url/test.py | 71 ++++++++++++++++------ 1 file changed, 54 insertions(+), 17 deletions(-) diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 6ffb38bd8d7..5591e63400c 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -1,31 +1,26 @@ import pytest - from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV -uuids = [] +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/conf.xml"], with_nginx=True +) -@pytest.fixture(scope="module") -def cluster(): +@pytest.fixture(scope="module", autouse=True) +def setup_node(): try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node1", main_configs=["configs/conf.xml"], with_nginx=True - ) cluster.start() - - yield cluster - + node1.query( + "insert into table function url(url1) partition by column3 values (1, 2, 3), (3, 2, 1), (1, 3, 2)" + ) + yield finally: cluster.shutdown() -def test_partition_by(cluster): - node1 = cluster.instances["node1"] - - node1.query( - f"insert into table function url(url1) partition by column3 values (1, 2, 3), (3, 2, 1), (1, 3, 2)" - ) +def test_partition_by(): result = node1.query( f"select * from url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" ) @@ -38,3 +33,45 @@ def test_partition_by(cluster): f"select * from url('http://nginx:80/test_3', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" ) assert result.strip() == "1\t2\t3" + + +def test_table_function_url_access_rights(): + node1.query("CREATE USER OR REPLACE u1") + + expected_error = "necessary to have grant CREATE TEMPORARY TABLE, URL ON *.*" + assert expected_error in node1.query_and_get_error( + f"SELECT * FROM url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) + + expected_error = "necessary to have grant CREATE TEMPORARY TABLE, URL ON *.*" + assert expected_error in node1.query_and_get_error( + f"SELECT * FROM url('http://nginx:80/test_1', 'TSV')", user="u1" + ) + + assert node1.query( + f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) + + assert node1.query( + f"DESCRIBE TABLE url('http://nginx:80/not-exist', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) + + expected_error = "necessary to have grant URL ON *.*" + assert expected_error in node1.query_and_get_error( + f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV')", user="u1" + ) + + node1.query("GRANT URL ON *.* TO u1") + assert node1.query( + f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV')", + user="u1", + ) == TSV( + [ + ["c1", "Nullable(Int64)"], + ["c2", "Nullable(Int64)"], + ["c3", "Nullable(Int64)"], + ] + ) From cfb250c46bb9544730c6f2c3bdb7674df99d9a8b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Sep 2022 18:58:39 +0200 Subject: [PATCH 347/582] Fix checking access rights for DESCRIBE TABLE url. --- src/TableFunctions/ITableFunction.h | 5 +++-- src/TableFunctions/TableFunctionHDFS.cpp | 5 +++++ src/TableFunctions/TableFunctionHDFSCluster.cpp | 4 ++++ src/TableFunctions/TableFunctionS3.cpp | 2 ++ src/TableFunctions/TableFunctionS3Cluster.cpp | 2 ++ src/TableFunctions/TableFunctionURL.cpp | 4 ++++ 6 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index b419c4cfeed..4b9a87b93f1 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -69,13 +69,14 @@ public: virtual ~ITableFunction() = default; +protected: + virtual AccessType getSourceAccessType() const; + private: virtual StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const = 0; virtual const char * getStorageTypeName() const = 0; - - virtual AccessType getSourceAccessType() const; }; using TableFunctionPtr = std::shared_ptr; diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index ed3000ec152..57f692eadad 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include namespace DB { @@ -29,7 +31,10 @@ StoragePtr TableFunctionHDFS::getStorage( ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context) const { if (structure == "auto") + { + context->checkAccess(getSourceAccessType()); return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context); + } return parseColumnsListFromString(structure, context); } diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index 4a68fec1a5e..385d280a100 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -74,7 +75,10 @@ void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, Conte ColumnsDescription TableFunctionHDFSCluster::getActualTableStructure(ContextPtr context) const { if (structure == "auto") + { + context->checkAccess(getSourceAccessType()); return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context); + } return parseColumnsListFromString(structure, context); } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index e81b67d70a4..86a7e9a0eae 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -133,6 +134,7 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) { if (configuration.structure == "auto") { + context->checkAccess(getSourceAccessType()); return StorageS3::getTableStructureFromData( configuration.format, S3::URI(Poco::URI(configuration.url)), diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index d76bd954d27..5c61207b717 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -83,6 +84,7 @@ ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr co { if (configuration.structure == "auto") { + context->checkAccess(getSourceAccessType()); return StorageS3::getTableStructureFromData( configuration.format, S3::URI(Poco::URI(configuration.url)), diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index bbae0990062..99ec87c2e8f 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include @@ -113,12 +114,15 @@ ReadWriteBufferFromHTTP::HTTPHeaderEntries TableFunctionURL::getHeaders() const ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context) const { if (structure == "auto") + { + context->checkAccess(getSourceAccessType()); return StorageURL::getTableStructureFromData(format, filename, chooseCompressionMethod(Poco::URI(filename).getPath(), compression_method), getHeaders(), std::nullopt, context); + } return parseColumnsListFromString(structure, context); } From 48dc32faf880659e5f96e66c50a6704c1246496e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 4 Sep 2022 19:10:46 +0200 Subject: [PATCH 348/582] Better test --- src/Common/CurrentMetrics.cpp | 1 - src/Common/ProfileEvents.cpp | 1 + src/Storages/MergeTree/MergeTreeMarksLoader.cpp | 8 ++------ tests/queries/0_stateless/02417_load_marks_async.sh | 2 +- 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 9e5f3705dad..eeb8b4e2832 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -98,7 +98,6 @@ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ - M(BackgroundLoadingMarksTasks, "Number of currently executing background marks load tasks") \ M(S3Requests, "S3 requests") \ M(KeeperAliveConnections, "Number of alive connections") \ M(KeeperOutstandingRequets, "Number of outstanding requests") \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index dd35220d945..183a06b6610 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -147,6 +147,7 @@ M(SelectedBytes, "Number of bytes (uncompressed; for columns as they stored in memory) SELECTed from all tables.") \ \ M(WaitMarksLoadMicroseconds, "Time spent loading marks") \ + M(BackgroundLoadingMarksTasks, "Number of background tasks for loading marks") \ \ M(Merge, "Number of launched background merges.") \ M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \ diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 67bc5deba49..27100f8f474 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -9,14 +9,10 @@ #include -namespace CurrentMetrics -{ - extern const Metric BackgroundLoadingMarksTasks; -} - namespace ProfileEvents { extern const Event WaitMarksLoadMicroseconds; + extern const Event BackgroundLoadingMarksTasks; } namespace DB @@ -196,7 +192,7 @@ std::future MergeTreeMarksLoader::loadMarksAsync() CurrentThread::detachQuery(); }); - CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundLoadingMarksTasks}; + ProfileEvents::increment(ProfileEvents::BackgroundLoadingMarksTasks); return loadMarks(); }); diff --git a/tests/queries/0_stateless/02417_load_marks_async.sh b/tests/queries/0_stateless/02417_load_marks_async.sh index 71d7fdfd173..310258e6b3a 100755 --- a/tests/queries/0_stateless/02417_load_marks_async.sh +++ b/tests/queries/0_stateless/02417_load_marks_async.sh @@ -40,7 +40,7 @@ function test else echo 'F' fi - result=$(${CLICKHOUSE_CLIENT} -q "WITH CurrentMetric_BackgroundLoadingMarksTasks as a SELECT count(a) FROM system.metric_log WHERE a > 0") + result=$(${CLICKHOUSE_CLIENT} -q "SELECT ProfileEvents['BackgroundLoadingMarksTasks'] FROM system.query_log WHERE query_id = '${QUERY_ID}' AND type = 'QueryFinish' AND current_database = currentDatabase()") if [[ $result -ne 0 ]]; then echo 'Ok' else From 25e3bebd9d3cb64d8e42172f6f46ec489261109f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 29 Aug 2022 20:36:13 +0200 Subject: [PATCH 349/582] Rework core collecting on CI (eliminate gcore usage) gcore is a gdb command, that internally uses gdb to dump the core. However with proper configuration of limits (core_dump.size_limit) it should not be required, althought some issues is possible: - non standard kernel.core_pattern - sanitizers So yes, gcore is more "universal" (you don't need to configure any `kernel_pattern`), but it is ad-hoc, and it has drawbacks - **it does not work when gdb fails**. For example gdb may fail with `Dwarf Error: DW_FORM_strx1 found in non-DWO CU` in case of DWARF-5 [1]. [1]: https://github.com/ClickHouse/ClickHouse/pull/40772#issuecomment-1236331323. Let's try to switch to more native way. Signed-off-by: Azat Khuzhin --- docker/test/fuzzer/run-fuzzer.sh | 23 +++++++++++++++++++++-- docker/test/stress/run.sh | 20 +++++++++++++++++--- tests/ci/ast_fuzzer_check.py | 6 +++++- tests/ci/stress_check.py | 2 +- 4 files changed, 44 insertions(+), 7 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 11ddb0bd2d3..93e38260395 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -1,8 +1,15 @@ #!/bin/bash # shellcheck disable=SC2086,SC2001,SC2046,SC2030,SC2031 -set -eux +set -x + +# core.COMM.PID-TID +sysctl kernel.core_pattern='core.%e.%p-%P' + +set -e +set -u set -o pipefail + trap "exit" INT TERM # The watchdog is in the separate process group, so we have to kill it separately # if the script terminates earlier. @@ -87,6 +94,19 @@ function configure # TODO figure out which ones are needed cp -av --dereference "$repo_dir"/tests/config/config.d/listen.xml db/config.d cp -av --dereference "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d + + cat > db/config.d/core.xml < + + + 107374182400 + + + $PWD + +EOL } function watchdog @@ -180,7 +200,6 @@ handle SIGUSR2 nostop noprint pass handle SIG$RTMIN nostop noprint pass info signals continue -gcore backtrace full thread apply all backtrace full info registers diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index f8ecdf1aa21..a28785084f1 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -5,6 +5,9 @@ set -x +# core.COMM.PID-TID +sysctl kernel.core_pattern='core.%e.%p-%P' + # Thread Fuzzer allows to check more permutations of possible thread scheduling # and find more potential issues. @@ -99,6 +102,19 @@ EOL +EOL + + cat > /etc/clickhouse-server/config.d/core.xml < + + + 107374182400 + + + $PWD + EOL } @@ -155,7 +171,6 @@ handle SIGUSR2 nostop noprint pass handle SIG$RTMIN nostop noprint pass info signals continue -gcore backtrace full thread apply all backtrace full info registers @@ -467,8 +482,7 @@ done clickhouse-local --structure "test String, res String" -q "SELECT 'failure', test FROM table WHERE res != 'OK' order by (lower(test) like '%hung%'), rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv [ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv -# Core dumps (see gcore) -# Default filename is 'core.PROCESS_ID' +# Core dumps for core in core.*; do pigz $core mv $core.gz /test_output/ diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 9f3ddbe9932..0e41aaf8fba 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -29,7 +29,11 @@ IMAGE_NAME = "clickhouse/fuzzer" def get_run_command(pr_number, sha, download_url, workspace_path, image): return ( - f"docker run --network=host --volume={workspace_path}:/workspace " + f"docker run " + # For sysctl + "--privileged " + "--network=host " + f"--volume={workspace_path}:/workspace " "--cap-add syslog --cap-add sys_admin --cap-add=SYS_PTRACE " f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD="{download_url}" ' f"{image}" diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index e644eef3bc8..8f310eaa99d 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -33,7 +33,7 @@ def get_run_command( "docker run --cap-add=SYS_PTRACE " # a static link, don't use S3_URL or S3_DOWNLOAD "-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets' " - # For dmesg + # For dmesg and sysctl "--privileged " f"--volume={build_path}:/package_folder " f"--volume={result_folder}:/test_output " From f621bd0055137a2a467be16340361402a5095f2e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 4 Sep 2022 23:54:05 +0200 Subject: [PATCH 350/582] Delete old cache directories creation from entrypoint (#40946) --- docker/server/entrypoint.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index d4da5f0f38c..16372230d91 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -37,7 +37,6 @@ if [ -n "$ERROR_LOG_PATH" ]; then ERROR_LOG_DIR="$(dirname "$ERROR_LOG_PATH")"; FORMAT_SCHEMA_PATH="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=format_schema_path || true)" # There could be many disks declared in config -readarray -t FILESYSTEM_CACHE_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.data_cache_path' || true) readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.path' || true) CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}" @@ -51,7 +50,6 @@ for dir in "$DATA_DIR" \ "$TMP_DIR" \ "$USER_PATH" \ "$FORMAT_SCHEMA_PATH" \ - "${FILESYSTEM_CACHE_PATHS[@]}" \ "${DISKS_PATHS[@]}" do # check if variable not empty From 053a2186b586bbcb3b10f446ad32864c9bec4ee1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 01:01:23 +0300 Subject: [PATCH 351/582] Update replicated.md --- docs/en/engines/database-engines/replicated.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 554345a3c15..2b659cff942 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -12,7 +12,7 @@ One ClickHouse server can have multiple replicated databases running and updatin ## Creating a Database {#creating-a-database} ``` sql - CREATE DATABASE testdb ENGINE = Replicated('zoo_path', 'shard_name', 'replica_name') [SETTINGS ...] +CREATE DATABASE testdb ENGINE = Replicated('zoo_path', 'shard_name', 'replica_name') [SETTINGS ...] ``` **Engine Parameters** From f8e72eb7cbec6d3aa9037cc586811b83fa7ec58e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 01:02:42 +0300 Subject: [PATCH 352/582] Update replicated.md --- docs/en/engines/database-engines/replicated.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 554345a3c15..fa9bd4112b2 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -21,9 +21,7 @@ One ClickHouse server can have multiple replicated databases running and updatin - `shard_name` — Shard name. Database replicas are grouped into shards by `shard_name`. - `replica_name` — Replica name. Replica names must be different for all replicas of the same shard. -:::warning For [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication) tables if no arguments provided, then default arguments are used: `/clickhouse/tables/{uuid}/{shard}` and `{replica}`. These can be changed in the server settings [default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path) and [default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name). Macro `{uuid}` is unfolded to table's uuid, `{shard}` and `{replica}` are unfolded to values from server config, not from database engine arguments. But in the future, it will be possible to use `shard_name` and `replica_name` of Replicated database. -::: ## Specifics and Recommendations {#specifics-and-recommendations} From 007680d93f9e9a0428828cd3b59273628bda82d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 02:13:51 +0200 Subject: [PATCH 353/582] Preparation --- .../data-types/lowcardinality.md | 2 +- .../test_join_set_family_s3/test.py | 2 +- tests/integration/test_log_family_s3/test.py | 2 +- tests/integration/test_merge_tree_s3/test.py | 56 +++++++++---------- .../test_profile_events_s3/test.py | 14 +++-- .../test_replicated_merge_tree_s3/test.py | 4 +- .../test.py | 4 +- .../test_s3_zero_copy_replication/test.py | 2 +- 8 files changed, 45 insertions(+), 41 deletions(-) diff --git a/docs/zh/sql-reference/data-types/lowcardinality.md b/docs/zh/sql-reference/data-types/lowcardinality.md index e089a7f9d41..0a500005399 100644 --- a/docs/zh/sql-reference/data-types/lowcardinality.md +++ b/docs/zh/sql-reference/data-types/lowcardinality.md @@ -57,4 +57,4 @@ ORDER BY id - [高效低基数类型](https://www.altinity.com/blog/2019/3/27/low-cardinality). - [使用低基数类型减少ClickHouse的存储成本 – 来自Instana工程师的分享](https://www.instana.com/blog/reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer/). -- [字符优化 (俄语视频分享)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [英语分享](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). \ No newline at end of file +- [字符优化 (俄语视频分享)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [英语分享](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). diff --git a/tests/integration/test_join_set_family_s3/test.py b/tests/integration/test_join_set_family_s3/test.py index b09d5735628..38b56b7b15b 100644 --- a/tests/integration/test_join_set_family_s3/test.py +++ b/tests/integration/test_join_set_family_s3/test.py @@ -27,7 +27,7 @@ def cluster(): def assert_objects_count(cluster, objects_count, path="data/"): minio = cluster.minio_client - s3_objects = list(minio.list_objects(cluster.minio_bucket, path)) + s3_objects = list(minio.list_objects(cluster.minio_bucket, path, recursive=True)) if objects_count != len(s3_objects): for s3_object in s3_objects: object_meta = minio.stat_object(cluster.minio_bucket, s3_object.object_name) diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index 76ff0930db3..bed379d098b 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -25,7 +25,7 @@ def cluster(): def assert_objects_count(cluster, objects_count, path="data/"): minio = cluster.minio_client - s3_objects = list(minio.list_objects(cluster.minio_bucket, path)) + s3_objects = list(minio.list_objects(cluster.minio_bucket, path, recursive=True)) if objects_count != len(s3_objects): for s3_object in s3_objects: object_meta = minio.stat_object(cluster.minio_bucket, s3_object.object_name) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 544f064bdff..fddc0e9b9b2 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -120,11 +120,11 @@ def run_s3_mocks(cluster): def wait_for_delete_s3_objects(cluster, expected, timeout=30): minio = cluster.minio_client while timeout > 0: - if len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == expected: + if len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == expected: return timeout -= 1 time.sleep(1) - assert len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == expected + assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == expected @pytest.fixture(autouse=True) @@ -140,7 +140,7 @@ def drop_table(cluster, node_name): wait_for_delete_s3_objects(cluster, 0) finally: # Remove extra objects to prevent tests cascade failing - for obj in list(minio.list_objects(cluster.minio_bucket, "data/")): + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): minio.remove_object(cluster.minio_bucket, obj.object_name) @@ -162,7 +162,7 @@ def test_simple_insert_select( node.query("INSERT INTO s3_test VALUES {}".format(values1)) assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values") == values1 assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + files_per_part ) @@ -173,7 +173,7 @@ def test_simple_insert_select( == values1 + "," + values2 ) assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + files_per_part * 2 ) @@ -217,7 +217,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): node.query("SELECT count(distinct(id)) FROM s3_test FORMAT Values") == "(8192)" ) assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD_PER_PART_WIDE * 6 + FILES_OVERHEAD ) @@ -306,28 +306,28 @@ def test_attach_detach_partition(cluster, node_name): ) assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("ALTER TABLE s3_test DETACH PARTITION '2020-01-03'") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(4096)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("ALTER TABLE s3_test ATTACH PARTITION '2020-01-03'") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("ALTER TABLE s3_test DROP PARTITION '2020-01-03'") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(4096)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE ) @@ -338,7 +338,7 @@ def test_attach_detach_partition(cluster, node_name): ) assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD ) @@ -356,21 +356,21 @@ def test_move_partition_to_another_disk(cluster, node_name): ) assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-04' TO DISK 'hdd'") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE ) node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-04' TO DISK 's3'") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) @@ -391,7 +391,7 @@ def test_table_manipulations(cluster, node_name): node.query("RENAME TABLE s3_test TO s3_renamed") assert node.query("SELECT count(*) FROM s3_renamed FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("RENAME TABLE s3_renamed TO s3_test") @@ -402,14 +402,14 @@ def test_table_manipulations(cluster, node_name): node.query("ATTACH TABLE s3_test") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("TRUNCATE TABLE s3_test") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD ) @@ -434,7 +434,7 @@ def test_move_replace_partition_to_another_table(cluster, node_name): assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)" assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4 ) @@ -448,7 +448,7 @@ def test_move_replace_partition_to_another_table(cluster, node_name): assert node.query("SELECT count(*) FROM s3_clone FORMAT Values") == "(8192)" # Number of objects in S3 should be unchanged. assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4 ) @@ -462,7 +462,7 @@ def test_move_replace_partition_to_another_table(cluster, node_name): assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)" assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 6 ) @@ -483,14 +483,14 @@ def test_move_replace_partition_to_another_table(cluster, node_name): assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)" # Data should remain in S3 assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4 ) node.query("ALTER TABLE s3_test FREEZE") # Number S3 objects should be unchanged. assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4 ) @@ -499,7 +499,7 @@ def test_move_replace_partition_to_another_table(cluster, node_name): wait_for_delete_s3_objects(cluster, FILES_OVERHEAD_PER_PART_WIDE * 4) - for obj in list(minio.list_objects(cluster.minio_bucket, "data/")): + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): minio.remove_object(cluster.minio_bucket, obj.object_name) @@ -520,7 +520,7 @@ def test_freeze_unfreeze(cluster, node_name): node.query("TRUNCATE TABLE s3_test") assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) @@ -533,7 +533,7 @@ def test_freeze_unfreeze(cluster, node_name): # Data should be removed from S3. assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD ) @@ -556,7 +556,7 @@ def test_freeze_system_unfreeze(cluster, node_name): node.query("TRUNCATE TABLE s3_test") node.query("DROP TABLE s3_test_removed NO DELAY") assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) @@ -565,7 +565,7 @@ def test_freeze_system_unfreeze(cluster, node_name): # Data should be removed from S3. assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD ) @@ -692,7 +692,7 @@ def test_lazy_seek_optimization_for_async_read(cluster, node_name): node.query("SELECT * FROM s3_test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10") node.query("DROP TABLE IF EXISTS s3_test NO DELAY") minio = cluster.minio_client - for obj in list(minio.list_objects(cluster.minio_bucket, "data/")): + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): minio.remove_object(cluster.minio_bucket, obj.object_name) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index aa578a1273a..edea19ccb96 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -52,7 +52,7 @@ init_list = { def get_s3_events(instance): result = init_list.copy() events = instance.query( - "SELECT event,value FROM system.events WHERE event LIKE '%S3%'" + "SELECT event, value FROM system.events WHERE event LIKE '%S3%'" ).split("\n") for event in events: ev = event.split("\t") @@ -118,8 +118,8 @@ def get_query_stat(instance, hint): def get_minio_size(cluster): minio = cluster.minio_client size = 0 - for obj in minio.list_objects(cluster.minio_bucket, "data/"): - size += obj.size + for obj_level1 in minio.list_objects(cluster.minio_bucket, prefix="data/", recursive=True): + size += obj_level1.size return size @@ -135,7 +135,7 @@ def test_profile_events(cluster): metrics0 = get_s3_events(instance) minio0 = get_minio_stat(cluster) - query1 = "CREATE TABLE test_s3.test_s3 (key UInt32, value UInt32) ENGINE=MergeTree PRIMARY KEY key ORDER BY key SETTINGS storage_policy='s3'" + query1 = "CREATE TABLE test_s3.test_s3 (key UInt32, value UInt32) ENGINE=MergeTree PRIMARY KEY key ORDER BY key SETTINGS storage_policy = 's3'" instance.query(query1) size1 = get_minio_size(cluster) @@ -157,7 +157,7 @@ def test_profile_events(cluster): metrics1["WriteBufferFromS3Bytes"] - metrics0["WriteBufferFromS3Bytes"] == size1 ) - query2 = "INSERT INTO test_s3.test_s3 FORMAT Values" + query2 = "INSERT INTO test_s3.test_s3 VALUES" instance.query(query2 + " (1,1)") size2 = get_minio_size(cluster) @@ -172,9 +172,12 @@ def test_profile_events(cluster): metrics2["S3WriteRequestsCount"] - metrics1["S3WriteRequestsCount"] == minio2["set_requests"] - minio1["set_requests"] ) + stat2 = get_query_stat(instance, query2) + for metric in stat2: assert stat2[metric] == metrics2[metric] - metrics1[metric] + assert ( metrics2["WriteBufferFromS3Bytes"] - metrics1["WriteBufferFromS3Bytes"] == size2 - size1 @@ -195,6 +198,7 @@ def test_profile_events(cluster): == minio3["set_requests"] - minio2["set_requests"] ) stat3 = get_query_stat(instance, query3) + # With async reads profile events are not updated fully because reads are done in a separate thread. # for metric in stat3: # print(metric) diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 37027d07969..a8bb805036f 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -113,7 +113,7 @@ def drop_table(cluster): minio = cluster.minio_client # Remove extra objects to prevent tests cascade failing - for obj in list(minio.list_objects(cluster.minio_bucket, "data/")): + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): minio.remove_object(cluster.minio_bucket, obj.object_name) @@ -130,7 +130,7 @@ def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_par insert(cluster, node_idxs=[1, 2, 3], verify=True) minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == 3 * ( + assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == 3 * ( FILES_OVERHEAD + files_per_part * 3 ) diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index 73b611ad169..cea5f1a25bf 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -87,7 +87,7 @@ def drop_table(cluster): minio = cluster.minio_client # Remove extra objects to prevent tests cascade failing - for obj in list(minio.list_objects(cluster.minio_bucket, "data/")): + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): minio.remove_object(cluster.minio_bucket, obj.object_name) @@ -124,6 +124,6 @@ def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_par ) minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == ( + assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == ( 3 * FILES_OVERHEAD ) + (files_per_part * 3) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 7b7fb9d21ad..a456e3a0d0a 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -39,7 +39,7 @@ def cluster(): def get_large_objects_count(cluster, size=100, folder="data"): minio = cluster.minio_client counter = 0 - for obj in minio.list_objects(cluster.minio_bucket, "{}/".format(folder)): + for obj in minio.list_objects(cluster.minio_bucket, "{}/".format(folder), recursive=True): if obj.size is not None and obj.size >= size: counter = counter + 1 return counter From a0b5567d3e7623023de7712dc63b11595bc0bb16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 02:16:48 +0200 Subject: [PATCH 354/582] Move strange test --- .../0_stateless}/02044_exists_operator.reference | 0 .../0_stateless}/02044_exists_operator.sql | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/{integration => queries/0_stateless}/02044_exists_operator.reference (100%) rename tests/{integration => queries/0_stateless}/02044_exists_operator.sql (100%) diff --git a/tests/integration/02044_exists_operator.reference b/tests/queries/0_stateless/02044_exists_operator.reference similarity index 100% rename from tests/integration/02044_exists_operator.reference rename to tests/queries/0_stateless/02044_exists_operator.reference diff --git a/tests/integration/02044_exists_operator.sql b/tests/queries/0_stateless/02044_exists_operator.sql similarity index 100% rename from tests/integration/02044_exists_operator.sql rename to tests/queries/0_stateless/02044_exists_operator.sql From 89d40b6f708e814bb128a38b15f19f5d070dce84 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 5 Sep 2022 00:23:36 +0000 Subject: [PATCH 355/582] Automatic style fix --- tests/integration/test_merge_tree_s3/test.py | 26 ++++++++++++++----- .../test_profile_events_s3/test.py | 4 ++- .../test_replicated_merge_tree_s3/test.py | 6 ++--- .../test.py | 6 ++--- .../test_s3_zero_copy_replication/test.py | 4 ++- 5 files changed, 31 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index fddc0e9b9b2..4276125c347 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -120,11 +120,17 @@ def run_s3_mocks(cluster): def wait_for_delete_s3_objects(cluster, expected, timeout=30): minio = cluster.minio_client while timeout > 0: - if len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == expected: + if ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == expected + ): return timeout -= 1 time.sleep(1) - assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == expected + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == expected + ) @pytest.fixture(autouse=True) @@ -140,7 +146,9 @@ def drop_table(cluster, node_name): wait_for_delete_s3_objects(cluster, 0) finally: # Remove extra objects to prevent tests cascade failing - for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): + for obj in list( + minio.list_objects(cluster.minio_bucket, "data/", recursive=True) + ): minio.remove_object(cluster.minio_bucket, obj.object_name) @@ -338,7 +346,8 @@ def test_attach_detach_partition(cluster, node_name): ) assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == FILES_OVERHEAD ) @@ -409,7 +418,8 @@ def test_table_manipulations(cluster, node_name): node.query("TRUNCATE TABLE s3_test") assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)" assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == FILES_OVERHEAD ) @@ -533,7 +543,8 @@ def test_freeze_unfreeze(cluster, node_name): # Data should be removed from S3. assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == FILES_OVERHEAD ) @@ -565,7 +576,8 @@ def test_freeze_system_unfreeze(cluster, node_name): # Data should be removed from S3. assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == FILES_OVERHEAD + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == FILES_OVERHEAD ) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index edea19ccb96..02817a21fdb 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -118,7 +118,9 @@ def get_query_stat(instance, hint): def get_minio_size(cluster): minio = cluster.minio_client size = 0 - for obj_level1 in minio.list_objects(cluster.minio_bucket, prefix="data/", recursive=True): + for obj_level1 in minio.list_objects( + cluster.minio_bucket, prefix="data/", recursive=True + ): size += obj_level1.size return size diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index a8bb805036f..0d978bb6967 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -130,9 +130,9 @@ def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_par insert(cluster, node_idxs=[1, 2, 3], verify=True) minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == 3 * ( - FILES_OVERHEAD + files_per_part * 3 - ) + assert len( + list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)) + ) == 3 * (FILES_OVERHEAD + files_per_part * 3) def test_drop_cache_on_cluster(cluster): diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index cea5f1a25bf..60a1b9b9746 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -124,6 +124,6 @@ def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_par ) minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) == ( - 3 * FILES_OVERHEAD - ) + (files_per_part * 3) + assert len( + list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)) + ) == (3 * FILES_OVERHEAD) + (files_per_part * 3) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index a456e3a0d0a..860b83d4ed1 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -39,7 +39,9 @@ def cluster(): def get_large_objects_count(cluster, size=100, folder="data"): minio = cluster.minio_client counter = 0 - for obj in minio.list_objects(cluster.minio_bucket, "{}/".format(folder), recursive=True): + for obj in minio.list_objects( + cluster.minio_bucket, "{}/".format(folder), recursive=True + ): if obj.size is not None and obj.size >= size: counter = counter + 1 return counter From e70d526896943b26ed460bf271a122596ba3eeb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 03:35:43 +0200 Subject: [PATCH 356/582] Fix trash --- tests/integration/test_profile_events_s3/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 02817a21fdb..4f0f6a9f40f 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -75,20 +75,20 @@ def get_minio_stat(cluster): ) ).text.split("\n") for line in stat: - x = re.search("s3_requests_total(\{.*\})?\s(\d+)(\s.*)?", line) + x = re.search(r"s3_requests_total(\{.*\})?\s(\d+)(\s.*)?", line) if x != None: y = re.search('.*api="(get|list|head|select).*', x.group(1)) if y != None: result["get_requests"] += int(x.group(2)) else: result["set_requests"] += int(x.group(2)) - x = re.search("s3_errors_total(\{.*\})?\s(\d+)(\s.*)?", line) + x = re.search(r"s3_errors_total(\{.*\})?\s(\d+)(\s.*)?", line) if x != None: result["errors"] += int(x.group(2)) - x = re.search("s3_rx_bytes_total(\{.*\})?\s([\d\.e\+\-]+)(\s.*)?", line) + x = re.search(r"s3_rx_bytes_total(\{.*\})?\s([\d\.e\+\-]+)(\s.*)?", line) if x != None: result["tx_bytes"] += float(x.group(2)) - x = re.search("s3_tx_bytes_total(\{.*\})?\s([\d\.e\+\-]+)(\s.*)?", line) + x = re.search(r"s3_tx_bytes_total(\{.*\})?\s([\d\.e\+\-]+)(\s.*)?", line) if x != None: result["rx_bytes"] += float(x.group(2)) return result From cda8052df04ce5cb991bcb4098935e313a9df4a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 03:46:10 +0200 Subject: [PATCH 357/582] Fix trash --- tests/integration/runner | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index f0d87b23a83..e1b9a55b43e 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -350,8 +350,7 @@ if __name__ == "__main__": # randomizer, we should remove it after Sep 2022 try: subprocess.check_call( - "docker volume rm $(docker volume ls -q | " - f"grep '{VOLUME_NAME}_.*_volume')", + f"docker volume ls -q | grep '{VOLUME_NAME}_.*_volume' | xargs --no-run-if-empty docker volume rm", shell=True, ) except Exception as ex: From 1ac8e739c96f724c28ebcd751cdbef95019f49e0 Mon Sep 17 00:00:00 2001 From: Luck-Chang Date: Mon, 5 Sep 2022 09:50:24 +0800 Subject: [PATCH 358/582] fix some typos and clang-tidy warnings --- programs/server/Server.cpp | 4 +-- src/Common/ConcurrencyControl.h | 2 +- src/Common/OvercommitTracker.h | 4 +-- src/Common/ZooKeeper/IKeeper.h | 4 +-- src/Functions/IFunction.h | 4 +-- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Interpreters/TreeCNFConverter.cpp | 2 +- src/Interpreters/TreeCNFConverter.h | 2 +- src/Interpreters/TreeOptimizer.cpp | 4 +-- src/Interpreters/TreeRewriter.h | 2 +- .../WhereConstraintsOptimizer.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 34 +++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 14 ++++---- .../ReplicatedMergeTreeAttachThread.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 20 +++++------ src/Storages/StorageReplicatedMergeTree.h | 2 +- 18 files changed, 54 insertions(+), 54 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d788270ecf9..93df877ab8e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1036,7 +1036,7 @@ int Server::main(const std::vector & /*args*/) try { LOG_DEBUG( - log, "Initiailizing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted); + log, "Initializing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted); global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); } catch (...) @@ -1089,7 +1089,7 @@ int Server::main(const std::vector & /*args*/) } } - LOG_DEBUG(log, "Initiailizing interserver credentials."); + LOG_DEBUG(log, "Initializing interserver credentials."); global_context->updateInterserverCredentials(config()); if (config().has("macros")) diff --git a/src/Common/ConcurrencyControl.h b/src/Common/ConcurrencyControl.h index 6f37bb45c84..72bf9899618 100644 --- a/src/Common/ConcurrencyControl.h +++ b/src/Common/ConcurrencyControl.h @@ -250,7 +250,7 @@ private: } } - SlotCount available(std::unique_lock &) + SlotCount available(std::unique_lock &) const { if (cur_concurrency < max_concurrency) return max_concurrency - cur_concurrency; diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 64fb6cdc926..598b877ef3c 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -61,7 +61,7 @@ enum class QueryCancellationState // Usually it's hard to set some reasonable hard memory limit // (especially, the default value). This class introduces new -// mechanisim for the limiting of memory usage. +// mechanism for the limiting of memory usage. // Soft limit represents guaranteed amount of memory query/user // may use. It's allowed to exceed this limit. But if hard limit // is reached, query with the biggest overcommit ratio @@ -82,7 +82,7 @@ protected: virtual void pickQueryToExcludeImpl() = 0; // This mutex is used to disallow concurrent access - // to picked_tracker and cancelation_state variables. + // to picked_tracker and cancellation_state variables. std::mutex overcommit_m; std::condition_variable cv; diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index c6aa954688b..0d470d4fa5d 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -80,7 +80,7 @@ enum class Error : int32_t ZUNIMPLEMENTED = -6, /// Operation is unimplemented ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments - ZINVALIDSTATE = -9, /// Invliad zhandle state + ZINVALIDSTATE = -9, /// Invalid handle state /** API errors. * This is never thrown by the server, it shouldn't be used other than @@ -439,7 +439,7 @@ public: /** Usage scenario: * - create an object and issue commands; * - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap: - * for example, just signal a condvar / fulfull a promise. + * for example, just signal a condvar / fulfill a promise. * - you also may provide callbacks for watches; they are also invoked in internal thread and must be cheap. * - whenever you receive exception with ZSESSIONEXPIRED code or method isExpired returns true, * the ZooKeeper instance is no longer usable - you may only destroy it and probably create another. diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 95af8a61aae..83b89b85b62 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -171,7 +171,7 @@ public: */ virtual bool isSuitableForConstantFolding() const { return true; } - /** If function isSuitableForConstantFolding then, this method will be called during query analyzis + /** If function isSuitableForConstantFolding then, this method will be called during query analysis * if some arguments are constants. For example logical functions (AndFunction, OrFunction) can * return they result based on some constant arguments. * Arguments are passed without modifications, useDefaultImplementationForNulls, useDefaultImplementationForNothing, @@ -394,7 +394,7 @@ private: using FunctionOverloadResolverPtr = std::shared_ptr; /// Old function interface. Check documentation in IFunction.h. -/// If client do not need statefull properties it can implement this interface. +/// If client do not need stateful properties it can implement this interface. class IFunction { public: diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 106b1611f80..a6d88c7d28b 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -753,7 +753,7 @@ bool InterpreterSystemQuery::dropReplicaImpl(ASTSystemQuery & query, const Stora "if you want to clean the data and drop this replica", ErrorCodes::TABLE_WAS_NOT_DROPPED); /// NOTE it's not atomic: replica may become active after this check, but before dropReplica(...) - /// However, the main usecase is to drop dead replica, which cannot become active. + /// However, the main use case is to drop dead replica, which cannot become active. /// This check prevents only from accidental drop of some other replica. if (zookeeper->exists(status.zookeeper_path + "/replicas/" + query.replica + "/is_active")) throw Exception("Can't drop replica: " + query.replica + ", because it's active", diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 1f61c88ddd0..8812e90a5f0 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -349,7 +349,7 @@ CNFQuery & CNFQuery::pullNotOutFunctions() return *this; } -CNFQuery & CNFQuery::pushNotInFuntions() +CNFQuery & CNFQuery::pushNotInFunctions() { transformAtoms([](const AtomicFormula & atom) -> AtomicFormula { diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index a5d42e6b989..70c8990f74a 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -133,7 +133,7 @@ public: /// Converts != -> NOT =; <,>= -> (NOT) <; >,<= -> (NOT) <= for simpler matching CNFQuery & pullNotOutFunctions(); /// Revert pullNotOutFunctions actions - CNFQuery & pushNotInFuntions(); + CNFQuery & pushNotInFunctions(); /// (a OR b OR ...) AND (NOT a OR b OR ...) -> (b OR ...) CNFQuery & reduce(); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 07c7cd85d1b..0307beada3c 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -154,7 +154,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) continue; } } - /// don't optimise functions that shadow any of it's arguments, e.g.: + /// don't optimize functions that shadow any of it's arguments, e.g.: /// SELECT toString(dummy) as dummy FROM system.one GROUP BY dummy; if (!function->alias.empty()) { @@ -632,7 +632,7 @@ bool convertQueryToCNF(ASTSelectQuery * select_query) if (!cnf_form) return false; - cnf_form->pushNotInFuntions(); + cnf_form->pushNotInFunctions(); select_query->refWhere() = TreeCNFConverter::fromCNF(*cnf_form); return true; } diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 16ff7f8b6c3..7954547c070 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -99,7 +99,7 @@ using TreeRewriterResultPtr = std::shared_ptr; /// Tree Rewriter in terms of CMU slides @sa https://15721.courses.cs.cmu.edu/spring2020/slides/19-optimizer1.pdf /// -/// Optimises AST tree and collect information for further expression analysis in ExpressionAnalyzer. +/// Optimizes AST tree and collect information for further expression analysis in ExpressionAnalyzer. /// Result AST has the following invariants: /// * all aliases are substituted /// * qualified names are translated diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 83bdcfeb2e1..234b99167bb 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -170,7 +170,7 @@ void WhereConstraintsOptimizer::perform() return replaceTermsToConstants(atom, compare_graph); }) .reduce() - .pushNotInFuntions(); + .pushNotInFunctions(); if (optimize_append_index) AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a8babeb59c0..9e9be760f4c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1062,7 +1062,7 @@ void MergeTreeData::loadDataPartsFromDisk( LOG_ERROR(log, "Detaching broken part {}{} (size: {}). " - "If it happened after update, it is likely because of backward incompability. " + "If it happened after update, it is likely because of backward incompatibility. " "You need to resolve this manually", getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); std::lock_guard loading_lock(mutex); @@ -1410,7 +1410,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) continue; } - /// Check if CSNs were witten after committing transaction, update and write if needed. + /// Check if CSNs were written after committing transaction, update and write if needed. bool version_updated = false; chassert(!version.creation_tid.isEmpty()); if (!part->version.creation_csn) @@ -1833,18 +1833,18 @@ size_t MergeTreeData::clearOldPartsFromFilesystem(bool force) void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts, bool throw_on_error, NameSet * parts_failed_to_delete) { - NameSet part_names_successeded; + NameSet part_names_succeed; - auto get_failed_parts = [&part_names_successeded, &parts_failed_to_delete, &parts] () + auto get_failed_parts = [&part_names_succeed, &parts_failed_to_delete, &parts] () { - if (part_names_successeded.size() == parts.size()) + if (part_names_succeed.size() == parts.size()) return; if (parts_failed_to_delete) { for (const auto & part : parts) { - if (!part_names_successeded.contains(part->name)) + if (!part_names_succeed.contains(part->name)) parts_failed_to_delete->insert(part->name); } } @@ -1852,7 +1852,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts, bool try { - clearPartsFromFilesystemImpl(parts, &part_names_successeded); + clearPartsFromFilesystemImpl(parts, &part_names_succeed); get_failed_parts(); } catch (...) @@ -1864,7 +1864,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts, bool } } -void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_to_remove, NameSet * part_names_successed) +void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_to_remove, NameSet * part_names_succeed) { const auto settings = getSettings(); if (parts_to_remove.size() > 1 && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold) @@ -1884,10 +1884,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t LOG_DEBUG(log, "Removing part from filesystem {}", part->name); part->remove(); - if (part_names_successed) + if (part_names_succeed) { std::lock_guard lock(part_names_mutex); - part_names_successed->insert(part->name); + part_names_succeed->insert(part->name); } }); } @@ -1900,13 +1900,13 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t { LOG_DEBUG(log, "Removing part from filesystem {}", part->name); part->remove(); - if (part_names_successed) - part_names_successed->insert(part->name); + if (part_names_succeed) + part_names_succeed->insert(part->name); } } } -size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirecory() +size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirectory() { /** * Remove old (configured by setting) broken detached parts. @@ -2059,7 +2059,7 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_ { /// Relies on storage path, so we drop it during rename - /// it will be recreated automatiaclly. + /// it will be recreated automatically. std::lock_guard wal_lock(write_ahead_log_mutex); if (write_ahead_log) { @@ -3894,7 +3894,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception("Volume " + name + " does not exists on policy " + getStoragePolicy()->getName(), ErrorCodes::UNKNOWN_DISK); if (parts.empty()) - throw Exception("Nothing to move (сheck that the partition exists).", ErrorCodes::NO_SUCH_DATA_PART); + throw Exception("Nothing to move (check that the partition exists).", ErrorCodes::NO_SUCH_DATA_PART); std::erase_if(parts, [&](auto part_ptr) { @@ -6248,7 +6248,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( { // Store metadata for replicated table. - // Do nothing for non-replocated. + // Do nothing for non-replicated. createAndStoreFreezeMetadata(disk, part, fs::path(backup_part_path) / part->data_part_storage->getPartDirectory()); }; @@ -6561,7 +6561,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge auto disk = moving_part.reserved_space->getDisk(); if (supportsReplication() && disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication) { - /// If we acuqired lock than let's try to move. After one + /// If we acquired lock than let's try to move. After one /// replica will actually move the part from disk to some /// zero-copy storage other replicas will just fetch /// metainformation. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 68ec191412b..2bccdcbb2ec 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -177,7 +177,7 @@ public: /// Rename map new_name -> old_name std::unordered_map rename_map; - bool isColumnRenamed(const String & new_name) const { return rename_map.count(new_name) > 0; } + bool isColumnRenamed(const String & new_name) const { return rename_map.contains(new_name); } String getColumnOldName(const String & new_name) const { return rename_map.at(new_name); } }; @@ -634,7 +634,7 @@ public: /// Delete WAL files containing parts, that all already stored on disk. size_t clearOldWriteAheadLogs(); - size_t clearOldBrokenPartsFromDetachedDirecory(); + size_t clearOldBrokenPartsFromDetachedDirectory(); /// Delete all directories which names begin with "tmp" /// Must be called with locked lockForShare() because it's using relative_data_path. @@ -761,7 +761,7 @@ public: const ColumnsDescription & getObjectColumns() const { return object_columns; } - /// Creates desciprion of columns of data type Object from the range of data parts. + /// Creates description of columns of data type Object from the range of data parts. static ColumnsDescription getObjectColumns( const DataPartsVector & parts, const ColumnsDescription & storage_columns); @@ -1083,7 +1083,7 @@ protected: DataPartsIndexes::index::type & data_parts_by_info; DataPartsIndexes::index::type & data_parts_by_state_and_info; - /// Current descriprion of columns of data type Object. + /// Current description of columns of data type Object. /// It changes only when set of parts is changed and is /// protected by @data_parts_mutex. ColumnsDescription object_columns; @@ -1125,7 +1125,7 @@ protected: return {begin, end}; } - /// Creates desciprion of columns of data type Object from the range of data parts. + /// Creates description of columns of data type Object from the range of data parts. static ColumnsDescription getObjectColumns( boost::iterator_range range, const ColumnsDescription & storage_columns); @@ -1263,7 +1263,7 @@ private: void checkPartCanBeAddedToTable(MutableDataPartPtr & part, DataPartsLock & lock) const; /// Preparing itself to be committed in memory: fill some fields inside part, add it to data_parts_indexes - /// in precommitted state and to transasction + /// in precommitted state and to transaction void preparePartForCommit(MutableDataPartPtr & part, Transaction & out_transaction, DataPartStorageBuilderPtr builder); /// Low-level method for preparing parts for commit (in-memory). @@ -1352,7 +1352,7 @@ private: /// Remove parts from disk calling part->remove(). Can do it in parallel in case of big set of parts and enabled settings. /// If we fail to remove some part and throw_on_error equal to `true` will throw an exception on the first failed part. /// Otherwise, in non-parallel case will break and return. - void clearPartsFromFilesystemImpl(const DataPartsVector & parts, NameSet * part_names_successed); + void clearPartsFromFilesystemImpl(const DataPartsVector & parts, NameSet * part_names_succeed); TemporaryParts temporary_parts; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index f3e33b6b38b..ba4979e57f2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -149,7 +149,7 @@ void ReplicatedMergeTreeAttachThread::runImpl() storage.clearOldTemporaryDirectories(0, {"tmp_", "delete_tmp_", "tmp-fetch_"}); storage.clearOldWriteAheadLogs(); if (storage.getSettings()->merge_tree_enable_clear_old_broken_detached) - storage.clearOldBrokenPartsFromDetachedDirecory(); + storage.clearOldBrokenPartsFromDetachedDirectory(); storage.createNewZooKeeperNodes(); storage.syncPinnedPartUUIDs(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 74e3d0881ff..cc983960847 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -66,7 +66,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() storage.clearOldWriteAheadLogs(); storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); if (storage.getSettings()->merge_tree_enable_clear_old_broken_detached) - storage.clearOldBrokenPartsFromDetachedDirecory(); + storage.clearOldBrokenPartsFromDetachedDirectory(); } /// This is loose condition: no problem if we actually had lost leadership at this moment diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 507047751f3..a2988baa770 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1202,7 +1202,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign cleared_count += clearOldMutations(); cleared_count += clearEmptyParts(); if (getSettings()->merge_tree_enable_clear_old_broken_detached) - cleared_count += clearOldBrokenPartsFromDetachedDirecory(); + cleared_count += clearOldBrokenPartsFromDetachedDirectory(); return cleared_count; /// TODO maybe take into account number of cleared objects when calculating backoff }, common_assignee_trigger, getStorageID()), /* need_trigger */ false); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 762c3d52627..4be97e01293 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4582,7 +4582,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer if (entry.alter_version < metadata_version) { /// TODO Can we replace it with LOGICAL_ERROR? - /// As for now, it may rerely happen due to reordering of ALTER_METADATA entries in the queue of + /// As for now, it may rarely happen due to reordering of ALTER_METADATA entries in the queue of /// non-initial replica and also may happen after stale replica recovery. LOG_WARNING(log, "Attempt to update metadata of version {} " "to older version {} when processing log entry {}: {}", @@ -4664,7 +4664,7 @@ PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAf } else { - /// TODO: Implement optimal block number aqcuisition algorithm in multiple (but not all) partitions + /// TODO: Implement optimal block number acquisition algorithm in multiple (but not all) partitions EphemeralLocksInAllPartitions lock_holder( fs::path(zookeeper_path) / "block_numbers", "block-", fs::path(zookeeper_path) / "temp", *zookeeper); @@ -4841,7 +4841,7 @@ void StorageReplicatedMergeTree::alter( Coordination::Responses results; Coordination::Error rc = zookeeper->tryMulti(ops, results); - /// For the sake of constitency with mechanics of concurrent background process of assigning parts merge tasks + /// For the sake of consistency with mechanics of concurrent background process of assigning parts merge tasks /// this placeholder must be held up until the moment of committing into ZK of the mutation entry /// See ReplicatedMergeTreeMergePredicate::canMergeTwoParts() method partition_block_numbers_holder.reset(); @@ -5897,7 +5897,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte /// partitions, saves them in the mutation entry and writes the mutation entry to a new ZK node in /// the /mutations folder. This block numbers are needed to determine which parts should be mutated and /// which shouldn't (parts inserted after the mutation will have the block number higher than the - /// block number acquired by the mutation in that partition and so will not be mutatied). + /// block number acquired by the mutation in that partition and so will not be mutated). /// This block number is called "mutation version" in that partition. /// /// Mutation versions are acquired atomically in all partitions, so the case when an insert in some @@ -7217,7 +7217,7 @@ bool StorageReplicatedMergeTree::addOpsToDropAllPartsInPartition( } void StorageReplicatedMergeTree::dropAllPartsInPartitions( - zkutil::ZooKeeper & zookeeper, const Strings partition_ids, std::vector & entries, ContextPtr query_context, bool detach) + zkutil::ZooKeeper & zookeeper, const Strings & partition_ids, std::vector & entries, ContextPtr query_context, bool detach) { entries.reserve(partition_ids.size()); @@ -7600,7 +7600,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!children.empty()) { - LOG_TRACE(logger, "Found {} ({}) zookeper locks for {}", zookeeper_part_uniq_node, children.size(), fmt::join(children, ", ")); + LOG_TRACE(logger, "Found {} ({}) zookeeper locks for {}", zookeeper_part_uniq_node, children.size(), fmt::join(children, ", ")); part_has_no_more_locks = false; continue; } @@ -7706,12 +7706,12 @@ String StorageReplicatedMergeTree::getSharedDataReplica( String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / id; Strings id_replicas; zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas); - LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); + LOG_TRACE(log, "Found zookeeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); replicas.insert(id_replicas.begin(), id_replicas.end()); } } - LOG_TRACE(log, "Found zookeper replicas for part {}: {}", part.name, replicas.size()); + LOG_TRACE(log, "Found zookeeper replicas for part {}: {}", part.name, replicas.size()); Strings active_replicas; @@ -7724,7 +7724,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( if ((replica != replica_name) && (zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active"))) active_replicas.push_back(replica); - LOG_TRACE(log, "Found zookeper active replicas for part {}: {}", part.name, active_replicas.size()); + LOG_TRACE(log, "Found zookeeper active replicas for part {}: {}", part.name, active_replicas.size()); if (active_replicas.empty()) return ""; @@ -8159,7 +8159,7 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode( if (!created) { - String mode_str = mode == zkutil::CreateMode::Persistent ? "persistent" : "ephemral"; + String mode_str = mode == zkutil::CreateMode::Persistent ? "persistent" : "ephemeral"; throw Exception(ErrorCodes::NOT_FOUND_NODE, "Cannot create {} zero copy lock {} because part was unlocked from zookeeper", mode_str, zookeeper_node); } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2e2a5ca79b7..79df4f11490 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -754,7 +754,7 @@ private: std::vector & delimiting_block_locks, std::vector & log_entry_ops_idx); void dropAllPartsInPartitions( - zkutil::ZooKeeper & zookeeper, const Strings partition_ids, std::vector & entries, ContextPtr query_context, bool detach); + zkutil::ZooKeeper & zookeeper, const Strings & partition_ids, std::vector & entries, ContextPtr query_context, bool detach); LogEntryPtr dropAllPartsInPartition( zkutil::ZooKeeper & zookeeper, const String & partition_id, ContextPtr query_context, bool detach); From 0800c0fd4cdc694082ee57da91b1f51906a83dd0 Mon Sep 17 00:00:00 2001 From: Luck-Chang Date: Mon, 5 Sep 2022 14:09:42 +0800 Subject: [PATCH 359/582] fix some typos --- src/Common/ZooKeeper/IKeeper.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 0d470d4fa5d..f7f0b036064 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -80,7 +80,7 @@ enum class Error : int32_t ZUNIMPLEMENTED = -6, /// Operation is unimplemented ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments - ZINVALIDSTATE = -9, /// Invalid handle state + ZINVALIDSTATE = -9, /// Invalid zhandle state /** API errors. * This is never thrown by the server, it shouldn't be used other than From dc17a125f8cfe7206ecb4448a59660640a8eb964 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 07:00:52 +0000 Subject: [PATCH 360/582] No parallel for heavy KeeperMap test --- tests/queries/0_stateless/02416_keeper_map.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index caf44fc0ed6..bde07db1328 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest +-- Tags: no-ordinary-database, no-fasttest, no-parallel DROP TABLE IF EXISTS 02416_test SYNC; From 569f0abfb7238439c454c44378d48c2736b188c9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 07:03:23 +0000 Subject: [PATCH 361/582] Change error code --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8f6f16a7098..ea333c44362 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -603,7 +603,7 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k continue; if (value.wait_until(wait_until) != std::future_status::ready) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fetch values: timeout"); + throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch values: timeout"); auto response = value.get(); Coordination::Error code = response.error; From 3d65e3f2eed31c9891d989e1a3cb437dcd5a431d Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Mon, 5 Sep 2022 16:37:55 +0800 Subject: [PATCH 362/582] Add cluster/distributed/remote to file --- src/Storages/Distributed/DirectoryMonitor.cpp | 28 +++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 16981d26146..f84ddeb4f5e 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -140,6 +140,11 @@ namespace size_t rows = 0; size_t bytes = 0; + UInt32 shard_num = 0; + std::string cluster_name; + std::string distributed_table; + std::string remote_table; + /// dumpStructure() of the header -- obsolete std::string block_header_string; Block block_header; @@ -195,6 +200,14 @@ namespace in.getFileName(), distributed_header.revision, DBMS_TCP_PROTOCOL_VERSION); } + if (header_buf.hasPendingData()) + { + readVarUInt(distributed_header.shard_num, header_buf); + readStringBinary(distributed_header.cluster_name, header_buf); + readStringBinary(distributed_header.distributed_table, header_buf); + readStringBinary(distributed_header.remote_table, header_buf); + } + /// Add handling new data here, for example: /// /// if (header_buf.hasPendingData()) @@ -621,18 +634,23 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa ReadBufferFromFile in(file_path); const auto & distributed_header = readDistributedHeader(in, log); - auto connection = pool->get(timeouts, &distributed_header.insert_settings); + thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + this->storage.getContext()->getOpenTelemetrySpanLog()); + thread_trace_context->root_span.addAttribute("clickhouse.shard_num", distributed_header.shard_num); + thread_trace_context->root_span.addAttribute("clickhouse.cluster", distributed_header.cluster_name); + thread_trace_context->root_span.addAttribute("clickhouse.distributed", distributed_header.distributed_table); + thread_trace_context->root_span.addAttribute("clickhouse.remote", distributed_header.remote_table); + thread_trace_context->root_span.addAttribute("clickhouse.rows", distributed_header.rows); + thread_trace_context->root_span.addAttribute("clickhouse.bytes", distributed_header.bytes); + auto connection = pool->get(timeouts, &distributed_header.insert_settings); LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", file_path, connection->getDescription(), formatReadableQuantity(distributed_header.rows), formatReadableSizeWithBinarySuffix(distributed_header.bytes)); - thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - this->storage.getContext()->getOpenTelemetrySpanLog()); - RemoteInserter remote{*connection, timeouts, distributed_header.insert_query, distributed_header.insert_settings, From a17bc51d5b245b40870abb6caaecd16924eeac32 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Mon, 5 Sep 2022 16:39:47 +0800 Subject: [PATCH 363/582] Save cluster/distributed/table to log --- src/Storages/Distributed/DistributedSink.cpp | 27 ++++++++++++-------- src/Storages/Distributed/DistributedSink.h | 4 +-- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index ae72fdd84e2..0e379a7bd89 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -338,7 +338,11 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); - span.addAttribute("clickhouse.written_rows", rows); + span.addAttribute("clickhouse.cluster", this->storage.cluster_name); + span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); + span.addAttribute("clickhouse.remote", [this]() { return storage.remote_database + "." + storage.remote_table; }); + span.addAttribute("clickhouse.rows", [rows]() { return std::to_string(rows); }); + span.addAttribute("clickhouse.bytes", [&shard_block]() { return std::to_string(shard_block.bytes()); }); if (!job.is_local_job || !settings.prefer_localhost_replica) { @@ -610,20 +614,15 @@ void DistributedSink::writeSplitAsync(const Block & block) void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) { - OpenTelemetry::SpanHolder span("DistributedSink::writeAsyncImpl()"); - const auto & shard_info = cluster->getShardsInfo()[shard_id]; const auto & settings = context->getSettingsRef(); Block block_to_send = removeSuperfluousColumns(block); - span.addAttribute("clickhouse.shard_num", shard_info.shard_num); - span.addAttribute("clickhouse.written_rows", block.rows()); - if (shard_info.hasInternalReplication()) { if (shard_info.isLocal() && settings.prefer_localhost_replica) /// Prefer insert into current instance directly - writeToLocal(block_to_send, shard_info.getLocalNodeCount()); + writeToLocal(shard_info, block_to_send, shard_info.getLocalNodeCount()); else { const auto & path = shard_info.insertPathForInternalReplication( @@ -631,13 +630,13 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) settings.use_compact_format_in_distributed_parts_names); if (path.empty()) throw Exception("Directory name for async inserts is empty", ErrorCodes::LOGICAL_ERROR); - writeToShard(block_to_send, {path}); + writeToShard(shard_info, block_to_send, {path}); } } else { if (shard_info.isLocal() && settings.prefer_localhost_replica) - writeToLocal(block_to_send, shard_info.getLocalNodeCount()); + writeToLocal(shard_info, block_to_send, shard_info.getLocalNodeCount()); std::vector dir_names; for (const auto & address : cluster->getShardsAddresses()[shard_id]) @@ -645,7 +644,7 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) dir_names.push_back(address.toFullString(settings.use_compact_format_in_distributed_parts_names)); if (!dir_names.empty()) - writeToShard(block_to_send, dir_names); + writeToShard(shard_info, block_to_send, dir_names); } } @@ -666,9 +665,10 @@ void DistributedSink::writeToLocal(const Block & block, size_t repeats) } -void DistributedSink::writeToShard(const Block & block, const std::vector & dir_names) +void DistributedSink::writeToShard(const Cluster::ShardInfo& shard_info, const Block & block, const std::vector & dir_names) { OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); + span.addAttribute("clickhouse.shard_num", shard_info.shard_num); const auto & settings = context->getSettingsRef(); const auto & distributed_settings = storage.getDistributedSettingsRef(); @@ -759,6 +759,11 @@ void DistributedSink::writeToShard(const Block & block, const std::vectorstorage.cluster_name, header_buf); + writeStringBinary(this->storage.getStorageID().getFullNameNotQuoted(), header_buf); + writeStringBinary(this->storage.remote_database + "." + this->storage.remote_table, header_buf); + /// Add new fields here, for example: /// writeVarUInt(my_new_data, header_buf); /// And note that it is safe, because we have checksum and size for header. diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 668cec22e8b..5d7a5268865 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -69,9 +69,9 @@ private: Block removeSuperfluousColumns(Block block) const; /// Increments finished_writings_count after each repeat. - void writeToLocal(const Block & block, size_t repeats); + void writeToLocal(const Cluster::ShardInfo& shard_info, const Block & block, size_t repeats); - void writeToShard(const Block & block, const std::vector & dir_names); + void writeToShard(const Cluster::ShardInfo& shard_info, const Block & block, const std::vector & dir_names); /// Performs synchronous insertion to remote nodes. If timeout_exceeded flag was set, throws. From 6ab1549d6c32182253586c0c7714a2ecce7a8fd1 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Mon, 5 Sep 2022 16:40:48 +0800 Subject: [PATCH 364/582] Update writeToLocal to record related info --- src/Storages/Distributed/DistributedSink.cpp | 28 +++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 0e379a7bd89..dc33cfa4b60 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -649,19 +649,33 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) } -void DistributedSink::writeToLocal(const Block & block, size_t repeats) +void DistributedSink::writeToLocal(const Cluster::ShardInfo& shard_info, const Block & block, size_t repeats) { OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("db.statement", this->query_string); + span.addAttribute("clickhouse.shard_num", shard_info.shard_num); + span.addAttribute("clickhouse.cluster", this->storage.cluster_name); + span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); + span.addAttribute("clickhouse.remote", [this]() { return storage.remote_database + "." + storage.remote_table; }); + span.addAttribute("clickhouse.rows", [&block]() { return std::to_string(block.rows()); }); + span.addAttribute("clickhouse.bytes", [&block]() { return std::to_string(block.bytes()); }); - InterpreterInsertQuery interp(query_ast, context, allow_materialized); + try + { + InterpreterInsertQuery interp(query_ast, context, allow_materialized); - auto block_io = interp.execute(); - PushingPipelineExecutor executor(block_io.pipeline); + auto block_io = interp.execute(); + PushingPipelineExecutor executor(block_io.pipeline); - executor.start(); - writeBlockConvert(executor, block, repeats, log); - executor.finish(); + executor.start(); + writeBlockConvert(executor, block, repeats, log); + executor.finish(); + } + catch (...) + { + span.addAttribute(std::current_exception()); + throw; + } } From 313747688e76825d33ead8190f0d285d7f4e2df2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Sep 2022 22:10:53 +0200 Subject: [PATCH 365/582] Ignore memory limits while logging Otherwise there can be pretty noisy messages, like here [1]: Cannot add message to the log: Code: 241. DB::Exception: Memory limit (for user) exceeded: would use 42.02 MiB (attempt to allocate chunk of 2101278 bytes), maximum: 4.77 MiB. OvercommitTracker decision: Waiting timeout for memory to be freed is reached. (MEMORY_LIMIT_EXCEEDED), Stack trace (when copying this message, always include the lines below): [1]: https://s3.amazonaws.com/clickhouse-test-reports/0/efc74e33e580b7d3ff33c3e269db86addfaa6b51/stateless_tests__msan__[2/3].html And it should be okay to ignore server limits too, since it always better to have a proper log message in a proper place, not stderr which does not inspected usually. Follow-up for: #24069 Signed-off-by: Azat Khuzhin --- src/Loggers/OwnSplitChannel.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 933fc09d3e4..b1502cc4558 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -46,6 +46,8 @@ void OwnSplitChannel::log(const Poco::Message & msg) void OwnSplitChannel::tryLogSplit(const Poco::Message & msg) { + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); + try { logSplit(msg); @@ -62,8 +64,6 @@ void OwnSplitChannel::tryLogSplit(const Poco::Message & msg) /// but let's log it into the stderr at least. catch (...) { - LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - const std::string & exception_message = getCurrentExceptionMessage(true); const std::string & message = msg.getText(); From 4e05c08360d2a10131d856fd2a795c52e4ef5c06 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Sep 2022 12:05:45 +0200 Subject: [PATCH 366/582] tests: remove 01594_too_low_memory_limits Since after memory limits ignored in logging it is not possible to test it. Signed-off-by: Azat Khuzhin --- .../01594_too_low_memory_limits.config.xml | 35 ------ .../01594_too_low_memory_limits.reference | 0 .../01594_too_low_memory_limits.sh | 108 ------------------ 3 files changed, 143 deletions(-) delete mode 100644 tests/queries/0_stateless/01594_too_low_memory_limits.config.xml delete mode 100644 tests/queries/0_stateless/01594_too_low_memory_limits.reference delete mode 100755 tests/queries/0_stateless/01594_too_low_memory_limits.sh diff --git a/tests/queries/0_stateless/01594_too_low_memory_limits.config.xml b/tests/queries/0_stateless/01594_too_low_memory_limits.config.xml deleted file mode 100644 index 0c286bfbd21..00000000000 --- a/tests/queries/0_stateless/01594_too_low_memory_limits.config.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - trace - true - - - 9000 - - ./ - - 0 - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - diff --git a/tests/queries/0_stateless/01594_too_low_memory_limits.reference b/tests/queries/0_stateless/01594_too_low_memory_limits.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/01594_too_low_memory_limits.sh b/tests/queries/0_stateless/01594_too_low_memory_limits.sh deleted file mode 100755 index b513a947bd9..00000000000 --- a/tests/queries/0_stateless/01594_too_low_memory_limits.sh +++ /dev/null @@ -1,108 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-fasttest -# Tag no-tsan: requires jemalloc to track small allocations -# Tag no-asan: requires jemalloc to track small allocations -# Tag no-ubsan: requires jemalloc to track small allocations -# Tag no-msan: requires jemalloc to track small allocations - -# -# Regression for INSERT SELECT, that abnormally terminates the server -# in case of too small memory limits. -# -# NOTE: After #24483 had been merged the only place where the allocation may -# fail is the insert into PODArray in DB::OwnSplitChannel::log, but after -# #24069 those errors will be ignored, so to check new behaviour separate -# server is required. -# - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -server_opts=( - "--config-file=$CURDIR/$(basename "${BASH_SOURCE[0]}" .sh).config.xml" - "--" - # to avoid multiple listen sockets (complexity for port discovering) - "--listen_host=127.1" - # we will discover the real port later. - "--tcp_port=0" - "--shutdown_wait_unfinished=0" -) -CLICKHOUSE_WATCHDOG_ENABLE=0 $CLICKHOUSE_SERVER_BINARY "${server_opts[@]}" >clickhouse-server.log 2>clickhouse-server.stderr & -server_pid=$! - -trap cleanup EXIT -function cleanup() -{ - kill -9 $server_pid - - echo "Test failed. Server log:" - cat clickhouse-server.log - cat clickhouse-server.stderr - rm -f clickhouse-server.log - rm -f clickhouse-server.stderr - - exit 1 -} - -server_port= -i=0 retries=300 -# wait until server will start to listen (max 30 seconds) -while [[ -z $server_port ]] && [[ $i -lt $retries ]]; do - server_port=$(lsof -n -a -P -i tcp -s tcp:LISTEN -p $server_pid 2>/dev/null | awk -F'[ :]' '/LISTEN/ { print $(NF-1) }') - ((++i)) - sleep 0.1 - if ! kill -0 $server_pid >& /dev/null; then - echo "No server (pid $server_pid)" - break - fi -done -if [[ -z $server_port ]]; then - echo "Cannot wait for LISTEN socket" >&2 - exit 1 -fi - -# wait for the server to start accepting tcp connections (max 30 seconds) -i=0 retries=300 -while ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do - sleep 0.1 - if ! kill -0 $server_pid >& /dev/null; then - echo "No server (pid $server_pid)" - break - fi -done -if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1'; then - echo "Cannot wait until server will start accepting connections on " >&2 - exit 1 -fi - -# it is not mandatory to use existing table since it fails earlier, hence just a placeholder. -# this is format of INSERT SELECT, that pass these settings exactly for INSERT query not the SELECT -if $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null --send_logs_level=warning --max_memory_usage=1 --max_untracked_memory=1 -q 'insert into placeholder_table_name select * from numbers_mt(65535)' >& /dev/null; then - echo "INSERT SELECT should fail" >&2 - exit 1 -fi - -# no sleep, since flushing to stderr should not be buffered. -if ! grep -E -q 'Cannot add message to the log: Code: 60.*placeholder_table_name' clickhouse-server.stderr; then - echo "Adding message to the log should fail" >&2 - exit 1 -fi - -# check that server is still alive -$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'SELECT 1' - -# send TERM and save the error code to ensure that it is 0 (EXIT_SUCCESS) -kill $server_pid -wait $server_pid -return_code=$? - -trap '' EXIT -if [ $return_code != 0 ]; then - cat clickhouse-server.log - cat clickhouse-server.stderr -fi -rm -f clickhouse-server.log -rm -f clickhouse-server.stderr - -exit $return_code From 0f305f8a7ad8765deaf7a49d24f493e1d19b4740 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Sep 2022 12:20:21 +0200 Subject: [PATCH 367/582] Fix test --- programs/local/LocalServer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 83af6e1b5f0..fa43c9f1283 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -368,8 +368,9 @@ try UseSSL use_ssl; ThreadStatus thread_status; SCOPE_EXIT_SAFE({ - if (connection) - connection.reset(); + /// Context should not live longer than thread_status. + global_context.reset(); + shared_context.reset(); }); StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true)); From 97de19d7ea4f89f615aa6f1158629fc695bcc718 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Sep 2022 12:12:22 +0200 Subject: [PATCH 368/582] fix tests --- tests/queries/0_stateless/00284_external_aggregation.sql | 1 + .../queries/0_stateless/01281_group_by_limit_memory_tracking.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00284_external_aggregation.sql b/tests/queries/0_stateless/00284_external_aggregation.sql index a42dd91b6a5..d19f9f5aee8 100644 --- a/tests/queries/0_stateless/00284_external_aggregation.sql +++ b/tests/queries/0_stateless/00284_external_aggregation.sql @@ -8,6 +8,7 @@ SET group_by_two_level_threshold_bytes = 50000000; SELECT sum(k), sum(c) FROM (SELECT number AS k, count() AS c FROM (SELECT * FROM system.numbers LIMIT 10000000) GROUP BY k); SELECT sum(k), sum(c), max(u) FROM (SELECT number AS k, count() AS c, uniqArray(range(number % 16)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k); +SET max_memory_usage = 0; SET group_by_two_level_threshold = 100000; SET max_bytes_before_external_group_by = '1Mi'; diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh index 7f353b84a80..b48958a18f6 100755 --- a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -32,7 +32,7 @@ function execute_group_by() # max_memory_usage_for_user is installed to 0 once there are no more # queries for user. local opts=( - "--max_memory_usage_for_user="$((150<<20)) + "--max_memory_usage_for_user="$((200<<20)) "--max_threads=2" ) execute_null "${opts[@]}" <<<'SELECT uniq(number) FROM numbers_mt(1e6) GROUP BY number % 5e5' From 33e530392e65e005b31e976d81edb55e2c29cb98 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 5 Sep 2022 12:38:44 +0200 Subject: [PATCH 369/582] Check grants for databases in case of renaming databases (#40906) --- src/Interpreters/InterpreterRenameQuery.cpp | 29 ++++++++++++++----- src/Interpreters/InterpreterRenameQuery.h | 8 ++++- .../02416_rename_database_rbac.reference | 0 .../0_stateless/02416_rename_database_rbac.sh | 23 +++++++++++++++ 4 files changed, 51 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02416_rename_database_rbac.reference create mode 100755 tests/queries/0_stateless/02416_rename_database_rbac.sh diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index f4dbbaec16d..c22863ef8e5 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextPtr context_) @@ -31,11 +32,11 @@ BlockIO InterpreterRenameQuery::execute() if (!rename.cluster.empty()) { DDLQueryOnClusterParams params; - params.access_to_check = getRequiredAccess(); + params.access_to_check = getRequiredAccess(rename.database ? RenameType::RenameDatabase : RenameType::RenameTable); return executeDDLQueryOnCluster(query_ptr, getContext(), params); } - getContext()->checkAccess(getRequiredAccess()); + getContext()->checkAccess(getRequiredAccess(rename.database ? RenameType::RenameDatabase : RenameType::RenameTable)); String path = getContext()->getPath(); String current_database = getContext()->getCurrentDatabase(); @@ -165,18 +166,30 @@ BlockIO InterpreterRenameQuery::executeToDatabase(const ASTRenameQuery &, const return {}; } -AccessRightsElements InterpreterRenameQuery::getRequiredAccess() const +AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRenameQuery::RenameType type) const { AccessRightsElements required_access; const auto & rename = query_ptr->as(); for (const auto & elem : rename.elements) { - required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.from.database, elem.from.table); - required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.database, elem.to.table); - if (rename.exchange) + if (type == RenameType::RenameTable) { - required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.database, elem.from.table); - required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.database, elem.to.table); + required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.from.database, elem.from.table); + required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.database, elem.to.table); + if (rename.exchange) + { + required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT , elem.from.database, elem.from.table); + required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.database, elem.to.table); + } + } + else if (type == RenameType::RenameDatabase) + { + required_access.emplace_back(AccessType::SELECT | AccessType::DROP_DATABASE, elem.from.database); + required_access.emplace_back(AccessType::CREATE_DATABASE | AccessType::INSERT, elem.to.database); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type of rename query"); } } return required_access; diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index 194f6266634..6141e8c1585 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -63,7 +63,13 @@ private: BlockIO executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards); BlockIO executeToDatabase(const ASTRenameQuery & rename, const RenameDescriptions & descriptions); - AccessRightsElements getRequiredAccess() const; + enum class RenameType + { + RenameTable, + RenameDatabase + }; + + AccessRightsElements getRequiredAccess(RenameType type) const; ASTPtr query_ptr; bool renamed_instead_of_exchange{false}; diff --git a/tests/queries/0_stateless/02416_rename_database_rbac.reference b/tests/queries/0_stateless/02416_rename_database_rbac.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02416_rename_database_rbac.sh b/tests/queries/0_stateless/02416_rename_database_rbac.sh new file mode 100755 index 00000000000..c319136d29c --- /dev/null +++ b/tests/queries/0_stateless/02416_rename_database_rbac.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS database_02416; +CREATE DATABASE database_02416; + +DROP USER IF EXISTS user_test_02416; +CREATE USER user_test_02416 IDENTIFIED WITH plaintext_password BY 'user_test_02416'; + +GRANT CREATE DATABASE ON *.* TO 'user_test_02416' WITH GRANT OPTION; +GRANT DROP DATABASE ON *.* TO 'user_test_02416' WITH GRANT OPTION; +REVOKE DROP DATABASE ON database_02416.* FROM 'user_test_02416'; +GRANT CREATE TABLE ON *.* TO 'user_test_02416' WITH GRANT OPTION; +GRANT DROP TABLE ON *.* TO 'user_test_02416' WITH GRANT OPTION; +""" +${CLICKHOUSE_CLIENT} --multiline --multiquery --user user_test_02416 --password user_test_02416 -q """ +RENAME DATABASE user_test_02416 to aaaaaaaaa; -- { serverError 497 } +""" From 8c3c3e766751ea9a1765b6f695b58446e5135644 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 5 Sep 2022 12:39:39 +0200 Subject: [PATCH 370/582] Minor update doc for mysql_port --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 24e08fe1fcd..b7fe7d49b7b 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1452,7 +1452,7 @@ Port for communicating with clients over MySQL protocol. **Possible values** -Positive integer. +Positive integer to specify the port number to listen to or empty value to disable. Example @@ -1466,7 +1466,7 @@ Port for communicating with clients over PostgreSQL protocol. **Possible values** -Positive integer. +Positive integer to specify the port number to listen to or empty value to disable. Example From 330e3324b03ccba8afde778f76fb6e36d1045536 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 5 Sep 2022 12:46:26 +0000 Subject: [PATCH 371/582] Fix test --- .../1_stateful/00175_obfuscator_schema_inference.reference | 6 +++--- .../queries/1_stateful/00175_obfuscator_schema_inference.sh | 4 ++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/queries/1_stateful/00175_obfuscator_schema_inference.reference b/tests/queries/1_stateful/00175_obfuscator_schema_inference.reference index a704091b1a9..bd7f726bffd 100644 --- a/tests/queries/1_stateful/00175_obfuscator_schema_inference.reference +++ b/tests/queries/1_stateful/00175_obfuscator_schema_inference.reference @@ -1,4 +1,4 @@ -403499 +403489 1000 320 171 23 -2500 569 354 13 -2500 569 354 13 +2500 597 332 14 +2500 597 332 14 diff --git a/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh b/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh index d8ee72aa1c0..8ff0d2fa648 100755 --- a/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh +++ b/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh @@ -4,6 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# Compared to explicitly specifying the structure of the input, +# schema inference adds Nullable(T) to all types, so the model and the results +# are a bit different from test '00175_obfuscator_schema_inference.sh' + $CLICKHOUSE_CLIENT --max_threads 1 --query="SELECT URL, Title, SearchPhrase FROM test.hits LIMIT 1000" > "${CLICKHOUSE_TMP}"/data.tsv # Test obfuscator without saving the model From e1def723f8dea97b5fe71c09fcf15131e539d48c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Sep 2022 12:04:10 +0000 Subject: [PATCH 372/582] Add special x86-SSE2-only build --- .github/workflows/master.yml | 48 ++++++++++++++++++++++++++++++ .github/workflows/pull_request.yml | 46 ++++++++++++++++++++++++++++ cmake/cpu_features.cmake | 17 +++++++++++ docker/packager/packager | 4 +++ tests/ci/ci_config.py | 11 +++++++ 5 files changed, 126 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e8e3deceef5..d3a303eb7ab 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -923,6 +923,53 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAmd64SSE2: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_amd64sse2 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -1011,6 +1058,7 @@ jobs: - BuilderBinFreeBSD # - BuilderBinGCC - BuilderBinPPC64 + - BuilderBinAmd64SSE2 - BuilderBinClangTidy - BuilderDebShared runs-on: [self-hosted, style-checker] diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7901008a7db..c100b079ed5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -935,6 +935,51 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAmd64SSE2: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_amd64sse2 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -1023,6 +1068,7 @@ jobs: - BuilderBinFreeBSD # - BuilderBinGCC - BuilderBinPPC64 + - BuilderBinAmd64SSE2 - BuilderBinClangTidy - BuilderDebShared runs-on: [self-hosted, style-checker] diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 1fc3c2db804..218b4deedce 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -24,6 +24,23 @@ option (ENABLE_BMI "Use BMI instructions on x86_64" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) +# X86: Allow compilation for a SSE2-only target machine. Done by a special build in CI for embedded or very old hardware. +option (NO_SSE3_OR_HIGHER "Disable SSE3 or higher on x86_64" 0) +if (NO_SSE3_OR_HIGHER) + SET(ENABLE_SSSE3 0) + SET(ENABLE_SSE41 0) + SET(ENABLE_SSE42 0) + SET(ENABLE_PCLMULQDQ 0) + SET(ENABLE_POPCNT 0) + SET(ENABLE_AVX 0) + SET(ENABLE_AVX2 0) + SET(ENABLE_AVX512 0) + SET(ENABLE_AVX512_VBMI 0) + SET(ENABLE_BMI 0) + SET(ENABLE_AVX2_FOR_SPEC_OP 0) + SET(ENABLE_AVX512_FOR_SPEC_OP 0) +endif() + option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) if (ARCH_NATIVE) diff --git a/docker/packager/packager b/docker/packager/packager index 66eb568d460..f878444d4bc 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -190,6 +190,9 @@ def parse_env_variables( cc = compiler result.append("DEB_ARCH=amd64") + if compiler.endswith("-amd64sse2"): + cmake_flags.append('-DNO_SSE3_OR_HIGHER=1') + cxx = cc.replace("gcc", "g++").replace("clang", "clang++") if package_type == "deb": @@ -339,6 +342,7 @@ if __name__ == "__main__": "clang-14-darwin-aarch64", "clang-14-aarch64", "clang-14-ppc64le", + "clang-14-amd64sse2", "clang-14-freebsd", "gcc-11", ), diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 3d0513bca47..b49e91a9e79 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -161,6 +161,16 @@ CI_CONFIG = { "tidy": "disable", "with_coverage": False, }, + "binary_amd64sse2": { + "compiler": "clang-14-amd64sse2", + "build_type": "", + "sanitizer": "", + "package_type": "binary", + "static_binary_name": "amd64sse2", + "libraries": "static", + "tidy": "disable", + "with_coverage": False, + }, }, "builds_report_config": { "ClickHouse build check": [ @@ -182,6 +192,7 @@ CI_CONFIG = { "binary_freebsd", "binary_darwin_aarch64", "binary_ppc64le", + "binary_amd64sse2", ], }, "tests_config": { From 4d42597b789dc548729b3decdddd4e9dc8611d2a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 10:09:52 +0000 Subject: [PATCH 373/582] Make test less heavy --- tests/queries/0_stateless/02416_keeper_map.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index bde07db1328..3866c33f03d 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -10,11 +10,11 @@ CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperM DROP TABLE IF EXISTS 02416_test SYNC; CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); -INSERT INTO 02416_test SELECT '1_1', number FROM numbers(10000); +INSERT INTO 02416_test SELECT '1_1', number FROM numbers(1000); SELECT COUNT(1) == 1 FROM 02416_test; -INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT COUNT(1) == 10000 FROM 02416_test; +INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(1000); +SELECT COUNT(1) == 1000 FROM 02416_test; SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02416_test LIMIT 32 SETTINGS max_block_size = 1); SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', '900_1'); @@ -24,9 +24,9 @@ DROP TABLE IF EXISTS 02416_test_memory; CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(k); CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory; -INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000) group by k; -INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000) group by k; SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test_memory) B USING a ORDER BY a; From 2724b6753706e38c7acac73ed9f83dce3fd03423 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Sep 2022 15:49:56 +0200 Subject: [PATCH 374/582] ci/stress: clear dmesg before run to fix "OOM in dmesg" check CI: https://s3.amazonaws.com/clickhouse-test-reports/40772/afa137ae2b6108e72c2d6e43556a04548afa2ea9/stress_test__ubsan_.html Signed-off-by: Azat Khuzhin --- docker/test/stress/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index f8ecdf1aa21..dc632f4184f 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -3,6 +3,9 @@ # shellcheck disable=SC2086 # shellcheck disable=SC2024 +# Avoid overlaps with previous runs +dmesg --clear + set -x # Thread Fuzzer allows to check more permutations of possible thread scheduling From 225773634c32217ad6cb04eccfa6b72b67493eae Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 5 Sep 2022 15:54:28 +0200 Subject: [PATCH 375/582] fix integration test for profile events s3 --- tests/integration/test_profile_events_s3/test.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index aa578a1273a..a0f664df000 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -33,9 +33,10 @@ def cluster(): init_list = { "ReadBufferFromS3Bytes": 0, - "S3ReadMicroseconds": 0, "ReadBufferFromS3Microseconds": 0, "ReadBufferFromS3RequestsErrors": 0, + "WriteBufferFromS3Bytes": 0, + "S3ReadMicroseconds": 0, "S3ReadRequestsCount": 0, "S3ReadRequestsErrorsTotal": 0, "S3ReadRequestsErrors503": 0, @@ -45,7 +46,16 @@ init_list = { "S3WriteRequestsErrorsTotal": 0, "S3WriteRequestsErrors503": 0, "S3WriteRequestsRedirects": 0, - "WriteBufferFromS3Bytes": 0, + "DiskS3ReadMicroseconds": 0, + "DiskS3ReadRequestsCount": 0, + "DiskS3ReadRequestsErrorsTotal": 0, + "DiskS3ReadRequestsErrors503": 0, + "DiskS3ReadRequestsRedirects": 0, + "DiskS3WriteMicroseconds": 0, + "DiskS3WriteRequestsCount": 0, + "DiskS3WriteRequestsErrorsTotal": 0, + "DiskS3WriteRequestsErrors503": 0, + "DiskS3WriteRequestsRedirects": 0, } From 15b0b8ff068793770fee2c76ca33c565350234fc Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 5 Sep 2022 17:01:19 +0300 Subject: [PATCH 376/582] Fix FreezeMetaData::load --- src/Storages/Freeze.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index f8fe1247816..d94f25ebf9b 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -90,8 +90,9 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) readBoolText(is_remote, buffer); DB::assertChar('\n', buffer); } - auto unescaped_replica_name = unescapeForFileName(replica_name); + std::string unescaped_replica_name; readString(unescaped_replica_name, buffer); + replica_name = unescapeForFileName(unescaped_replica_name); DB::assertChar('\n', buffer); readString(zookeeper_name, buffer); DB::assertChar('\n', buffer); From 360bc93c873152b7efcfd901c36e9d17950428cf Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Sep 2022 16:16:49 +0200 Subject: [PATCH 377/582] Turn off always green mode --- tests/ci/commit_status_helper.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index ae1900ae045..8b9d28502c1 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -19,9 +19,6 @@ def override_status(status, check_name, invert=False): if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): return "success" - if "s3 storage" in check_name and ("debug" in check_name or "thread" in check_name): - return "success" - if invert: if status == "success": return "error" From 1338f7d65e8db4dfe8da6760d4c1133a477c8bc9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 5 Sep 2022 15:41:52 +0000 Subject: [PATCH 378/582] disable inode lock for macos and freebsd --- utils/self-extracting-executable/decompressor.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index ad4cd5b2f63..c997526d38d 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -361,6 +361,8 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #endif +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) + uint32_t getInode(const char * self) { std::ifstream maps("/proc/self/maps"); @@ -386,6 +388,8 @@ uint32_t getInode(const char * self) return 0; } +#endif + int main(int/* argc*/, char* argv[]) { char self[4096] = {0}; @@ -409,6 +413,7 @@ int main(int/* argc*/, char* argv[]) else name = file_path; +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint32_t inode = getInode(self); if (inode == 0) @@ -460,6 +465,7 @@ int main(int/* argc*/, char* argv[]) printf("No target executable - decompression only was performed.\n"); return 0; } +#endif int input_fd = open(self, O_RDONLY); if (input_fd == -1) @@ -522,19 +528,21 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed write(lock, "1", 1); - +#endif execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } - +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 ftruncate(lock, 0); +#endif printf("No target executable - decompression only was performed.\n"); } From 438ed368a1e56af77367b42f411b099992c5ba38 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Sep 2022 17:49:00 +0000 Subject: [PATCH 379/582] fix: correct compiler parsing --- docker/packager/packager | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index f878444d4bc..363be9ab2dd 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -130,6 +130,7 @@ def parse_env_variables( ARM_SUFFIX = "-aarch64" FREEBSD_SUFFIX = "-freebsd" PPC_SUFFIX = "-ppc64le" + AMD64_SSE2_SUFFIX = "-amd64sse2" result = [] result.append("OUTPUT_DIR=/output") @@ -141,6 +142,7 @@ def parse_env_variables( is_cross_arm = compiler.endswith(ARM_SUFFIX) is_cross_ppc = compiler.endswith(PPC_SUFFIX) is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX) + is_amd64_sse2 = compiler.endswidth(AMD64_SSE2_SUFFIX) if is_cross_darwin: cc = compiler[: -len(DARWIN_SUFFIX)] @@ -186,13 +188,13 @@ def parse_env_variables( cmake_flags.append( "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-ppc64le.cmake" ) + elif is_amd64_sse2: + cc = compiler[: -len(AMD64_SSE2_SUFFIX)] + result.append("DEB_ARCH=amd64") else: cc = compiler result.append("DEB_ARCH=amd64") - if compiler.endswith("-amd64sse2"): - cmake_flags.append('-DNO_SSE3_OR_HIGHER=1') - cxx = cc.replace("gcc", "g++").replace("clang", "clang++") if package_type == "deb": From 83514fa2ef3e0f8b1465622b5f162f67997790b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Sep 2022 18:55:00 +0200 Subject: [PATCH 380/582] Refactor --- src/Common/FieldVisitorToString.cpp | 6 ++ src/Common/FieldVisitorToString.h | 5 +- .../getDictionaryConfigurationFromAST.cpp | 19 ++---- src/Disks/IDisk.h | 11 ++- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- .../DiskObjectStorageMetadata.cpp | 8 +++ .../DiskObjectStorageMetadata.h | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 1 - src/Storages/MergeTree/AlterConversions.h | 24 +++++++ .../IMergeTreeDataPartInfoForReader.h | 68 +++++++++++++++++++ src/Storages/MergeTree/IMergeTreeReader.cpp | 29 ++++---- src/Storages/MergeTree/IMergeTreeReader.h | 9 +-- .../LoadedMergeTreeDataPartInfoForReader.h | 55 +++++++++++++++ src/Storages/MergeTree/MarkRange.cpp | 12 ++++ src/Storages/MergeTree/MarkRange.h | 2 + .../MergeTreeBaseSelectProcessor.cpp | 53 +++++++++++---- .../MergeTree/MergeTreeBaseSelectProcessor.h | 14 ++++ .../MergeTree/MergeTreeBlockReadUtils.cpp | 50 ++++++++------ .../MergeTree/MergeTreeBlockReadUtils.h | 15 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 15 +--- .../MergeTree/MergeTreeDataPartCompact.cpp | 42 +++++++----- .../MergeTree/MergeTreeDataPartCompact.h | 5 ++ .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- .../MergeTree/MergeTreeDataPartWide.cpp | 47 +++++++------ .../MergeTree/MergeTreeDataPartWide.h | 5 ++ .../MergeTree/MergeTreeIndexGranularityInfo.h | 2 + .../MergeTree/MergeTreeRangeReader.cpp | 9 +-- src/Storages/MergeTree/MergeTreeReadPool.cpp | 5 +- src/Storages/MergeTree/MergeTreeReadPool.h | 14 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 42 ++++++------ .../MergeTree/MergeTreeReaderCompact.h | 4 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 5 +- .../MergeTree/MergeTreeReaderInMemory.h | 1 + .../MergeTree/MergeTreeReaderWide.cpp | 24 +++---- src/Storages/MergeTree/MergeTreeReaderWide.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 3 +- .../MergeTree/MergeTreeSequentialSource.cpp | 3 +- .../System/StorageSystemRemoteDataPaths.cpp | 11 ++- 39 files changed, 444 insertions(+), 185 deletions(-) create mode 100644 src/Storages/MergeTree/AlterConversions.h create mode 100644 src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h create mode 100644 src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h diff --git a/src/Common/FieldVisitorToString.cpp b/src/Common/FieldVisitorToString.cpp index 7d93cfba78f..e0e138d744c 100644 --- a/src/Common/FieldVisitorToString.cpp +++ b/src/Common/FieldVisitorToString.cpp @@ -145,5 +145,11 @@ String FieldVisitorToString::operator() (const Object & x) const } +String convertFieldToString(const Field & field) +{ + if (field.getType() == Field::Types::Which::String) + return field.get(); + return applyVisitor(FieldVisitorToString(), field); } +} diff --git a/src/Common/FieldVisitorToString.h b/src/Common/FieldVisitorToString.h index 324a4aa73d5..cca29a8f7e0 100644 --- a/src/Common/FieldVisitorToString.h +++ b/src/Common/FieldVisitorToString.h @@ -31,5 +31,8 @@ public: String operator() (const bool & x) const; }; -} +/// Get value from field and convert it to string. +/// Also remove quotes from strings. +String convertFieldToString(const Field & field); +} diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 7d8253c47ce..e19495a27a3 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -44,15 +44,6 @@ struct AttributeConfiguration using AttributeNameToConfiguration = std::unordered_map; -/// Get value from field and convert it to string. -/// Also remove quotes from strings. -String getFieldAsString(const Field & field) -{ - if (field.getType() == Field::Types::Which::String) - return field.get(); - return applyVisitor(FieldVisitorToString(), field); -} - String getAttributeExpression(const ASTDictionaryAttributeDeclaration * dict_attr) { if (!dict_attr->expression) @@ -61,7 +52,7 @@ String getAttributeExpression(const ASTDictionaryAttributeDeclaration * dict_att /// EXPRESSION PROPERTY should be expression or string String expression_str; if (const auto * literal = dict_attr->expression->as(); literal && literal->value.getType() == Field::Types::String) - expression_str = getFieldAsString(literal->value); + expression_str = convertFieldToString(literal->value); else expression_str = queryToString(dict_attr->expression); @@ -275,7 +266,7 @@ void buildSingleAttribute( AutoPtr null_value_element(doc->createElement("null_value")); String null_value_str; if (dict_attr->default_value) - null_value_str = getFieldAsString(dict_attr->default_value->as()->value); + null_value_str = convertFieldToString(dict_attr->default_value->as()->value); AutoPtr null_value(doc->createTextNode(null_value_str)); null_value_element->appendChild(null_value); attribute_element->appendChild(null_value_element); @@ -452,7 +443,7 @@ void buildConfigurationFromFunctionWithKeyValueArguments( } else if (const auto * literal = pair->second->as()) { - AutoPtr value(doc->createTextNode(getFieldAsString(literal->value))); + AutoPtr value(doc->createTextNode(convertFieldToString(literal->value))); current_xml_element->appendChild(value); } else if (const auto * list = pair->second->as()) @@ -473,7 +464,7 @@ void buildConfigurationFromFunctionWithKeyValueArguments( Field value; result->get(0, value); - AutoPtr text_value(doc->createTextNode(getFieldAsString(value))); + AutoPtr text_value(doc->createTextNode(convertFieldToString(value))); current_xml_element->appendChild(text_value); } else @@ -519,7 +510,7 @@ void buildSourceConfiguration( { AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(getFieldAsString(value))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(value))); setting_change_element->appendChild(setting_value); } } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index bfbdba0e050..81f33b27056 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -239,7 +239,16 @@ public: } /// For one local path there might be multiple remote paths in case of Log family engines. - using LocalPathWithObjectStoragePaths = std::pair; + struct LocalPathWithObjectStoragePaths + { + std::string local_path; + std::string common_prefix_for_objects; + StoredObjects objects; + + LocalPathWithObjectStoragePaths( + const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_) + : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {} + }; virtual void getRemotePathsRecursive(const String &, std::vector &) { diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 4e9dea7f481..913fd76bf8a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -127,7 +127,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: { try { - paths_map.emplace_back(local_path, getStorageObjects(local_path)); + paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path)); } catch (const Exception & e) { diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index ae03915d944..f18debe8a8b 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -68,6 +68,14 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) } } +void DiskObjectStorageMetadata::createFromSingleObject(const std::string & relative_path, size_t bytes_size, size_t ref_count_, bool read_only_) +{ + storage_objects.emplace_back(relative_path, bytes_size); + total_size = bytes_size; + ref_count = ref_count_; + read_only = read_only_; +} + void DiskObjectStorageMetadata::deserializeFromString(const std::string & data) { ReadBufferFromString buf(data); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index d3ea5795dd3..09e0f4ee85b 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -50,6 +50,7 @@ public: void deserialize(ReadBuffer & buf); void deserializeFromString(const std::string & data); + void createFromSingleObject(const std::string & relative_path, size_t bytes_size, size_t ref_count_, bool is_read_only_); void serialize(WriteBuffer & buf, bool sync) const; std::string serializeToString() const; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6cae86c9717..60bf8d6a15c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -179,7 +179,6 @@ Pipe ReadFromMergeTree::readFromPool( sum_marks, min_marks_for_concurrent_read, std::move(parts_with_range), - data, storage_snapshot, prewhere_info, required_columns, diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h new file mode 100644 index 00000000000..0d58499d424 --- /dev/null +++ b/src/Storages/MergeTree/AlterConversions.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Alter conversions which should be applied on-fly for part. Build from of +/// the most recent mutation commands for part. Now we have only rename_map +/// here (from ALTER_RENAME) command, because for all other type of alters +/// we can deduce conversions for part from difference between +/// part->getColumns() and storage->getColumns(). +struct AlterConversions +{ + /// Rename map new_name -> old_name + std::unordered_map rename_map; + + bool isColumnRenamed(const std::string & new_name) const { return rename_map.count(new_name) > 0; } + std::string getColumnOldName(const std::string & new_name) const { return rename_map.at(new_name); } +}; + +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h new file mode 100644 index 00000000000..28f834d661d --- /dev/null +++ b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h @@ -0,0 +1,68 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class IDataPartStorage; +using DataPartStoragePtr = std::shared_ptr; +class MergeTreeIndexGranularity; +struct MergeTreeDataPartChecksums; +struct MergeTreeIndexGranularityInfo; +class ISerialization; +using SerializationPtr = std::shared_ptr; + +/** + * A class which contains all information about a data part that is required + * in order to use MergeTreeDataPartReader's. + * It is a separate interface and not a simple struct because + * otherwise it will need to copy all the information which might not + * be even used (for example, an IndexGranulary class object is quite heavy). + */ +class IMergeTreeDataPartInfoForReader : public WithContext +{ +public: + explicit IMergeTreeDataPartInfoForReader(ContextPtr context_) : WithContext(context_) {} + + virtual ~IMergeTreeDataPartInfoForReader() = default; + + virtual bool isCompactPart() const = 0; + + virtual bool isWidePart() const = 0; + + virtual bool isInMemoryPart() const = 0; + + virtual bool isProjectionPart() const = 0; + + virtual const DataPartStoragePtr & getDataPartStorage() const = 0; + + virtual const NamesAndTypesList & getColumns() const = 0; + + virtual std::optional getColumnPosition(const String & column_name) const = 0; + + virtual String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const = 0; + + virtual const MergeTreeDataPartChecksums & getChecksums() const = 0; + + virtual AlterConversions getAlterConversions() const = 0; + + virtual size_t getMarksCount() const = 0; + + virtual size_t getFileSizeOrZero(const std::string & file_name) const = 0; + + virtual const MergeTreeIndexGranularityInfo & getIndexGranularityInfo() const = 0; + + virtual const MergeTreeIndexGranularity & getIndexGranularity() const = 0; + + virtual SerializationPtr getSerialization(const NameAndTypePair & column) const = 0; + + virtual const SerializationInfoByName & getSerializationInfos() const = 0; + + virtual void reportBroken() = 0; +}; + +using MergeTreeDataPartInfoForReaderPtr = std::shared_ptr; + +} diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 8c861248580..851b0378e6f 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes IMergeTreeReader::IMergeTreeReader( - const MergeTreeData::DataPartPtr & data_part_, + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, @@ -31,19 +31,18 @@ IMergeTreeReader::IMergeTreeReader( const MarkRanges & all_mark_ranges_, const MergeTreeReaderSettings & settings_, const ValueSizeMap & avg_value_size_hints_) - : data_part(data_part_) + : data_part_info_for_read(data_part_info_for_read_) , avg_value_size_hints(avg_value_size_hints_) , uncompressed_cache(uncompressed_cache_) , mark_cache(mark_cache_) , settings(settings_) - , storage(data_part_->storage) , metadata_snapshot(metadata_snapshot_) , all_mark_ranges(all_mark_ranges_) - , alter_conversions(storage.getAlterConversionsForPart(data_part)) + , alter_conversions(data_part_info_for_read->getAlterConversions()) /// For wide parts convert plain arrays of Nested to subcolumns /// to allow to use shared offset column from cache. - , requested_columns(isWidePart(data_part) ? Nested::convertToSubcolumns(columns_) : columns_) - , part_columns(isWidePart(data_part) ? Nested::collect(data_part->getColumns()) : data_part->getColumns()) + , requested_columns(data_part_info_for_read->isWidePart() ? Nested::convertToSubcolumns(columns_) : columns_) + , part_columns(data_part_info_for_read->isWidePart() ? Nested::collect(data_part_info_for_read->getColumns()) : data_part_info_for_read->getColumns()) { columns_to_read.reserve(requested_columns.size()); serializations.reserve(requested_columns.size()); @@ -71,7 +70,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e catch (Exception & e) { /// Better diagnostics. - e.addMessage("(while reading from part " + data_part->data_part_storage->getFullPath() + ")"); + e.addMessage("(while reading from part " + data_part_info_for_read->getDataPartStorage()->getFullPath() + ")"); throw; } } @@ -99,13 +98,13 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } auto dag = DB::evaluateMissingDefaults( - additional_columns, requested_columns, metadata_snapshot->getColumns(), storage.getContext()); + additional_columns, requested_columns, metadata_snapshot->getColumns(), data_part_info_for_read->getContext()); if (dag) { dag->addMaterializingOutputActions(); auto actions = std::make_shared< ExpressionActions>(std::move(dag), - ExpressionActionsSettings::fromSettings(storage.getContext()->getSettingsRef())); + ExpressionActionsSettings::fromSettings(data_part_info_for_read->getContext()->getSettingsRef())); actions->execute(additional_columns); } @@ -117,7 +116,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns catch (Exception & e) { /// Better diagnostics. - e.addMessage("(while reading from part " + data_part->data_part_storage->getFullPath() + ")"); + e.addMessage("(while reading from part " + data_part_info_for_read->getDataPartStorage()->getFullPath() + ")"); throw; } } @@ -151,7 +150,7 @@ SerializationPtr IMergeTreeReader::getSerializationInPart(const NameAndTypePair if (!column_in_part) return IDataType::getSerialization(required_column); - const auto & infos = data_part->getSerializationInfos(); + const auto & infos = data_part_info_for_read->getSerializationInfos(); if (auto it = infos.find(column_in_part->getNameInStorage()); it != infos.end()) return IDataType::getSerialization(*column_in_part, *it->second); @@ -187,7 +186,7 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const copy_block.insert({res_columns[pos], getColumnInPart(*name_and_type).type, name_and_type->name}); } - DB::performRequiredConversions(copy_block, requested_columns, storage.getContext()); + DB::performRequiredConversions(copy_block, requested_columns, data_part_info_for_read->getContext()); /// Move columns from block. name_and_type = requested_columns.begin(); @@ -197,7 +196,7 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const catch (Exception & e) { /// Better diagnostics. - e.addMessage("(while reading from part " + data_part->data_part_storage->getFullPath() + ")"); + e.addMessage("(while reading from part " + data_part_info_for_read->getDataPartStorage()->getFullPath() + ")"); throw; } } @@ -205,11 +204,11 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const String & column_name) const { String table_name = Nested::extractTableName(column_name); - for (const auto & part_column : data_part->getColumns()) + for (const auto & part_column : data_part_info_for_read->getColumns()) { if (typeid_cast(part_column.type.get())) { - auto position = data_part->getColumnPosition(part_column.getNameInStorage()); + auto position = data_part_info_for_read->getColumnPosition(part_column.getNameInStorage()); if (position && Nested::extractTableName(part_column.name) == table_name) return position; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 453563522a5..f88f916908f 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB { @@ -20,7 +22,7 @@ public: using DeserializeBinaryBulkStateMap = std::map; IMergeTreeReader( - const MergeTreeData::DataPartPtr & data_part_, + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, @@ -57,7 +59,7 @@ public: size_t getFirstMarkToRead() const { return all_mark_ranges.front().begin; } - MergeTreeData::DataPartPtr data_part; + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read; protected: /// Returns actual column name in part, which can differ from table metadata. @@ -86,7 +88,6 @@ protected: MergeTreeReaderSettings settings; - const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; MarkRanges all_mark_ranges; @@ -95,7 +96,7 @@ protected: private: /// Alter conversions, which must be applied on fly if required - MergeTreeData::AlterConversions alter_conversions; + AlterConversions alter_conversions; /// Columns that are requested to read. NamesAndTypesList requested_columns; diff --git a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h new file mode 100644 index 00000000000..a16aaa728ae --- /dev/null +++ b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h @@ -0,0 +1,55 @@ +#pragma once +#include +#include + + +namespace DB +{ + +class LoadedMergeTreeDataPartInfoForReader final : public IMergeTreeDataPartInfoForReader +{ +public: + explicit LoadedMergeTreeDataPartInfoForReader(MergeTreeData::DataPartPtr data_part_) + : IMergeTreeDataPartInfoForReader(data_part_->storage.getContext()) + , data_part(data_part_) + {} + + bool isCompactPart() const override { return DB::isCompactPart(data_part); } + + bool isWidePart() const override { return DB::isWidePart(data_part); } + + bool isInMemoryPart() const override { return DB::isInMemoryPart(data_part); } + + bool isProjectionPart() const override { return data_part->isProjectionPart(); } + + const DataPartStoragePtr & getDataPartStorage() const override { return data_part->data_part_storage; } + + const NamesAndTypesList & getColumns() const override { return data_part->getColumns(); } + + std::optional getColumnPosition(const String & column_name) const override { return data_part->getColumnPosition(column_name); } + + AlterConversions getAlterConversions() const override { return data_part->storage.getAlterConversionsForPart(data_part); } + + String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const override { return data_part->getColumnNameWithMinimumCompressedSize(with_subcolumns); } + + const MergeTreeDataPartChecksums & getChecksums() const override { return data_part->checksums; } + + void reportBroken() override { data_part->storage.reportBrokenPart(data_part); } + + size_t getMarksCount() const override { return data_part->getMarksCount(); } + + size_t getFileSizeOrZero(const std::string & file_name) const override { return data_part->getFileSizeOrZero(file_name); } + + const MergeTreeIndexGranularityInfo & getIndexGranularityInfo() const override { return data_part->index_granularity_info; } + + const MergeTreeIndexGranularity & getIndexGranularity() const override { return data_part->index_granularity; } + + const SerializationInfoByName & getSerializationInfos() const override { return data_part->getSerializationInfos(); } + + SerializationPtr getSerialization(const NameAndTypePair & column) const override { return data_part->getSerialization(column.name); } + +private: + MergeTreeData::DataPartPtr data_part; +}; + +} diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index 343c4ecaf22..903940efa94 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -36,4 +36,16 @@ size_t getLastMark(const MarkRanges & ranges) return current_task_last_mark; } +std::string toString(const MarkRanges & ranges) +{ + std::string result; + for (const auto & mark_range : ranges) + { + if (!result.empty()) + result += ", "; + result += "(" + std::to_string(mark_range.begin) + ", " + std::to_string(mark_range.end) + ")"; + } + return result; +} + } diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index 4f32be6ab14..fe02eb056b7 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -32,4 +32,6 @@ using MarkRanges = std::deque; */ size_t getLastMark(const MarkRanges & ranges); +std::string toString(const MarkRanges & ranges); + } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index e2cd797ab92..475407a402b 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -43,6 +43,7 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( , storage(storage_) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) + , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings)) , max_block_size_rows(max_block_size_rows_) , preferred_block_size_bytes(preferred_block_size_bytes_) , preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_) @@ -72,7 +73,12 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( header_without_virtual_columns.erase(*it); } } +} + +std::unique_ptr MergeTreeBaseSelectProcessor::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings) +{ + std::unique_ptr prewhere_actions; if (prewhere_info) { prewhere_actions = std::make_unique(); @@ -100,6 +106,8 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( prewhere_actions->steps.emplace_back(std::move(prewhere_step)); } + + return prewhere_actions; } @@ -262,45 +270,62 @@ void MergeTreeBaseSelectProcessor::initializeMergeTreeReadersForPart( void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & current_task) { - MergeTreeRangeReader* prev_reader = nullptr; + return initializeRangeReadersImpl( + current_task.range_reader, current_task.pre_range_readers, prewhere_info, prewhere_actions.get(), + reader.get(), current_task.data_part->hasLightweightDelete(), reader_settings, + pre_reader_for_step, lightweight_delete_filter_step, non_const_virtual_column_names); +} + +void MergeTreeBaseSelectProcessor::initializeRangeReadersImpl( + MergeTreeRangeReader & range_reader, std::deque & pre_range_readers, + PrewhereInfoPtr prewhere_info, const PrewhereExprInfo * prewhere_actions, + IMergeTreeReader * reader, bool has_lightweight_delete, const MergeTreeReaderSettings & reader_settings, + const std::vector> & pre_reader_for_step, + const PrewhereExprStep & lightweight_delete_filter_step, const Names & non_const_virtual_column_names) +{ + MergeTreeRangeReader * prev_reader = nullptr; bool last_reader = false; size_t pre_readers_shift = 0; /// Add filtering step with lightweight delete mask - if (reader_settings.apply_deleted_mask && current_task.data_part->hasLightweightDelete()) + if (reader_settings.apply_deleted_mask && has_lightweight_delete) { - current_task.pre_range_readers.push_back( - MergeTreeRangeReader(pre_reader_for_step[0].get(), prev_reader, &lightweight_delete_filter_step, last_reader, non_const_virtual_column_names)); - prev_reader = ¤t_task.pre_range_readers.back(); + MergeTreeRangeReader pre_range_reader(pre_reader_for_step[0].get(), prev_reader, &lightweight_delete_filter_step, last_reader, non_const_virtual_column_names); + pre_range_readers.push_back(std::move(pre_range_reader)); + prev_reader = &pre_range_readers.back(); pre_readers_shift++; } if (prewhere_info) { if (prewhere_actions->steps.size() + pre_readers_shift != pre_reader_for_step.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "PREWHERE steps count mismatch, actions: {}, readers: {}", - prewhere_actions->steps.size(), pre_reader_for_step.size()); + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "PREWHERE steps count mismatch, actions: {}, readers: {}", + prewhere_actions->steps.size(), pre_reader_for_step.size()); + } for (size_t i = 0; i < prewhere_actions->steps.size(); ++i) { last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions->steps.size()); - current_task.pre_range_readers.push_back( - MergeTreeRangeReader(pre_reader_for_step[i + pre_readers_shift].get(), prev_reader, &prewhere_actions->steps[i], last_reader, non_const_virtual_column_names)); - prev_reader = ¤t_task.pre_range_readers.back(); + MergeTreeRangeReader current_reader(pre_reader_for_step[i + pre_readers_shift].get(), prev_reader, &prewhere_actions->steps[i], last_reader, non_const_virtual_column_names); + + pre_range_readers.push_back(std::move(current_reader)); + prev_reader = &pre_range_readers.back(); } } if (!last_reader) { - current_task.range_reader = MergeTreeRangeReader(reader.get(), prev_reader, nullptr, true, non_const_virtual_column_names); + range_reader = MergeTreeRangeReader(reader, prev_reader, nullptr, true, non_const_virtual_column_names); } else { /// If all columns are read by pre_range_readers than move last pre_range_reader into range_reader - current_task.range_reader = std::move(current_task.pre_range_readers.back()); - current_task.pre_range_readers.pop_back(); + range_reader = std::move(pre_range_readers.back()); + pre_range_readers.pop_back(); } } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index aa1b9d3541e..051854d8bc1 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -89,6 +89,20 @@ protected: static void injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); + static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings); + + static void initializeRangeReadersImpl( + MergeTreeRangeReader & range_reader, + std::deque & pre_range_readers, + PrewhereInfoPtr prewhere_info, + const PrewhereExprInfo * prewhere_actions, + IMergeTreeReader * reader, + bool has_lightweight_delete, + const MergeTreeReaderSettings & reader_settings, + const std::vector> & pre_reader_for_step, + const PrewhereExprStep & lightweight_delete_filter_step, + const Names & non_const_virtual_column_names); + /// Sets up data readers for each step of prewhere and where void initializeMergeTreeReadersForPart( MergeTreeData::DataPartPtr & data_part, diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 62fac84fc36..c3f069498be 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -28,8 +29,8 @@ namespace bool injectRequiredColumnsRecursively( const String & column_name, const StorageSnapshotPtr & storage_snapshot, - const MergeTreeData::AlterConversions & alter_conversions, - const MergeTreeData::DataPartPtr & part, + const AlterConversions & alter_conversions, + const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, const GetColumnsOptions & options, Names & columns, NameSet & required_columns, @@ -47,7 +48,7 @@ bool injectRequiredColumnsRecursively( if (alter_conversions.isColumnRenamed(column_name_in_part)) column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); - auto column_in_part = part->getColumns().tryGetByName(column_name_in_part); + auto column_in_part = data_part_info_for_reader.getColumns().tryGetByName(column_name_in_part); if (column_in_part && (!column_in_storage->isSubcolumn() @@ -78,7 +79,7 @@ bool injectRequiredColumnsRecursively( bool result = false; for (const auto & identifier : identifiers) result |= injectRequiredColumnsRecursively( - identifier, storage_snapshot, alter_conversions, part, + identifier, storage_snapshot, alter_conversions, data_part_info_for_reader, options, columns, required_columns, injected_columns); return result; @@ -87,9 +88,8 @@ bool injectRequiredColumnsRecursively( } NameSet injectRequiredColumns( - const MergeTreeData & storage, + const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, const StorageSnapshotPtr & storage_snapshot, - const MergeTreeData::DataPartPtr & part, bool with_subcolumns, Names & columns) { @@ -97,9 +97,9 @@ NameSet injectRequiredColumns( NameSet injected_columns; bool have_at_least_one_physical_column = false; - MergeTreeData::AlterConversions alter_conversions; - if (!part->isProjectionPart()) - alter_conversions = storage.getAlterConversionsForPart(part); + AlterConversions alter_conversions; + if (!data_part_info_for_reader.isProjectionPart()) + alter_conversions = data_part_info_for_reader.getAlterConversions(); auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) .withExtendedObjects() @@ -115,7 +115,7 @@ NameSet injectRequiredColumns( have_at_least_one_physical_column |= injectRequiredColumnsRecursively( columns[i], storage_snapshot, alter_conversions, - part, options, columns, required_columns, injected_columns); + data_part_info_for_reader, options, columns, required_columns, injected_columns); } /** Add a column of the minimum size. @@ -124,7 +124,7 @@ NameSet injectRequiredColumns( */ if (!have_at_least_one_physical_column) { - const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(with_subcolumns); + const auto minimum_size_column_name = data_part_info_for_reader.getColumnNameWithMinimumCompressedSize(with_subcolumns); columns.push_back(minimum_size_column_name); /// correctly report added column injected_columns.insert(columns.back()); @@ -135,13 +135,22 @@ NameSet injectRequiredColumns( MergeTreeReadTask::MergeTreeReadTask( - const MergeTreeData::DataPartPtr & data_part_, const MarkRanges & mark_ranges_, size_t part_index_in_query_, - const Names & ordered_names_, const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, + const MergeTreeData::DataPartPtr & data_part_, + const MarkRanges & mark_ranges_, + size_t part_index_in_query_, + const Names & ordered_names_, + const NameSet & column_name_set_, + const MergeTreeReadTaskColumns & task_columns_, bool remove_prewhere_column_, MergeTreeBlockSizePredictorPtr && size_predictor_) - : data_part{data_part_}, mark_ranges{mark_ranges_}, part_index_in_query{part_index_in_query_}, - ordered_names{ordered_names_}, column_name_set{column_name_set_}, task_columns{task_columns_}, - remove_prewhere_column{remove_prewhere_column_}, size_predictor{std::move(size_predictor_)} + : data_part{data_part_} + , mark_ranges{mark_ranges_} + , part_index_in_query{part_index_in_query_} + , ordered_names{ordered_names_} + , column_name_set{column_name_set_} + , task_columns{task_columns_} + , remove_prewhere_column{remove_prewhere_column_} + , size_predictor{std::move(size_predictor_)} { } @@ -270,9 +279,8 @@ void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Colum MergeTreeReadTaskColumns getReadTaskColumns( - const MergeTreeData & storage, + const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, const StorageSnapshotPtr & storage_snapshot, - const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, const Names & system_columns, const PrewhereInfoPtr & prewhere_info, @@ -284,13 +292,13 @@ MergeTreeReadTaskColumns getReadTaskColumns( /// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part for (const auto & name : system_columns) { - if (data_part->getColumns().contains(name)) + if (data_part_info_for_reader.getColumns().contains(name)) column_names.push_back(name); } /// inject columns required for defaults evaluation injectRequiredColumns( - storage, storage_snapshot, data_part, with_subcolumns, column_names); + data_part_info_for_reader, storage_snapshot, with_subcolumns, column_names); MergeTreeReadTaskColumns result; auto options = GetColumnsOptions(GetColumnsOptions::All) @@ -316,7 +324,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( Names all_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumnsNames(); const auto injected_pre_columns = injectRequiredColumns( - storage, storage_snapshot, data_part, with_subcolumns, all_pre_column_names); + data_part_info_for_reader, storage_snapshot, with_subcolumns, all_pre_column_names); for (const auto & name : all_pre_column_names) { diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 5a36955b4d3..e1c06869bb7 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -12,6 +12,7 @@ namespace DB class MergeTreeData; struct MergeTreeReadTask; struct MergeTreeBlockSizePredictor; +class IMergeTreeDataPartInfoForReader; using MergeTreeReadTaskPtr = std::unique_ptr; using MergeTreeBlockSizePredictorPtr = std::shared_ptr; @@ -23,9 +24,8 @@ using MergeTreeBlockSizePredictorPtr = std::shared_ptr #include #include +#include #include #include #include @@ -167,20 +168,6 @@ public: STRONG_TYPEDEF(String, PartitionID) - /// Alter conversions which should be applied on-fly for part. Build from of - /// the most recent mutation commands for part. Now we have only rename_map - /// here (from ALTER_RENAME) command, because for all other type of alters - /// we can deduce conversions for part from difference between - /// part->getColumns() and storage->getColumns(). - struct AlterConversions - { - /// Rename map new_name -> old_name - std::unordered_map rename_map; - - bool isColumnRenamed(const String & new_name) const { return rename_map.count(new_name) > 0; } - String getColumnOldName(const String & new_name) const { return rename_map.at(new_name); } - }; - struct LessDataPart { using is_transparent = void; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 046a7d274c0..14b3e33a157 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -45,9 +46,9 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { - auto ptr = std::static_pointer_cast(shared_from_this()); + auto read_info = std::make_shared(shared_from_this()); return std::make_unique( - ptr, columns_to_read, metadata_snapshot, uncompressed_cache, + read_info, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } @@ -90,39 +91,44 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac total_size.marks += mrk_checksum->second.file_size; } -void MergeTreeDataPartCompact::loadIndexGranularity() +void MergeTreeDataPartCompact::loadIndexGranularityImpl( + MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, + const NamesAndTypesList & columns_, const DataPartStoragePtr & data_part_storage_) { - //String full_path = getRelativePath(); - - if (columns.empty()) - throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - - if (!index_granularity_info.is_adaptive) + if (!index_granularity_info_.is_adaptive) throw Exception("MergeTreeDataPartCompact cannot be created with non-adaptive granulary.", ErrorCodes::NOT_IMPLEMENTED); - auto marks_file_path = index_granularity_info.getMarksFilePath("data"); - if (!data_part_storage->exists(marks_file_path)) + auto marks_file_path = index_granularity_info_.getMarksFilePath("data"); + if (!data_part_storage_->exists(marks_file_path)) throw Exception( ErrorCodes::NO_FILE_IN_DATA_PART, "Marks file '{}' doesn't exist", - std::string(fs::path(data_part_storage->getFullPath()) / marks_file_path)); + std::string(fs::path(data_part_storage_->getFullPath()) / marks_file_path)); - size_t marks_file_size = data_part_storage->getFileSize(marks_file_path); + size_t marks_file_size = data_part_storage_->getFileSize(marks_file_path); - auto buffer = data_part_storage->readFile(marks_file_path, ReadSettings().adjustBufferSize(marks_file_size), marks_file_size, std::nullopt); + auto buffer = data_part_storage_->readFile(marks_file_path, ReadSettings().adjustBufferSize(marks_file_size), marks_file_size, std::nullopt); while (!buffer->eof()) { /// Skip offsets for columns - buffer->seek(columns.size() * sizeof(MarkInCompressedFile), SEEK_CUR); + buffer->seek(columns_.size() * sizeof(MarkInCompressedFile), SEEK_CUR); size_t granularity; readIntBinary(granularity, *buffer); - index_granularity.appendMark(granularity); + index_granularity_.appendMark(granularity); } - if (index_granularity.getMarksCount() * index_granularity_info.getMarkSizeInBytes(columns.size()) != marks_file_size) + if (index_granularity_.getMarksCount() * index_granularity_info_.getMarkSizeInBytes(columns_.size()) != marks_file_size) throw Exception("Cannot read all marks from file " + marks_file_path, ErrorCodes::CANNOT_READ_ALL_DATA); - index_granularity.setInitialized(); + index_granularity_.setInitialized(); +} + +void MergeTreeDataPartCompact::loadIndexGranularity() +{ + if (columns.empty()) + throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + + loadIndexGranularityImpl(index_granularity, index_granularity_info, columns, data_part_storage); } bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index b1c0851afde..26c335f4324 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -65,6 +65,11 @@ public: ~MergeTreeDataPartCompact() override; +protected: + static void loadIndexGranularityImpl( + MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, + const NamesAndTypesList & columns_, const DataPartStoragePtr & data_part_storage_); + private: void checkConsistency(bool require_part_metadata) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 1c5006f4211..c7c831c23ec 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -48,9 +49,10 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( const ValueSizeMap & /* avg_value_size_hints */, const ReadBufferFromFileBase::ProfileCallback & /* profile_callback */) const { + auto read_info = std::make_shared(shared_from_this()); auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); + read_info, ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index c7b6ff0c4dd..58a0e48caab 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -47,9 +48,9 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { - auto ptr = std::static_pointer_cast(shared_from_this()); + auto read_info = std::make_shared(shared_from_this()); return std::make_unique( - ptr, columns_to_read, + read_info, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); @@ -103,46 +104,52 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( return size; } -void MergeTreeDataPartWide::loadIndexGranularity() +void MergeTreeDataPartWide::loadIndexGranularityImpl( + MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, + const DataPartStoragePtr & data_part_storage_, const std::string & any_column_file_name) { - index_granularity_info.changeGranularityIfRequired(data_part_storage); - - - if (columns.empty()) - throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + index_granularity_info_.changeGranularityIfRequired(data_part_storage_); /// We can use any column, it doesn't matter - std::string marks_file_path = index_granularity_info.getMarksFilePath(getFileNameForColumn(columns.front())); - if (!data_part_storage->exists(marks_file_path)) + std::string marks_file_path = index_granularity_info_.getMarksFilePath(any_column_file_name); + if (!data_part_storage_->exists(marks_file_path)) throw Exception( ErrorCodes::NO_FILE_IN_DATA_PART, "Marks file '{}' doesn't exist", - std::string(fs::path(data_part_storage->getFullPath()) / marks_file_path)); + std::string(fs::path(data_part_storage_->getFullPath()) / marks_file_path)); - size_t marks_file_size = data_part_storage->getFileSize(marks_file_path); + size_t marks_file_size = data_part_storage_->getFileSize(marks_file_path); - if (!index_granularity_info.is_adaptive) + if (!index_granularity_info_.is_adaptive) { - size_t marks_count = marks_file_size / index_granularity_info.getMarkSizeInBytes(); - index_granularity.resizeWithFixedGranularity(marks_count, index_granularity_info.fixed_index_granularity); /// all the same + size_t marks_count = marks_file_size / index_granularity_info_.getMarkSizeInBytes(); + index_granularity_.resizeWithFixedGranularity(marks_count, index_granularity_info_.fixed_index_granularity); /// all the same } else { - auto buffer = data_part_storage->readFile(marks_file_path, ReadSettings().adjustBufferSize(marks_file_size), marks_file_size, std::nullopt); + auto buffer = data_part_storage_->readFile(marks_file_path, ReadSettings().adjustBufferSize(marks_file_size), marks_file_size, std::nullopt); while (!buffer->eof()) { buffer->seek(sizeof(size_t) * 2, SEEK_CUR); /// skip offset_in_compressed file and offset_in_decompressed_block size_t granularity; readIntBinary(granularity, *buffer); - index_granularity.appendMark(granularity); + index_granularity_.appendMark(granularity); } - if (index_granularity.getMarksCount() * index_granularity_info.getMarkSizeInBytes() != marks_file_size) + if (index_granularity_.getMarksCount() * index_granularity_info_.getMarkSizeInBytes() != marks_file_size) throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all marks from file {}", - std::string(fs::path(data_part_storage->getFullPath()) / marks_file_path)); + std::string(fs::path(data_part_storage_->getFullPath()) / marks_file_path)); } - index_granularity.setInitialized(); + index_granularity_.setInitialized(); +} + +void MergeTreeDataPartWide::loadIndexGranularity() +{ + if (columns.empty()) + throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + + loadIndexGranularityImpl(index_granularity, index_granularity_info, data_part_storage, getFileNameForColumn(columns.front())); } bool MergeTreeDataPartWide::isStoredOnRemoteDisk() const diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 325193557b3..52afa9e82d4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -61,6 +61,11 @@ public: bool hasColumnFiles(const NameAndTypePair & column) const override; +protected: + static void loadIndexGranularityImpl( + MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, + const DataPartStoragePtr & data_part_storage_, const std::string & any_column_file_name); + private: void checkConsistency(bool require_part_metadata) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index dbb027c244e..a5adc919f4f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -29,6 +29,8 @@ public: MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MergeTreeDataPartType type_); + MergeTreeIndexGranularityInfo(MergeTreeDataPartType type_, bool is_adaptive_, size_t index_granularity_, size_t index_granularity_bytes_); + void changeGranularityIfRequired(const DataPartStoragePtr & data_part_storage); String getMarksFilePath(const String & path_prefix) const diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 79854785016..1a5a4d91806 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -83,7 +83,7 @@ MergeTreeRangeReader::DelayedStream::DelayedStream( : current_mark(from_mark), current_offset(0), num_delayed_rows(0) , current_task_last_mark(current_task_last_mark_) , merge_tree_reader(merge_tree_reader_) - , index_granularity(&(merge_tree_reader->data_part->index_granularity)) + , index_granularity(&(merge_tree_reader->data_part_info_for_read->getIndexGranularity())) , continue_reading(false), is_finished(false) { } @@ -181,7 +181,7 @@ MergeTreeRangeReader::Stream::Stream( : current_mark(from_mark), offset_after_current_mark(0) , last_mark(to_mark) , merge_tree_reader(merge_tree_reader_) - , index_granularity(&(merge_tree_reader->data_part->index_granularity)) + , index_granularity(&(merge_tree_reader->data_part_info_for_read->getIndexGranularity())) , current_mark_index_granularity(index_granularity->getMarkRows(from_mark)) , stream(from_mark, current_task_last_mark, merge_tree_reader) { @@ -652,7 +652,7 @@ MergeTreeRangeReader::MergeTreeRangeReader( bool last_reader_in_chain_, const Names & non_const_virtual_column_names_) : merge_tree_reader(merge_tree_reader_) - , index_granularity(&(merge_tree_reader->data_part->index_granularity)) + , index_granularity(&(merge_tree_reader->data_part_info_for_read->getIndexGranularity())) , prev_reader(prev_reader_) , prewhere_info(prewhere_info_) , last_reader_in_chain(last_reader_in_chain_) @@ -946,7 +946,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t result.addRows(stream.finalize(result.columns)); /// Last granule may be incomplete. - result.adjustLastGranule(); + if (!result.rowsPerGranule().empty()) + result.adjustLastGranule(); for (const auto & column_name : non_const_virtual_column_names) { diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index cc2c20eda5a..3f51673a6b1 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -22,7 +23,6 @@ MergeTreeReadPool::MergeTreeReadPool( size_t sum_marks_, size_t min_marks_for_concurrent_read_, RangesInDataParts && parts_, - const MergeTreeData & data_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const Names & column_names_, @@ -32,7 +32,6 @@ MergeTreeReadPool::MergeTreeReadPool( bool do_not_steal_tasks_) : backoff_settings{backoff_settings_} , backoff_state{threads_} - , data{data_} , storage_snapshot{storage_snapshot_} , column_names{column_names_} , virtual_column_names{virtual_column_names_} @@ -214,7 +213,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & per_part_sum_marks.push_back(sum_marks); auto task_columns = getReadTaskColumns( - data, storage_snapshot, part.data_part, + LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot, column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); auto size_predictor = !predict_block_size_bytes ? nullptr diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 01a1280b6cb..c9fe70d9a78 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -70,11 +70,16 @@ private: public: MergeTreeReadPool( - size_t threads_, size_t sum_marks_, size_t min_marks_for_concurrent_read_, - RangesInDataParts && parts_, const MergeTreeData & data_, const StorageSnapshotPtr & storage_snapshot_, + size_t threads_, + size_t sum_marks_, + size_t min_marks_for_concurrent_read_, + RangesInDataParts && parts_, + const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, - const Names & column_names_, const Names & virtual_column_names_, - const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, + const Names & column_names_, + const Names & virtual_column_names_, + const BackoffSettings & backoff_settings_, + size_t preferred_block_size_bytes_, bool do_not_steal_tasks_ = false); MergeTreeReadTaskPtr getTask(size_t min_marks_to_read, size_t thread, const Names & ordered_names); @@ -94,7 +99,6 @@ private: size_t threads, size_t sum_marks, std::vector per_part_sum_marks, const RangesInDataParts & parts, size_t min_marks_for_concurrent_read); - const MergeTreeData & data; StorageSnapshotPtr storage_snapshot; const Names column_names; const Names virtual_column_names; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 44b5fa1a11b..413e6838665 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes MergeTreeReaderCompact::MergeTreeReaderCompact( - DataPartCompactPtr data_part_, + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, @@ -26,7 +26,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) : IMergeTreeReader( - data_part_, + data_part_info_for_read_, columns_, metadata_snapshot_, uncompressed_cache_, @@ -35,14 +35,14 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( settings_, avg_value_size_hints_) , marks_loader( - data_part->data_part_storage, + data_part_info_for_read_->getDataPartStorage(), mark_cache, - data_part->index_granularity_info.getMarksFilePath(MergeTreeDataPartCompact::DATA_FILE_NAME), - data_part->getMarksCount(), - data_part->index_granularity_info, + data_part_info_for_read_->getIndexGranularityInfo().getMarksFilePath(MergeTreeDataPartCompact::DATA_FILE_NAME), + data_part_info_for_read_->getMarksCount(), + data_part_info_for_read_->getIndexGranularityInfo(), settings.save_marks_in_cache, settings.read_settings, - data_part->getColumns().size()) + data_part_info_for_read_->getColumns().size()) { try { @@ -64,7 +64,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( continue; } - auto position = data_part->getColumnPosition(column_to_read.getNameInStorage()); + auto position = data_part_info_for_read->getColumnPosition(column_to_read.getNameInStorage()); if (!position && typeid_cast(column_to_read.type.get())) { /// If array of Nested column is missing in part, @@ -77,7 +77,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } /// Do not use max_read_buffer_size, but try to lower buffer size with maximal size of granule to avoid reading much data. - auto buffer_size = getReadBufferSize(data_part, marks_loader, column_positions, all_mark_ranges); + auto buffer_size = getReadBufferSize(*data_part_info_for_read, marks_loader, column_positions, all_mark_ranges); if (buffer_size) settings.read_settings = settings.read_settings.adjustBufferSize(buffer_size); @@ -88,10 +88,10 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( if (uncompressed_cache) { auto buffer = std::make_unique( - std::string(fs::path(data_part->data_part_storage->getFullPath()) / path), + std::string(fs::path(data_part_info_for_read->getDataPartStorage()->getFullPath()) / path), [this, path]() { - return data_part->data_part_storage->readFile( + return data_part_info_for_read->getDataPartStorage()->readFile( path, settings.read_settings, std::nullopt, std::nullopt); @@ -113,7 +113,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { auto buffer = std::make_unique( - data_part->data_part_storage->readFile( + data_part_info_for_read->getDataPartStorage()->readFile( path, settings.read_settings, std::nullopt, std::nullopt), @@ -132,7 +132,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } catch (...) { - storage.reportBrokenPart(data_part); + data_part_info_for_read->reportBroken(); throw; } } @@ -156,7 +156,7 @@ size_t MergeTreeReaderCompact::readRows( while (read_rows < max_rows_to_read) { - size_t rows_to_read = data_part->index_granularity.getMarkRows(from_mark); + size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark); for (size_t pos = 0; pos < num_columns; ++pos) { @@ -179,7 +179,7 @@ size_t MergeTreeReaderCompact::readRows( catch (Exception & e) { if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) - storage.reportBrokenPart(data_part); + data_part_info_for_read->reportBroken(); /// Better diagnostics. e.addMessage("(while reading column " + columns_to_read[pos].name + ")"); @@ -187,7 +187,7 @@ size_t MergeTreeReaderCompact::readRows( } catch (...) { - storage.reportBrokenPart(data_part); + data_part_info_for_read->reportBroken(); throw; } } @@ -279,7 +279,7 @@ void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index) void MergeTreeReaderCompact::adjustUpperBound(size_t last_mark) { size_t right_offset = 0; - if (last_mark < data_part->getMarksCount()) /// Otherwise read until the end of file + if (last_mark < data_part_info_for_read->getMarksCount()) /// Otherwise read until the end of file right_offset = marks_loader.getMark(last_mark).offset_in_compressed_file; if (right_offset == 0) @@ -307,7 +307,7 @@ bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_posi return false; const auto & [last_mark, last_column] = *last_read_granule; return (mark == last_mark && column_position == last_column + 1) - || (mark == last_mark + 1 && column_position == 0 && last_column == data_part->getColumns().size() - 1); + || (mark == last_mark + 1 && column_position == 0 && last_column == data_part_info_for_read->getColumns().size() - 1); } namespace @@ -359,16 +359,16 @@ private: } size_t MergeTreeReaderCompact::getReadBufferSize( - const DataPartPtr & part, + const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, MergeTreeMarksLoader & marks_loader, const ColumnPositions & column_positions, const MarkRanges & mark_ranges) { size_t buffer_size = 0; size_t columns_num = column_positions.size(); - size_t file_size = part->getFileSizeOrZero(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION); + size_t file_size = data_part_info_for_reader.getFileSizeOrZero(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION); - MarksCounter counter(part->getMarksCount(), part->getColumns().size()); + MarksCounter counter(data_part_info_for_reader.getMarksCount(), data_part_info_for_reader.getColumns().size()); for (const auto & mark_range : mark_ranges) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index aa0eb949aa1..9a426fda71d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -19,7 +19,7 @@ class MergeTreeReaderCompact : public IMergeTreeReader { public: MergeTreeReaderCompact( - DataPartCompactPtr data_part_, + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, @@ -67,7 +67,7 @@ private: /// Returns maximal value of granule size in compressed file from @mark_ranges. /// This value is used as size of read buffer. static size_t getReadBufferSize( - const DataPartPtr & part, + const IMergeTreeDataPartInfoForReader & data_part_info_for_read_, MergeTreeMarksLoader & marks_loader, const ColumnPositions & column_positions, const MarkRanges & mark_ranges); diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 766c28c99b9..c392199fa9e 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -16,13 +16,14 @@ namespace ErrorCodes MergeTreeReaderInMemory::MergeTreeReaderInMemory( + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, const StorageMetadataPtr & metadata_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_) : IMergeTreeReader( - data_part_, + data_part_info_for_read_, columns_, metadata_snapshot_, nullptr, @@ -48,7 +49,7 @@ size_t MergeTreeReaderInMemory::readRows( if (!continue_reading) total_rows_read = 0; - size_t total_marks = data_part->index_granularity.getMarksCount(); + size_t total_marks = data_part_info_for_read->getIndexGranularity().getMarksCount(); if (from_mark >= total_marks) throw Exception("Mark " + toString(from_mark) + " is out of bound. Max mark: " + toString(total_marks), ErrorCodes::ARGUMENT_OUT_OF_BOUND); diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index ff6eb92d9c3..cb67bc46eae 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -15,6 +15,7 @@ class MergeTreeReaderInMemory : public IMergeTreeReader { public: MergeTreeReaderInMemory( + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, const StorageMetadataPtr & metadata_snapshot_, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 1274017b865..5a048e8bc1a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes } MergeTreeReaderWide::MergeTreeReaderWide( - DataPartWidePtr data_part_, + MergeTreeDataPartInfoForReaderPtr data_part_info_, NamesAndTypesList columns_, const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, @@ -37,7 +37,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) : IMergeTreeReader( - data_part_, + data_part_info_, columns_, metadata_snapshot_, uncompressed_cache_, @@ -53,7 +53,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( } catch (...) { - storage.reportBrokenPart(data_part); + data_part_info_for_read->reportBroken(); throw; } } @@ -73,7 +73,7 @@ size_t MergeTreeReaderWide::readRows( std::unordered_map caches; std::unordered_set prefetched_streams; - if (data_part->data_part_storage->isStoredOnRemoteDisk() ? settings.read_settings.remote_fs_prefetch : settings.read_settings.local_fs_prefetch) + if (data_part_info_for_read->getDataPartStorage()->isStoredOnRemoteDisk() ? settings.read_settings.remote_fs_prefetch : settings.read_settings.local_fs_prefetch) { /// Request reading of data in advance, /// so if reading can be asynchronous, it will also be performed in parallel for all columns. @@ -136,17 +136,17 @@ size_t MergeTreeReaderWide::readRows( catch (Exception & e) { if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) - storage.reportBrokenPart(data_part); + data_part_info_for_read->reportBroken(); /// Better diagnostics. - e.addMessage("(while reading from part " + data_part->data_part_storage->getFullPath() + " " + e.addMessage("(while reading from part " + data_part_info_for_read->getDataPartStorage()->getFullPath() + " " "from mark " + toString(from_mark) + " " "with max_rows_to_read = " + toString(max_rows_to_read) + ")"); throw; } catch (...) { - storage.reportBrokenPart(data_part); + data_part_info_for_read->reportBroken(); throw; } @@ -167,7 +167,7 @@ void MergeTreeReaderWide::addStreams( if (streams.contains(stream_name)) return; - bool data_file_exists = data_part->checksums.files.contains(stream_name + DATA_FILE_EXTENSION); + bool data_file_exists = data_part_info_for_read->getChecksums().files.contains(stream_name + DATA_FILE_EXTENSION); /** If data file is missing then we will not try to open it. * It is necessary since it allows to add new column to structure of the table without creating new files for old parts. @@ -178,10 +178,10 @@ void MergeTreeReaderWide::addStreams( bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; streams.emplace(stream_name, std::make_unique( - data_part->data_part_storage, stream_name, DATA_FILE_EXTENSION, - data_part->getMarksCount(), all_mark_ranges, settings, mark_cache, - uncompressed_cache, data_part->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), - &data_part->index_granularity_info, + data_part_info_for_read->getDataPartStorage(), stream_name, DATA_FILE_EXTENSION, + data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache, + uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), + &data_part_info_for_read->getIndexGranularityInfo(), profile_callback, clock_type, is_lc_dict)); }; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 2137695b6d7..dbfc0310242 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -15,7 +15,7 @@ class MergeTreeReaderWide : public IMergeTreeReader { public: MergeTreeReaderWide( - DataPartWidePtr data_part_, + MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 3e346df6662..59cbae3f914 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -51,7 +52,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( void MergeTreeSelectProcessor::initializeReaders() { task_columns = getReadTaskColumns( - storage, storage_snapshot, data_part, + LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, required_columns, virt_column_names, prewhere_info, /*with_subcolumns=*/ true); /// Will be used to distinguish between PREWHERE and WHERE columns when applying filter diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 5b9eceece51..9e0c96fd88a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -102,7 +103,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( addTotalRowsApprox(data_part->rows_count); /// Add columns because we don't want to read empty blocks - injectRequiredColumns(storage, storage_snapshot, data_part, /*with_subcolumns=*/ false, columns_to_read); + injectRequiredColumns(LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, /*with_subcolumns=*/ false, columns_to_read); NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index fe7aaf97970..de7e1911e44 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -1,6 +1,7 @@ #include "StorageSystemRemoteDataPaths.h" #include #include +#include #include #include #include @@ -23,6 +24,8 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab {"cache_base_path", std::make_shared()}, {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, + {"size", std::make_shared()}, + {"common_prefix_for_blobs", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, })); setInMemoryMetadata(storage_metadata); @@ -44,6 +47,8 @@ Pipe StorageSystemRemoteDataPaths::read( MutableColumnPtr col_cache_base_path = ColumnString::create(); MutableColumnPtr col_local_path = ColumnString::create(); MutableColumnPtr col_remote_path = ColumnString::create(); + MutableColumnPtr col_size = ColumnUInt64::create(); + MutableColumnPtr col_namespace = ColumnString::create(); MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); auto disks = context->getDisksMap(); @@ -61,7 +66,7 @@ Pipe StorageSystemRemoteDataPaths::read( if (!cache_base_path.empty()) cache = FileCacheFactory::instance().get(cache_base_path); - for (const auto & [local_path, storage_objects] : remote_paths_by_local_path) + for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path) { for (const auto & object : storage_objects) { @@ -70,6 +75,8 @@ Pipe StorageSystemRemoteDataPaths::read( col_cache_base_path->insert(cache_base_path); col_local_path->insert(local_path); col_remote_path->insert(object.absolute_path); + col_size->insert(object.bytes_size); + col_namespace->insert(common_prefox_for_objects); if (cache) { @@ -91,6 +98,8 @@ Pipe StorageSystemRemoteDataPaths::read( res_columns.emplace_back(std::move(col_cache_base_path)); res_columns.emplace_back(std::move(col_local_path)); res_columns.emplace_back(std::move(col_remote_path)); + res_columns.emplace_back(std::move(col_size)); + res_columns.emplace_back(std::move(col_namespace)); res_columns.emplace_back(std::move(col_cache_paths)); UInt64 num_rows = res_columns.at(0)->size(); From 4c8aa04b73d9f1b4da8b1c2ccebe1865d9fd83af Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Sep 2022 20:12:40 +0200 Subject: [PATCH 381/582] Revert "Fix" This reverts commit ec55c90f36ae142cfa1ad149d55ca4562e88cf47. --- src/Interpreters/Context.cpp | 13 +++++++++++++ src/Interpreters/Context.h | 1 + .../MergeTree/MergeTreeDataPartCompact.cpp | 5 ++++- src/Storages/MergeTree/MergeTreeIndexReader.cpp | 5 ++++- src/Storages/MergeTree/MergeTreeMarksLoader.cpp | 16 ++++------------ src/Storages/MergeTree/MergeTreeMarksLoader.h | 4 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 2 ++ src/Storages/MergeTree/MergeTreeReaderCompact.h | 1 + src/Storages/MergeTree/MergeTreeReaderStream.cpp | 6 ++++-- src/Storages/MergeTree/MergeTreeReaderStream.h | 3 ++- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 5 ++++- 11 files changed, 41 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d6ee5aca1a8..d39c39cdb15 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -215,6 +215,7 @@ struct ContextSharedPart std::unique_ptr access_control; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. + mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. @@ -1711,6 +1712,18 @@ void Context::dropMarkCache() const shared->mark_cache->reset(); } +ThreadPool & Context::getLoadMarksThreadpool() const +{ + auto lock = getLock(); + if (!shared->load_marks_threadpool) + { + constexpr size_t pool_size = 50; + constexpr size_t queue_size = 1000000; + shared->load_marks_threadpool = std::make_unique(pool_size, pool_size, queue_size); + } + return *shared->load_marks_threadpool; +} + void Context::setIndexUncompressedCache(size_t max_size_in_bytes) { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ea03b8e6586..b8a0d9ab600 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -788,6 +788,7 @@ public: void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy); std::shared_ptr getMarkCache() const; void dropMarkCache() const; + ThreadPool & getLoadMarksThreadpool() const; /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 0002a4bb599..851153cd619 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -47,9 +47,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { auto ptr = std::static_pointer_cast(shared_from_this()); + auto context = storage.getContext(); + auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + return std::make_unique( ptr, columns_to_read, metadata_snapshot, uncompressed_cache, - mark_cache, mark_ranges, reader_settings, + mark_cache, mark_ranges, reader_settings, load_marks_threadpool, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 2f9ea7c4ede..33106f7ab64 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -16,6 +16,9 @@ std::unique_ptr makeIndexReader( UncompressedCache * uncompressed_cache, MergeTreeReaderSettings settings) { + auto context = part->storage.getContext(); + auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + return std::make_unique( part->data_part_storage, index->getFileName(), extension, marks_count, @@ -23,7 +26,7 @@ std::unique_ptr makeIndexReader( std::move(settings), mark_cache, uncompressed_cache, part->getFileSizeOrZero(index->getFileName() + extension), &part->index_granularity_info, - ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false); + ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false, load_marks_threadpool); } } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 27100f8f474..ad5d828c431 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -33,6 +33,7 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( const MergeTreeIndexGranularityInfo & index_granularity_info_, bool save_marks_in_cache_, const ReadSettings & read_settings_, + ThreadPool * load_marks_threadpool_, size_t columns_in_mark_) : data_part_storage(std::move(data_part_storage_)) , mark_cache(mark_cache_) @@ -42,8 +43,9 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( , save_marks_in_cache(save_marks_in_cache_) , columns_in_mark(columns_in_mark_) , read_settings(read_settings_) + , load_marks_threadpool(load_marks_threadpool_) { - if (read_settings_.load_marks_asynchronously) + if (load_marks_threadpool) { future = loadMarksAsync(); } @@ -57,16 +59,6 @@ MergeTreeMarksLoader::~MergeTreeMarksLoader() } } -std::shared_ptr MergeTreeMarksLoader::getLoadMarksThreadpool() -{ - constexpr size_t pool_size = 50; - constexpr size_t queue_size = 1000000; - static std::shared_ptr load_marks_threadpool = std::make_shared(pool_size, pool_size, queue_size); - - return load_marks_threadpool; -} - - const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index) { if (!marks) @@ -197,7 +189,7 @@ std::future MergeTreeMarksLoader::loadMarksAsync() }); auto task_future = task->get_future(); - getLoadMarksThreadpool()->scheduleOrThrow([task]{ (*task)(); }); + load_marks_threadpool->scheduleOrThrow([task]{ (*task)(); }); return task_future; } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 2474710f098..60ccc953e9b 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -23,6 +23,7 @@ public: const MergeTreeIndexGranularityInfo & index_granularity_info_, bool save_marks_in_cache_, const ReadSettings & read_settings_, + ThreadPool * load_marks_threadpool_, size_t columns_in_mark_ = 1); ~MergeTreeMarksLoader(); @@ -30,8 +31,6 @@ public: const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0); private: - static std::shared_ptr getLoadMarksThreadpool(); - DataPartStoragePtr data_part_storage; MarkCache * mark_cache = nullptr; String mrk_path; @@ -47,6 +46,7 @@ private: MarkCache::MappedPtr loadMarksImpl(); std::future future; + ThreadPool * load_marks_threadpool; }; } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 44b5fa1a11b..88237091547 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -22,6 +22,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( MarkCache * mark_cache_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_, + ThreadPool * load_marks_cache_threadpool_, ValueSizeMap avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) @@ -42,6 +43,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( data_part->index_granularity_info, settings.save_marks_in_cache, settings.read_settings, + load_marks_cache_threadpool_, data_part->getColumns().size()) { try diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index aa0eb949aa1..348e4802e2a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -26,6 +26,7 @@ public: MarkCache * mark_cache_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_, + ThreadPool * load_marks_cache_threadpool_, ValueSizeMap avg_value_size_hints_ = {}, const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {}, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE); diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index 7ee939c3f06..47f8b0f6008 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -27,7 +27,8 @@ MergeTreeReaderStream::MergeTreeReaderStream( const MergeTreeIndexGranularityInfo * index_granularity_info_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_, - bool is_low_cardinality_dictionary_) + bool is_low_cardinality_dictionary_, + ThreadPool * load_marks_cache_threadpool_) : settings(settings_) , profile_callback(profile_callback_) , clock_type(clock_type_) @@ -49,7 +50,8 @@ MergeTreeReaderStream::MergeTreeReaderStream( marks_count, *index_granularity_info, save_marks_in_cache, - settings.read_settings) + settings.read_settings, + load_marks_cache_threadpool_) { } diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.h b/src/Storages/MergeTree/MergeTreeReaderStream.h index f480e355294..83e314eef42 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.h +++ b/src/Storages/MergeTree/MergeTreeReaderStream.h @@ -31,7 +31,8 @@ public: const MergeTreeIndexGranularityInfo * index_granularity_info_, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type, - bool is_low_cardinality_dictionary_); + bool is_low_cardinality_dictionary_, + ThreadPool * load_marks_cache_threadpool_); void seekToMark(size_t index); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 4f21708fc32..9ac7b5a5c5a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -178,12 +178,15 @@ void MergeTreeReaderWide::addStreams( bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; + auto context = data_part->storage.getContext(); + auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + streams.emplace(stream_name, std::make_unique( data_part->data_part_storage, stream_name, DATA_FILE_EXTENSION, data_part->getMarksCount(), all_mark_ranges, settings, mark_cache, uncompressed_cache, data_part->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), &data_part->index_granularity_info, - profile_callback, clock_type, is_lc_dict)); + profile_callback, clock_type, is_lc_dict, load_marks_threadpool)); }; serialization->enumerateStreams(callback); From 68e013e8199321aa98b4ed1f18cbed279e9b0643 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Sep 2022 22:13:07 +0200 Subject: [PATCH 382/582] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 10b0c606b93..69ca2e763b2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -250,7 +250,7 @@ CI_CONFIG = { "Stateless tests (debug, s3 storage)": { "required_build": "package_debug", }, - "Stateless tests (thread, s3 storage)": { + "Stateless tests (tsan, s3 storage)": { "required_build": "package_tsan", }, "Stress test (asan)": { From 0296d44a7c2018670d7f2f9ea3fda22b6b4cbd8e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Sep 2022 22:15:05 +0200 Subject: [PATCH 383/582] Update pull_request.yml --- .github/workflows/pull_request.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7693abbaf8d..6dd28215d78 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1374,7 +1374,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, s3 storage) + CHECK_NAME=Stateless tests (tsan, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1411,7 +1411,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, s3 storage) + CHECK_NAME=Stateless tests (tsan, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1448,7 +1448,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, s3 storage) + CHECK_NAME=Stateless tests (tsan, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 From 7a8b8e7a39a663042a0a00c86694ca59ea76fc59 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 5 Sep 2022 20:47:11 +0000 Subject: [PATCH 384/582] Optimizer setting: read in window order optimization's setting is checked before applying it, not inside the optimization code --- src/Processors/QueryPlan/Optimizations/Optimizations.h | 2 +- .../Optimizations/QueryPlanOptimizationSettings.cpp | 1 + .../Optimizations/QueryPlanOptimizationSettings.h | 3 +++ .../reuseStorageOrderingForWindowFunctions.cpp | 7 +------ 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 904f30e84b0..f45200f3026 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -63,7 +63,7 @@ inline const auto & getOptimizations() {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::optimize_plan}, {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan}, - {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan} + {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_read_in_window_order} }}; return optimizations; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 1472fb87a89..f9707b973e4 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -11,6 +11,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.optimize_plan = from.query_plan_enable_optimizations; settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply; settings.filter_push_down = from.query_plan_filter_push_down; + settings.optimize_read_in_window_order = from.optimize_read_in_window_order; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index b5a37bf69d6..99e52b60a73 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -21,6 +21,9 @@ struct QueryPlanOptimizationSettings /// If filter push down optimization is enabled. bool filter_push_down = true; + /// window functions read in order optimization + bool optimize_read_in_window_order = true; + static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 401774b390e..8377b62c947 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -61,12 +61,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, return 0; } - auto context = read_from_merge_tree->getContext(); - if (!context->getSettings().optimize_read_in_window_order) - { - return 0; - } - + const auto context = read_from_merge_tree->getContext(); const auto & query_info = read_from_merge_tree->getQueryInfo(); const auto * select_query = query_info.query->as(); From cbcf267f5526ab7b72383071fcd86f646e0b4e6e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Sep 2022 22:54:38 +0200 Subject: [PATCH 385/582] One more line of logging in write buffer from S3 --- src/IO/WriteBufferFromS3.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 7646e2514a5..2fb6502b0c4 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -174,6 +174,8 @@ void WriteBufferFromS3::finalizeImpl() if (!response.IsSuccess()) throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", key, bucket); + else + LOG_TRACE(log, "Object {} exists after upload", key); } } From 8977af51505719a56ad65a9f9bc4bbfda7bfb16a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Sep 2022 22:59:25 +0200 Subject: [PATCH 386/582] Remove some outdated assertions --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a52de88321c..d89e3637a6f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -532,25 +532,6 @@ void IMergeTreeDataPart::removeIfNeeded() LOG_TRACE(storage.log, "Removed part from old location {}", path); } } - catch (const Exception & ex) - { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while removing part {} with path {}", name, path)); - - /// In this case we want to avoid assertions, because such errors are unavoidable in setup - /// with zero-copy replication. - if (const auto * keeper_exception = dynamic_cast(&ex)) - { - if (Coordination::isHardwareError(keeper_exception->code)) - return; - } - - /// FIXME If part it temporary, then directory will not be removed for 1 day (temporary_directories_lifetime). - /// If it's tmp_merge_ or tmp_fetch_, - /// then all future attempts to execute part producing operation will fail with "directory already exists". - assert(!is_temp); - assert(state != MergeTreeDataPartState::DeleteOnDestroy); - assert(state != MergeTreeDataPartState::Temporary); - } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while removing part {} with path {}", name, path)); @@ -558,11 +539,6 @@ void IMergeTreeDataPart::removeIfNeeded() /// FIXME If part it temporary, then directory will not be removed for 1 day (temporary_directories_lifetime). /// If it's tmp_merge_ or tmp_fetch_, /// then all future attempts to execute part producing operation will fail with "directory already exists". - /// - /// For remote disks this issue is really frequent, so we don't about server here - assert(!is_temp); - assert(state != MergeTreeDataPartState::DeleteOnDestroy); - assert(state != MergeTreeDataPartState::Temporary); } } From 33f541042a4e07236983a7c68d012b704f8d78ab Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 6 Sep 2022 00:04:50 +0200 Subject: [PATCH 387/582] Fix clang tidy --- src/Storages/MergeTree/MergeTreeReaderCompact.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 9a426fda71d..afc360adc51 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -67,7 +67,7 @@ private: /// Returns maximal value of granule size in compressed file from @mark_ranges. /// This value is used as size of read buffer. static size_t getReadBufferSize( - const IMergeTreeDataPartInfoForReader & data_part_info_for_read_, + const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, MergeTreeMarksLoader & marks_loader, const ColumnPositions & column_positions, const MarkRanges & mark_ranges); From 494dfff73c98ffd6c61d330b818f66f320ad368a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Sep 2022 02:04:18 +0200 Subject: [PATCH 388/582] Remove trash --- programs/server/users.xml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/programs/server/users.xml b/programs/server/users.xml index 96067d01a6b..67858c0f3eb 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -6,15 +6,6 @@ - - random From b7bdd38e410826953353e6cfa1687d91215a9bd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Sep 2022 02:04:36 +0200 Subject: [PATCH 389/582] Revert "Remove trash" This reverts commit 494dfff73c98ffd6c61d330b818f66f320ad368a. --- programs/server/users.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/programs/server/users.xml b/programs/server/users.xml index 67858c0f3eb..96067d01a6b 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -6,6 +6,15 @@ + + random From 51dc796275a69e1dc6dc55eed8bfcd7fbeb0b9ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Sep 2022 02:04:18 +0200 Subject: [PATCH 390/582] Remove trash --- programs/server/users.xml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/programs/server/users.xml b/programs/server/users.xml index 96067d01a6b..67858c0f3eb 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -6,15 +6,6 @@ - - random From 6950016b8a83c990e7332decc8509fdf85132fbf Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 5 Sep 2022 19:06:16 +0800 Subject: [PATCH 391/582] fix grouping set with group_by_use_nulls --- src/Processors/QueryPlan/AggregatingStep.cpp | 9 ++++--- .../02343_group_by_use_nulls.reference | 24 +++++++++++++++++++ .../0_stateless/02343_group_by_use_nulls.sql | 13 ++++++++++ 3 files changed, 43 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 5a5326091e6..03f346d8f72 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -251,14 +251,17 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B outputs.push_back(grouping_node); const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; + const auto & used_keys = grouping_sets_params[set_counter].used_keys; auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr); for (size_t i = 0; i < output_header.columns(); ++i) { auto & col = output_header.getByPosition(i); - const auto it = std::find_if( + const auto missing_it = std::find_if( missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; }); - if (it != missing_columns.end()) + const auto used_it = std::find_if( + used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; }); + if (missing_it != missing_columns.end()) { auto column_with_default = col.column->cloneEmpty(); col.type->insertDefaultInto(*column_with_default); @@ -270,7 +273,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B else { const auto * column_node = dag->getOutputs()[header.getPositionByName(col.name)]; - if (group_by_use_nulls && column_node->result_type->canBeInsideNullable()) + if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) outputs.push_back(&dag->addFunction(to_nullable_function, { column_node }, col.name)); else outputs.push_back(column_node); diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference index 24b7bb5277c..c694b7abfa1 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -213,3 +213,27 @@ SETTINGS group_by_use_nulls=1; \N \N 45 0 0 45 +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY 1, tuple(val) +SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +\N 0 20 +\N 1 25 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index a14db824013..f875edfb4a6 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -60,3 +60,16 @@ FROM numbers(10) GROUP BY CUBE(number, number % 2) WITH TOTALS ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls=1; + +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY 1, tuple(val) +SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; \ No newline at end of file From ad52cfdf3ac0960e866e680b261cf39746510bf1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 5 Sep 2022 19:09:07 +0800 Subject: [PATCH 392/582] fix test style --- tests/queries/0_stateless/02343_group_by_use_nulls.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index f875edfb4a6..a979a78be0d 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -72,4 +72,4 @@ GROUP BY (number % 2) ) ORDER BY 1, tuple(val) -SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; \ No newline at end of file +SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; From b7cf8783eb18b372126843a3c0385f8d66d51966 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Sep 2022 00:14:55 -0300 Subject: [PATCH 393/582] test for column name primary --- .../00488_column_name_primary.reference | 1 + .../0_stateless/00488_column_name_primary.sql | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/00488_column_name_primary.reference create mode 100644 tests/queries/0_stateless/00488_column_name_primary.sql diff --git a/tests/queries/0_stateless/00488_column_name_primary.reference b/tests/queries/0_stateless/00488_column_name_primary.reference new file mode 100644 index 00000000000..a6905f8ba44 --- /dev/null +++ b/tests/queries/0_stateless/00488_column_name_primary.reference @@ -0,0 +1 @@ +999 diff --git a/tests/queries/0_stateless/00488_column_name_primary.sql b/tests/queries/0_stateless/00488_column_name_primary.sql new file mode 100644 index 00000000000..124d0e14239 --- /dev/null +++ b/tests/queries/0_stateless/00488_column_name_primary.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS primary; + +CREATE TABLE primary +( + `primary` String +) +ENGINE = MergeTree +ORDER BY primary +settings min_bytes_for_wide_part=0,min_bytes_for_wide_part=0 + AS +SELECT * +FROM numbers(1000); + +select max(primary) from primary; + +DROP TABLE primary; From 8365e7bfac1f210cf664b833507d3771f1981640 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 6 Sep 2022 15:41:21 +0800 Subject: [PATCH 394/582] Remove extra attribute --- src/Storages/Distributed/DistributedSink.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index dc33cfa4b60..57397c6908e 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -171,7 +171,6 @@ void DistributedSink::writeAsync(const Block & block) } else { - if (storage.getShardingKeyExpr() && (cluster->getShardsInfo().size() > 1)) return writeSplitAsync(block); @@ -652,7 +651,6 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) void DistributedSink::writeToLocal(const Cluster::ShardInfo& shard_info, const Block & block, size_t repeats) { OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); - span.addAttribute("db.statement", this->query_string); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); span.addAttribute("clickhouse.cluster", this->storage.cluster_name); span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); From 1ebcc3a14ee7bd60ea5819648fb5c44360e1e07c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Sep 2022 07:41:37 +0000 Subject: [PATCH 395/582] fix: endswidth --> endswith --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 363be9ab2dd..de41b23acb0 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -142,7 +142,7 @@ def parse_env_variables( is_cross_arm = compiler.endswith(ARM_SUFFIX) is_cross_ppc = compiler.endswith(PPC_SUFFIX) is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX) - is_amd64_sse2 = compiler.endswidth(AMD64_SSE2_SUFFIX) + is_amd64_sse2 = compiler.endswith(AMD64_SSE2_SUFFIX) if is_cross_darwin: cc = compiler[: -len(DARWIN_SUFFIX)] From 49556dad975052c748bb8db6ae414da58bf2094d Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 6 Sep 2022 15:42:45 +0800 Subject: [PATCH 396/582] Add test cases --- ...etry_insert_on_distributed_table.reference | 4 + ...entelemetry_insert_on_distributed_table.sh | 84 +++++++++++++++++++ 2 files changed, 88 insertions(+) create mode 100644 tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference create mode 100755 tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference new file mode 100644 index 00000000000..fac9fabce8a --- /dev/null +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference @@ -0,0 +1,4 @@ +{'clickhouse.shard_num':'1','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} +{'clickhouse.shard_num':'2','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} +{'clickhouse.shard_num':'1','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} +{'clickhouse.shard_num':'2','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh new file mode 100755 index 00000000000..6f766e9f3bb --- /dev/null +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -0,0 +1,84 @@ +#!/usr/bin/env bash +# Tags: distributed + +set -ue + +unset CLICKHOUSE_LOG_COMMENT + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} -nq " +SET distributed_ddl_output_mode = 'none'; + +SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; +TRUNCATE TABLE IF EXISTS system.opentelemetry_span_log ON CLUSTER test_cluster_two_shards; + +DROP TABLE IF EXISTS default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; +DROP TABLE IF EXISTS default.local_opentelemetry ON CLUSTER test_cluster_two_shards; + +CREATE TABLE default.dist_opentelemetry ON CLUSTER test_cluster_two_shards (key UInt64) Engine=Distributed('test_cluster_two_shards', default, local_opentelemetry, key % 2); +CREATE TABLE default.local_opentelemetry ON CLUSTER test_cluster_two_shards (key UInt64) Engine=MergeTree ORDER BY key; +" + +# +# INSERT ASYNC test +# Do test with opentelemetry enabled +# +${CLICKHOUSE_CLIENT} -nq " +-- Make sure it's async +SET insert_distributed_sync=0; +INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1 VALUES(1),(2); +" + +# Wait complete of ASYNC INSERT on distributed table +wait + +# Check log +${CLICKHOUSE_CLIENT} -nq " +-- Flush opentelemetry span log on all nodes +SET distributed_ddl_output_mode = 'none'; +SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; + +-- Above INSERT will insert data to two shards respectively, so there will be two spans generated +SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%writeToLocal%'; +SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%processFile%'; +" + +# +# INSERT SYNC test +# Do test with opentelemetry enabled and in SYNC mode +# +${CLICKHOUSE_CLIENT} -nq " + +-- Clear log +SET distributed_ddl_output_mode = 'none'; +TRUNCATE TABLE IF EXISTS system.opentelemetry_span_log ON CLUSTER test_cluster_two_shards; + +-- Make sure it's SYNC +SET insert_distributed_sync=1; + +-- INSERT test +INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1 VALUES(1),(2); +" + +# Check log +${CLICKHOUSE_CLIENT} -nq " +-- Flush opentelemetry span log on all nodes +SET distributed_ddl_output_mode = 'none'; +SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; + +-- Above INSERT will insert data to two shards in the same flow, so there should be two spans generated with the same operation name +SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%runWritingJob%'; +" + +# +# Cleanup +# +${CLICKHOUSE_CLIENT} -nq " +SET distributed_ddl_output_mode = 'none'; +DROP TABLE default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; +DROP TABLE default.local_opentelemetry ON CLUSTER test_cluster_two_shards; +" From 206709603502b8f0f8f99996d332458976f15c7f Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 6 Sep 2022 16:01:31 +0800 Subject: [PATCH 397/582] Optimize span log for SYNC insert --- src/Storages/Distributed/DistributedSink.cpp | 22 +++++++++++--------- src/Storages/Distributed/DistributedSink.h | 4 ++-- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 57397c6908e..8099a7f2002 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -290,6 +290,8 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si auto thread_group = CurrentThread::getGroup(); return [this, thread_group, &job, ¤t_block, num_shards]() { + OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); + if (thread_group) CurrentThread::attachToIfDetached(thread_group); setThreadName("DistrOutStrProc"); @@ -330,18 +332,18 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si const Block & shard_block = (num_shards > 1) ? job.current_shard_block : current_block; const Settings & settings = context->getSettingsRef(); - /// Do not initiate INSERT for empty block. size_t rows = shard_block.rows(); - if (rows == 0) - return; - OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); span.addAttribute("clickhouse.cluster", this->storage.cluster_name); span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); span.addAttribute("clickhouse.remote", [this]() { return storage.remote_database + "." + storage.remote_table; }); - span.addAttribute("clickhouse.rows", [rows]() { return std::to_string(rows); }); - span.addAttribute("clickhouse.bytes", [&shard_block]() { return std::to_string(shard_block.bytes()); }); + span.addAttribute("clickhouse.rows", rows); + span.addAttribute("clickhouse.bytes", [&shard_block]() { return toString(shard_block.bytes()); }); + + /// Do not initiate INSERT for empty block. + if (rows == 0) + return; if (!job.is_local_job || !settings.prefer_localhost_replica) { @@ -648,15 +650,15 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id) } -void DistributedSink::writeToLocal(const Cluster::ShardInfo& shard_info, const Block & block, size_t repeats) +void DistributedSink::writeToLocal(const Cluster::ShardInfo & shard_info, const Block & block, size_t repeats) { OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); span.addAttribute("clickhouse.cluster", this->storage.cluster_name); span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); span.addAttribute("clickhouse.remote", [this]() { return storage.remote_database + "." + storage.remote_table; }); - span.addAttribute("clickhouse.rows", [&block]() { return std::to_string(block.rows()); }); - span.addAttribute("clickhouse.bytes", [&block]() { return std::to_string(block.bytes()); }); + span.addAttribute("clickhouse.rows", [&block]() { return toString(block.rows()); }); + span.addAttribute("clickhouse.bytes", [&block]() { return toString(block.bytes()); }); try { @@ -677,7 +679,7 @@ void DistributedSink::writeToLocal(const Cluster::ShardInfo& shard_info, const B } -void DistributedSink::writeToShard(const Cluster::ShardInfo& shard_info, const Block & block, const std::vector & dir_names) +void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const Block & block, const std::vector & dir_names) { OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 5d7a5268865..af0c64cbd78 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -69,9 +69,9 @@ private: Block removeSuperfluousColumns(Block block) const; /// Increments finished_writings_count after each repeat. - void writeToLocal(const Cluster::ShardInfo& shard_info, const Block & block, size_t repeats); + void writeToLocal(const Cluster::ShardInfo & shard_info, const Block & block, size_t repeats); - void writeToShard(const Cluster::ShardInfo& shard_info, const Block & block, const std::vector & dir_names); + void writeToShard(const Cluster::ShardInfo & shard_info, const Block & block, const std::vector & dir_names); /// Performs synchronous insertion to remote nodes. If timeout_exceeded flag was set, throws. From 5e7cf1f238a0ce6fce5f7374ad0f64a43305f032 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 6 Sep 2022 12:14:40 +0300 Subject: [PATCH 398/582] Fix test_merge_tree_s3/test.py::test_freeze_system_unfreeze[node] --- .../test_merge_tree_s3/configs/config.xml | 16 ---------------- tests/integration/test_merge_tree_s3/test.py | 5 +++++ 2 files changed, 5 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/configs/config.xml b/tests/integration/test_merge_tree_s3/configs/config.xml index 5e5cf1f37e1..17c75fe9dd8 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.xml @@ -1,20 +1,4 @@ - 9000 - 127.0.0.1 - - - - true - none - - AcceptCertificateHandler - - - - - 500 - ./clickhouse/ - users.xml true diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f5a9bf153b7..2c51f8a97ed 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -21,6 +21,7 @@ def cluster(): cluster.add_instance( "node", main_configs=[ + "configs/config.xml", "configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml", ], @@ -519,6 +520,8 @@ def test_freeze_unfreeze(cluster, node_name): # Unfreeze all partitions from backup2. node.query("ALTER TABLE s3_test UNFREEZE WITH NAME 'backup2'") + wait_for_delete_s3_objects(cluster, FILES_OVERHEAD) + # Data should be removed from S3. assert ( len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD @@ -551,6 +554,8 @@ def test_freeze_system_unfreeze(cluster, node_name): # Unfreeze all data from backup3. node.query("SYSTEM UNFREEZE WITH NAME 'backup3'") + wait_for_delete_s3_objects(cluster, FILES_OVERHEAD) + # Data should be removed from S3. assert ( len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD From 7c4f42d014ba23a785f24f64e09710332532a049 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Sep 2022 13:58:26 +0200 Subject: [PATCH 399/582] Skip empty literals in lz4 decompression (#40142) --- src/Compression/LZ4_decompress_faster.cpp | 28 ++++++++++------ tests/performance/lz4.xml | 22 +++++++++++++ tests/performance/lz4_hits_columns.xml | 39 +++++++++++++++++++++++ 3 files changed, 80 insertions(+), 9 deletions(-) create mode 100644 tests/performance/lz4.xml create mode 100644 tests/performance/lz4_hits_columns.xml diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 269c1a681c5..f2084f34274 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -478,11 +478,7 @@ template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match, /// See also https://stackoverflow.com/a/30669632 template -bool NO_INLINE decompressImpl( - const char * const source, - char * const dest, - size_t source_size, - size_t dest_size) +bool NO_INLINE decompressImpl(const char * const source, char * const dest, size_t source_size, size_t dest_size) { const UInt8 * ip = reinterpret_cast(source); UInt8 * op = reinterpret_cast(dest); @@ -515,6 +511,18 @@ bool NO_INLINE decompressImpl( const unsigned token = *ip++; length = token >> 4; + + UInt8 * copy_end; + size_t real_length; + + /// It might be true fairly often for well-compressed columns. + /// ATST it may hurt performance in other cases because this condition is hard to predict (especially if the number of zeros is ~50%). + /// In such cases this `if` will significantly increase number of mispredicted instructions. But seems like it results in a + /// noticeable slowdown only for implementations with `copy_amount` > 8. Probably because they use havier instructions. + if constexpr (copy_amount == 8) + if (length == 0) + goto decompress_match; + if (length == 0x0F) { if (unlikely(ip + 1 >= input_end)) @@ -524,7 +532,7 @@ bool NO_INLINE decompressImpl( /// Copy literals. - UInt8 * copy_end = op + length; + copy_end = op + length; /// input: Hello, world /// ^-ip @@ -541,7 +549,7 @@ bool NO_INLINE decompressImpl( return false; // Due to implementation specifics the copy length is always a multiple of copy_amount - size_t real_length = 0; + real_length = 0; static_assert(copy_amount == 8 || copy_amount == 16 || copy_amount == 32); if constexpr (copy_amount == 8) @@ -552,9 +560,9 @@ bool NO_INLINE decompressImpl( real_length = (((length >> 5) + 1) * 32); if (unlikely(ip + real_length >= input_end + ADDITIONAL_BYTES_AT_END_OF_BUFFER)) - return false; + return false; - wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. + wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. if (copy_end == output_end) return true; @@ -562,6 +570,8 @@ bool NO_INLINE decompressImpl( ip += length; op = copy_end; + decompress_match: + if (unlikely(ip + 1 >= input_end)) return false; diff --git a/tests/performance/lz4.xml b/tests/performance/lz4.xml new file mode 100644 index 00000000000..c5cf2772121 --- /dev/null +++ b/tests/performance/lz4.xml @@ -0,0 +1,22 @@ + + create table t_lz4(a UInt64) engine=MergeTree order by tuple() + create table t_lz4_norm(a UInt64) engine=MergeTree order by tuple() + create table t_lz4_uncomp(a UInt32) engine=MergeTree order by a + + insert into t_lz4 select number % 100 from numbers_mt(5e7) order by rand() + optimize table t_lz4 final + + insert into t_lz4_norm select number from numbers_mt(5e7) order by rand() + optimize table t_lz4_norm final + + insert into t_lz4_uncomp select number from numbers_mt(5e7) + optimize table t_lz4_uncomp final + + select a from t_lz4 format Null + select a from t_lz4_norm format Null + select a from t_lz4_uncomp format Null + + drop table t_lz4 + drop table t_lz4_norm + drop table t_lz4_uncomp + diff --git a/tests/performance/lz4_hits_columns.xml b/tests/performance/lz4_hits_columns.xml new file mode 100644 index 00000000000..0b93e4bc4e8 --- /dev/null +++ b/tests/performance/lz4_hits_columns.xml @@ -0,0 +1,39 @@ + + + + column + + ClientIP + ClientTimeZone + CookieEnable + CounterClass + CounterID + EventDate + EventTime + GoodEvent + HitColor + JavaEnable + OpenerName + PageCharset + ParamCurrency + ParamPrice + Referer + RefererCategoryID + RefererHash + RegionID + SearchPhrase + SilverlightVersion4 + Title + TraficSourceID + URLCategoryID + UserAgent + UserAgentMinor + UserID + WatchID + WindowName + + + + + select {column} from hits_100m_single format Null + From 6ea7f1e0118af6d667dd03e10b76bb7fbb1bfa37 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Sep 2022 13:59:55 +0200 Subject: [PATCH 400/582] Better exception handling for ReadBufferFromS3 --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 4 +-- src/IO/ReadBufferFromS3.cpp | 14 ++++++-- src/IO/S3Common.cpp | 20 +++++++++++ src/IO/S3Common.h | 36 +++++++++++++++++-- src/IO/WriteBufferFromS3.cpp | 9 ++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 6 files changed, 73 insertions(+), 12 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index b3fa36ea169..08eba4758a0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -56,7 +56,7 @@ void throwIfError(const Aws::Utils::Outcome & response) if (!response.IsSuccess()) { const auto & err = response.GetError(); - throw Exception(ErrorCodes::S3_ERROR, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); + throw S3Exception(fmt::format("{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())), err.GetErrorType()); } } @@ -70,7 +70,7 @@ void throwIfUnexpectedError(const Aws::Utils::Outcome & response, if (!response.IsSuccess() && (!if_exists || !isNotFoundError(response.GetError().GetErrorType()))) { const auto & err = response.GetError(); - throw Exception(ErrorCodes::S3_ERROR, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); + throw S3Exception(fmt::format("{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())), err.GetErrorType()); } } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f7fd06cac08..38b59dae186 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -130,12 +130,19 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); break; } - catch (const Exception & e) + catch (const S3Exception & e) { watch.stop(); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1); + /// It doesn't make sense to retry Access Denied or No Such Key + if (!e.isRetryableError()) + { + tryLogCurrentException(log); + throw; + } + LOG_DEBUG( log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, Attempt: {}, Message: {}", @@ -306,7 +313,10 @@ std::unique_ptr ReadBufferFromS3::initialize() return std::make_unique(read_result.GetBody(), buffer_size); } else - throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + { + const auto & error = outcome.GetError(); + throw S3Exception(error.GetMessage(), error.GetErrorType()); + } } SeekableReadBufferPtr ReadBufferS3Factory::getReader() diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 1ff1c609952..2e2f5078769 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -35,6 +35,26 @@ # include +namespace DB +{ + +bool S3Exception::isRetryableError() const +{ + /// Looks like these list is quite conservative, add more codes if you wish + static const std::unordered_set unretryable_errors = { + Aws::S3::S3Errors::NO_SUCH_KEY, + Aws::S3::S3Errors::ACCESS_DENIED, + Aws::S3::S3Errors::INVALID_ACCESS_KEY_ID, + Aws::S3::S3Errors::INVALID_SIGNATURE, + Aws::S3::S3Errors::NO_SUCH_UPLOAD, + Aws::S3::S3Errors::NO_SUCH_BUCKET, + }; + + return !unretryable_errors.contains(code); +} + +} + namespace { diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 46a09ee8901..b190f0afdc5 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -7,23 +7,53 @@ #include #include #include +#include #include #include +#include + namespace Aws::S3 { class S3Client; } + namespace DB { - class RemoteHostFilter; - struct HttpHeader; - using HeaderCollection = std::vector; +namespace ErrorCodes +{ + extern const int S3_ERROR; } +class RemoteHostFilter; +struct HttpHeader; +using HeaderCollection = std::vector; + +class S3Exception : public Exception +{ +public: + S3Exception(const std::string & msg, const Aws::S3::S3Errors code_) + : Exception(msg, ErrorCodes::S3_ERROR) + , code(code_) + {} + + Aws::S3::S3Errors getS3ErrorCode() const + { + return code; + } + + bool isRetryableError() const; + +private: + const Aws::S3::S3Errors code; +}; +} + + namespace DB::S3 { + class ClientFactory { public: diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 2fb6502b0c4..cc46af361cd 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -173,7 +174,7 @@ void WriteBufferFromS3::finalizeImpl() auto response = client_ptr->HeadObject(request); if (!response.IsSuccess()) - throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", key, bucket); + throw S3Exception(fmt::format("Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", key, bucket), response.GetError().GetErrorType()); else LOG_TRACE(log, "Object {} exists after upload", key); } @@ -199,7 +200,7 @@ void WriteBufferFromS3::createMultipartUpload() LOG_TRACE(log, "Multipart upload has created. Bucket: {}, Key: {}, Upload id: {}", bucket, key, multipart_upload_id); } else - throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } void WriteBufferFromS3::writePart() @@ -311,7 +312,7 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) LOG_TRACE(log, "Writing part finished. Bucket: {}, Key: {}, Upload_id: {}, Etag: {}, Parts: {}", bucket, key, multipart_upload_id, task.tag, part_tags.size()); } else - throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); total_parts_uploaded++; } @@ -432,7 +433,7 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) if (outcome.IsSuccess()) LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); else - throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } void WriteBufferFromS3::waitForReadyBackGroundTasks() diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 762c3d52627..421aa33db97 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7654,7 +7654,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - LOG_TRACE(logger, "Can't remove parent zookeeper lock {} for part {}, because children {} ({}) were concurrently created", + LOG_TRACE(logger, "Can't remove parent zookeeper lock {} for part {}, because children {} ({}) exists", zookeeper_part_node, part_name, children.size(), fmt::join(children, ", ")); } } From 652f1bfd19a086349723b39a13054a1b9da586d5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Sep 2022 12:18:11 +0000 Subject: [PATCH 401/582] fix: pass -DNO_SSE3_OR_HIGHER=1 from packager --- docker/packager/packager | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/packager/packager b/docker/packager/packager index de41b23acb0..591262959b4 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -191,6 +191,7 @@ def parse_env_variables( elif is_amd64_sse2: cc = compiler[: -len(AMD64_SSE2_SUFFIX)] result.append("DEB_ARCH=amd64") + cmake_flags.append("-DNO_SSE3_OR_HIGHER=1") else: cc = compiler result.append("DEB_ARCH=amd64") From 422b1658eba9de10b176a75b2178f5688b3aa0f9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Sep 2022 14:42:48 +0200 Subject: [PATCH 402/582] Review fix --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/IO/S3Common.h | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 08eba4758a0..45304ac2fac 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -70,7 +70,7 @@ void throwIfUnexpectedError(const Aws::Utils::Outcome & response, if (!response.IsSuccess() && (!if_exists || !isNotFoundError(response.GetError().GetErrorType()))) { const auto & err = response.GetError(); - throw S3Exception(fmt::format("{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())), err.GetErrorType()); + throw S3Exception(err.GetErrorType(), "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); } } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index b190f0afdc5..3b1e3d82bc9 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -33,7 +33,16 @@ using HeaderCollection = std::vector; class S3Exception : public Exception { public: - S3Exception(const std::string & msg, const Aws::S3::S3Errors code_) + + // Format message with fmt::format, like the logging functions. + template + S3Exception(Aws::S3::S3Errors code_, fmt::format_string fmt, Args &&... args) + : Exception(fmt::format(fmt, std::forward(args)...), ErrorCodes::S3_ERROR) + , code(code_) + { + } + + S3Exception(const std::string & msg, Aws::S3::S3Errors code_) : Exception(msg, ErrorCodes::S3_ERROR) , code(code_) {} From e9b75deeba51daa59d4d3873548d05baf360fcd0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 3 Sep 2022 17:30:08 +0200 Subject: [PATCH 403/582] Make conversion YAML->XML more conventional. --- src/Common/Config/YAMLParser.cpp | 185 ++++++++++++------------- src/Common/tests/gtest_yaml_parser.cpp | 169 ++++++++++++++++++---- 2 files changed, 228 insertions(+), 126 deletions(-) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index bb83563ecc9..a34b539ee81 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -26,114 +26,107 @@ namespace ErrorCodes extern const int CANNOT_PARSE_YAML; } -/// A prefix symbol in yaml key -/// We add attributes to nodes by using a prefix symbol in the key part. -/// Currently we use @ as a prefix symbol. Note, that @ is reserved -/// by YAML standard, so we need to write a key-value pair like this: "@attribute": attr_value -const char YAML_ATTRIBUTE_PREFIX = '@'; - namespace { + /// A prefix symbol in yaml key + /// We add attributes to nodes by using a prefix symbol in the key part. + /// Currently we use @ as a prefix symbol. Note, that @ is reserved + /// by YAML standard, so we need to write a key-value pair like this: "@attribute": attr_value + const char YAML_ATTRIBUTE_PREFIX = '@'; -Poco::AutoPtr createCloneNode(Poco::XML::Element & original_node) -{ - Poco::AutoPtr clone_node = original_node.ownerDocument()->createElement(original_node.nodeName()); - original_node.parentNode()->appendChild(clone_node); - return clone_node; -} - -void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_element) -{ - auto * xml_document = parent_xml_element.ownerDocument(); - switch (node.Type()) + Poco::AutoPtr cloneXMLNode(const Poco::XML::Element & original_node) { - case YAML::NodeType::Scalar: - { - std::string value = node.as(); - Poco::AutoPtr xml_value = xml_document->createTextNode(value); - parent_xml_element.appendChild(xml_value); - break; - } + Poco::AutoPtr clone_node = original_node.ownerDocument()->createElement(original_node.nodeName()); + original_node.parentNode()->appendChild(clone_node); + return clone_node; + } - /// We process YAML Sequences as a - /// list of value tags with same key and different values. - /// For example, we translate this sequence - /// seq: - /// - val1 - /// - val2 - /// - /// into this: - /// val1 - /// val2 - case YAML::NodeType::Sequence: + void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_node) + { + auto * xml_document = parent_xml_node.ownerDocument(); + switch (node.Type()) { - for (const auto & child_node : node) - /// For sequences it depends how we want to process them. - /// Sequences of key-value pairs such as: - /// seq: - /// - k1: val1 - /// - k2: val2 - /// into xml like this: - /// - /// val1 - /// val2 - /// - /// - /// But, if the sequence is just a list, the root-node needs to be repeated, such as: - /// seq: - /// - val1 - /// - val2 - /// into xml like this: - /// val1 - /// val2 - /// - /// Therefore check what type the child is, for further processing. - /// Mixing types (values list or map) will lead to strange results but should not happen. - if (parent_xml_element.hasChildNodes() && !child_node.IsMap()) - { - /// Create a new parent node with same tag for each child node - processNode(child_node, *createCloneNode(parent_xml_element)); - } - else - { - /// Map, so don't recreate the parent node but add directly - processNode(child_node, parent_xml_element); - } - break; - } - case YAML::NodeType::Map: - { - for (const auto & key_value_pair : node) + case YAML::NodeType::Scalar: { - const auto & key_node = key_value_pair.first; - const auto & value_node = key_value_pair.second; - std::string key = key_node.as(); - bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar()); - if (is_attribute) - { - /// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key - auto attribute_name = key.substr(1); - std::string value = value_node.as(); - parent_xml_element.setAttribute(attribute_name, value); - } - else - { - Poco::AutoPtr xml_key = xml_document->createElement(key); - parent_xml_element.appendChild(xml_key); - processNode(value_node, *xml_key); - } + std::string value = node.as(); + Poco::AutoPtr xml_value = xml_document->createTextNode(value); + parent_xml_node.appendChild(xml_value); + break; + } + + /// For sequences we repeat the parent xml node. For example, + /// seq: + /// - val1 + /// - val2 + /// is converted into the following xml: + /// val1 + /// val2 + /// + /// A sequence of mappings is converted in the same way: + /// seq: + /// - k1: val1 + /// k2: val2 + /// - k3: val3 + /// is converted into the following xml: + /// val1val2 + /// val3 + case YAML::NodeType::Sequence: + { + size_t i = 0; + for (auto it = node.begin(); it != node.end(); ++it, ++i) + { + const auto & child_node = *it; + + bool need_clone_parent_xml_node = (i > 0); + + if (need_clone_parent_xml_node) + { + /// Create a new parent node with same tag for each child node + processNode(child_node, *cloneXMLNode(parent_xml_node)); + } + else + { + /// Map, so don't recreate the parent node but add directly + processNode(child_node, parent_xml_node); + } + } + break; + } + + case YAML::NodeType::Map: + { + for (const auto & key_value_pair : node) + { + const auto & key_node = key_value_pair.first; + const auto & value_node = key_value_pair.second; + std::string key = key_node.as(); + bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar()); + if (is_attribute) + { + /// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key + auto attribute_name = key.substr(1); + std::string value = value_node.as(); + parent_xml_node.setAttribute(attribute_name, value); + } + else + { + Poco::AutoPtr xml_key = xml_document->createElement(key); + parent_xml_node.appendChild(xml_key); + processNode(value_node, *xml_key); + } + } + break; + } + + case YAML::NodeType::Null: break; + case YAML::NodeType::Undefined: + { + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with undefined type and cannot continue parsing of the file"); } - break; - } - case YAML::NodeType::Null: break; - case YAML::NodeType::Undefined: - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with undefined type and cannot continue parsing of the file"); } } } -} Poco::AutoPtr YAMLParser::parse(const String& path) { diff --git a/src/Common/tests/gtest_yaml_parser.cpp b/src/Common/tests/gtest_yaml_parser.cpp index 8457e6fd4e1..4ffd66ae3a1 100644 --- a/src/Common/tests/gtest_yaml_parser.cpp +++ b/src/Common/tests/gtest_yaml_parser.cpp @@ -13,40 +13,12 @@ using namespace DB; -TEST(Common, YamlParserInvalidFile) +TEST(YamlParser, InvalidFile) { ASSERT_THROW(YAMLParser::parse("some-non-existing-file.yaml"), Exception); } -TEST(Common, YamlParserProcessKeysList) -{ - auto yaml_file = getFileWithContents("keys-list.yaml", R"YAML( -operator: - access_management: "1" - networks: - - ip: "10.1.6.168" - - ip: "::1" - - ip: "127.0.0.1" -)YAML"); - SCOPE_EXIT({ yaml_file->remove(); }); - - Poco::AutoPtr xml = YAMLParser::parse(yaml_file->path()); - auto *p_node = xml->getNodeByPath("/clickhouse"); - EXPECT_EQ(xmlNodeAsString(p_node), R"CONFIG( - -1 - -10.1.6.168 -::1 -127.0.0.1 - - - -)CONFIG"); - -} - -TEST(Common, YamlParserProcessValuesList) +TEST(YamlParser, ProcessValuesList) { auto yaml_file = getFileWithContents("values-list.yaml", R"YAML( rules: @@ -75,4 +47,141 @@ rules: )CONFIG"); } + +TEST(YamlParser, ProcessKeysList) +{ + auto yaml_file = getFileWithContents("keys-list.yaml", R"YAML( +operator: + access_management: 1 + networks: + ip: + - 10.1.6.168 + - ::1 + - 127.0.0.1 +)YAML"); + SCOPE_EXIT({ yaml_file->remove(); }); + + Poco::AutoPtr xml = YAMLParser::parse(yaml_file->path()); + auto *p_node = xml->getNodeByPath("/clickhouse"); + EXPECT_EQ(xmlNodeAsString(p_node), R"CONFIG( + +1 + +10.1.6.168 +::1 +127.0.0.1 + + + +)CONFIG"); + +} + +TEST(YamlParser, ProcessListAttributes) +{ + auto yaml_file = getFileWithContents("list_attributes.yaml", R"YAML( +seq: + - "@attr1": x + - k1: val1 + k2: val2 + "@attr2": y + - k3: val3 + "@attr3": z +)YAML"); + SCOPE_EXIT({ yaml_file->remove(); }); + + Poco::AutoPtr xml = YAMLParser::parse(yaml_file->path()); + auto *p_node = xml->getNodeByPath("/clickhouse"); + EXPECT_EQ(xmlNodeAsString(p_node), R"CONFIG( + + +val1 +val2 + + +val3 + + +)CONFIG"); + +} + +TEST(YamlParser, ProcessMapAttributes) +{ + auto yaml_file = getFileWithContents("map_attributes.yaml", R"YAML( +map: + "@attr1": x + k1: val1 + k2: val2 + "@attr2": y + k3: val3 + "@attr3": z +)YAML"); + SCOPE_EXIT({ yaml_file->remove(); }); + + Poco::AutoPtr xml = YAMLParser::parse(yaml_file->path()); + auto *p_node = xml->getNodeByPath("/clickhouse"); + EXPECT_EQ(xmlNodeAsString(p_node), R"CONFIG( + +val1 +val2 +val3 + + +)CONFIG"); + +} + +TEST(YamlParser, ClusterDef) +{ + auto yaml_file = getFileWithContents("cluster_def.yaml", R"YAML( +test_cluster: + shard: + - internal_replication: false + replica: + - host: 127.0.0.1 + port: 9000 + - host: 127.0.0.2 + port: 9000 + - internal_replication: true + replica: + - host: 127.0.0.3 + port: 9000 + - host: 127.0.0.4 + port: 9000 +)YAML"); + SCOPE_EXIT({ yaml_file->remove(); }); + + Poco::AutoPtr xml = YAMLParser::parse(yaml_file->path()); + auto *p_node = xml->getNodeByPath("/clickhouse"); + EXPECT_EQ(xmlNodeAsString(p_node), R"CONFIG( + + +false + +127.0.0.1 +9000 + + +127.0.0.2 +9000 + + + +true + +127.0.0.3 +9000 + + +127.0.0.4 +9000 + + + + +)CONFIG"); + +} + #endif From 6b55f4dd6870fe25a47d230865b48470964e5c70 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 3 Sep 2022 19:46:04 +0200 Subject: [PATCH 404/582] Use pretty-print to output preprocessed configs in readable form. --- src/Common/Config/ConfigProcessor.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 8d7c26e967f..d3462321e83 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -729,7 +730,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, if (!preprocessed_path_parent.empty()) fs::create_directories(preprocessed_path_parent); } - DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml); + DOMWriter writer; + writer.setNewLine("\n"); + writer.setIndent(" "); + writer.setOptions(Poco::XML::XMLWriter::PRETTY_PRINT); + writer.writeNode(preprocessed_path, loaded_config.preprocessed_xml); LOG_DEBUG(log, "Saved preprocessed configuration to '{}'.", preprocessed_path); } catch (Poco::Exception & e) From 46f281f9ea74b581d895075bb00c73c729b4489a Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 6 Sep 2022 21:36:58 +0800 Subject: [PATCH 405/582] remove parser logic for WITH GROUPING SETS --- src/Parsers/ParserSelectQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 66428b144bf..cf335270734 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -224,8 +224,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->group_by_with_rollup = true; else if (s_cube.ignore(pos, expected)) select_query->group_by_with_cube = true; - else if (s_grouping_sets.ignore(pos, expected)) - select_query->group_by_with_grouping_sets = true; else if (s_totals.ignore(pos, expected)) select_query->group_by_with_totals = true; else From f76c1482bd148d5282222eff468bccc350421b3a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Sep 2022 13:56:32 +0000 Subject: [PATCH 406/582] try to fix filling of missed Nested columns with multiple levels --- src/Interpreters/inplaceBlockConversions.cpp | 179 ++++++++++-------- src/Interpreters/inplaceBlockConversions.h | 2 + src/Storages/MergeTree/IMergeTreeReader.cpp | 37 ++-- src/Storages/MergeTree/IMergeTreeReader.h | 2 + .../MergeTree/MergeTreeReaderCompact.cpp | 2 + .../MergeTree/MergeTreeReaderWide.cpp | 13 ++ src/Storages/StorageMemory.cpp | 2 +- .../0_stateless/01825_type_json_17.reference | 20 ++ .../0_stateless/01825_type_json_17.sql | 32 +++- 9 files changed, 191 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index e4f1b46fc91..116cd91c7cf 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -188,49 +188,13 @@ ActionsDAGPtr evaluateMissingDefaults( return createExpressions(header, expr_list, save_unneeded_columns, context); } -static bool arrayHasNoElementsRead(const IColumn & column) +static std::unordered_map collectOffsetsColumns( + const NamesAndTypesList & available_columns, const Columns & res_columns) { - const auto * column_array = typeid_cast(&column); + std::unordered_map offsets_columns; - if (!column_array) - return false; - - size_t size = column_array->size(); - if (!size) - return false; - - const auto & array_data = column_array->getData(); - if (const auto * nested_array = typeid_cast(&array_data)) - return arrayHasNoElementsRead(*nested_array); - - if (!array_data.empty()) - return false; - - size_t last_offset = column_array->getOffsets()[size - 1]; - return last_offset != 0; -} - -void fillMissingColumns( - Columns & res_columns, - size_t num_rows, - const NamesAndTypesList & requested_columns, - const NamesAndTypesList & available_columns, - StorageMetadataPtr metadata_snapshot) -{ - size_t num_columns = requested_columns.size(); - if (num_columns != res_columns.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid number of columns passed to fillMissingColumns. Expected {}, got {}", - num_columns, res_columns.size()); - - /// For a missing column of a nested data structure we must create not a column of empty - /// arrays, but a column of arrays of correct length. - - /// First, collect offset columns for all arrays in the block. - - std::unordered_map offset_columns; auto available_column = available_columns.begin(); - for (size_t i = 0; i < num_columns; ++i, ++available_column) + for (size_t i = 0; i < available_columns.size(); ++i, ++available_column) { if (res_columns[i] == nullptr || isColumnConst(*res_columns[i])) continue; @@ -243,75 +207,122 @@ void fillMissingColumns( if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) return; - auto subname = ISerialization::getSubcolumnNameForStream(subpath); - auto & offsets_column = offset_columns[Nested::concatenateName(name_in_storage, subname)]; + const auto & current_offsets_column = subpath.back().data.column; /// If for some reason multiple offsets columns are present /// for the same nested data structure, choose the one that is not empty. - if (!offsets_column || offsets_column->empty()) - offsets_column = subpath.back().data.column; + if (current_offsets_column && !current_offsets_column->empty()) + { + auto subname = ISerialization::getSubcolumnNameForStream(subpath); + auto full_name = Nested::concatenateName(name_in_storage, subname); + auto & offsets_column = offsets_columns[full_name]; + if (!offsets_column) + offsets_column = current_offsets_column; + + #ifndef NDEBUG + const auto & offsets_data = assert_cast(*offsets_column).getData(); + const auto & current_offsets_data = assert_cast(*current_offsets_column).getData(); + + if (offsets_data != current_offsets_data) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Found non-equal columns with offsets (sizes: {} and {}) for stream {}", + offsets_data.size(), current_offsets_data.size(), full_name); + #endif + } }, available_column->type, res_columns[i]); } + return offsets_columns; +} + +void fillMissingColumns( + Columns & res_columns, + size_t num_rows, + const NamesAndTypesList & requested_columns, + const NamesAndTypesList & available_columns, + const NameSet & partially_read_columns, + StorageMetadataPtr metadata_snapshot) +{ + size_t num_columns = requested_columns.size(); + if (num_columns != res_columns.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid number of columns passed to fillMissingColumns. Expected {}, got {}", + num_columns, res_columns.size()); + + /// For a missing column of a nested data structure + /// we must create not a column of empty arrays, + /// but a column of arrays of correct length. + + /// First, collect offset columns for all arrays in the block. + auto offset_columns = collectOffsetsColumns(available_columns, res_columns); + /// Insert default values only for columns without default expressions. auto requested_column = requested_columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++requested_column) { const auto & [name, type] = *requested_column; - if (res_columns[i] && arrayHasNoElementsRead(*res_columns[i])) + if (res_columns[i] && partially_read_columns.contains(name)) res_columns[i] = nullptr; - if (res_columns[i] == nullptr) + if (res_columns[i]) + continue; + + if (metadata_snapshot && metadata_snapshot->getColumns().hasDefault(name)) + continue; + + std::vector current_offsets; + size_t num_dimensions = 0; + + if (const auto * array_type = typeid_cast(type.get())) { - if (metadata_snapshot && metadata_snapshot->getColumns().hasDefault(name)) - continue; + num_dimensions = getNumberOfDimensions(*array_type); + current_offsets.resize(num_dimensions); - std::vector current_offsets; - bool has_all_offsets = true; + auto serialization = IDataType::getSerialization(*requested_column); + auto name_in_storage = Nested::extractTableName(requested_column->name); - const auto * array_type = typeid_cast(type.get()); - if (array_type) + serialization->enumerateStreams([&](const auto & subpath) { - auto serialization = IDataType::getSerialization(*requested_column); - auto name_in_storage = Nested::extractTableName(requested_column->name); + if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) + return; - serialization->enumerateStreams([&](const auto & subpath) + size_t level = ISerialization::getArrayLevel(subpath); + assert(level < num_dimensions); + + auto subname = ISerialization::getSubcolumnNameForStream(subpath); + auto it = offset_columns.find(Nested::concatenateName(name_in_storage, subname)); + if (it != offset_columns.end()) + current_offsets[level] = it->second; + }); + + for (size_t j = 0; j < num_dimensions; ++j) + { + if (!current_offsets[j]) { - if (!has_all_offsets) - return; - - if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) - return; - - auto subname = ISerialization::getSubcolumnNameForStream(subpath); - auto it = offset_columns.find(Nested::concatenateName(name_in_storage, subname)); - if (it != offset_columns.end()) - current_offsets.emplace_back(it->second); - else - has_all_offsets = false; - - }, type); + current_offsets.resize(j); + break; + } } + } - if (array_type && has_all_offsets) - { - assert(!current_offsets.empty()); - auto scalar_type = getBaseTypeOfArray(type); + if (!current_offsets.empty()) + { + size_t num_empty_dimensions = num_dimensions - current_offsets.size(); + auto scalar_type = createArrayOfType(getBaseTypeOfArray(type), num_empty_dimensions); - size_t data_size = assert_cast(*current_offsets.back()).getData().back(); - res_columns[i] = scalar_type->createColumnConstWithDefaultValue(data_size)->convertToFullColumnIfConst(); + size_t data_size = assert_cast(*current_offsets.back()).getData().back(); + res_columns[i] = scalar_type->createColumnConstWithDefaultValue(data_size)->convertToFullColumnIfConst(); - for (auto it = current_offsets.rbegin(); it != current_offsets.rend(); ++it) - res_columns[i] = ColumnArray::create(res_columns[i], *it); - } - else - { - /// We must turn a constant column into a full column because the interpreter could infer - /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. - res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); - } + for (auto it = current_offsets.rbegin(); it != current_offsets.rend(); ++it) + res_columns[i] = ColumnArray::create(res_columns[i], *it); + } + else + { + /// We must turn a constant column into a full column because the interpreter could infer + /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. + res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); } } } diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index 70187d5aace..bea44bf6db9 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -44,6 +45,7 @@ void fillMissingColumns( size_t num_rows, const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, + const NameSet & partially_read_columns, StorageMetadataPtr metadata_snapshot); } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index b77975ff9ad..73f9cc8b75d 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -65,7 +65,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e try { NamesAndTypesList available_columns(columns_to_read.begin(), columns_to_read.end()); - DB::fillMissingColumns(res_columns, num_rows, requested_columns, available_columns, metadata_snapshot); + DB::fillMissingColumns(res_columns, num_rows, requested_columns, available_columns, partially_read_columns, metadata_snapshot); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); @@ -206,9 +206,9 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const NameAndTypePair & required_column) const { - auto get_offset_streams = [](const auto & serialization, const auto & name_in_storage) + auto get_offsets_streams = [](const auto & serialization, const auto & name_in_storage) { - NameSet offset_streams; + Names offsets_streams; serialization->enumerateStreams([&](const auto & subpath) { if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) @@ -216,31 +216,44 @@ IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const Na auto subname = ISerialization::getSubcolumnNameForStream(subpath); auto full_name = Nested::concatenateName(name_in_storage, subname); - offset_streams.insert(full_name); + offsets_streams.push_back(full_name); }); - return offset_streams; + return offsets_streams; }; auto required_name_in_storage = Nested::extractTableName(required_column.getNameInStorage()); - auto required_offset_streams = get_offset_streams(getSerializationInPart(required_column), required_name_in_storage); + auto required_offsets_streams = get_offsets_streams(getSerializationInPart(required_column), required_name_in_storage); + size_t max_matched_streams = 0; + ColumnPosition position; + + /// Find column that has maximal number of matching + /// offsets columns with required_column. for (const auto & part_column : data_part->getColumns()) { auto name_in_storage = Nested::extractTableName(part_column.name); if (name_in_storage != required_name_in_storage) continue; - auto offset_streams = get_offset_streams(data_part->getSerialization(part_column.name), name_in_storage); + auto offsets_streams = get_offsets_streams(data_part->getSerialization(part_column.name), name_in_storage); + NameSet offsets_streams_set(offsets_streams.begin(), offsets_streams.end()); - bool has_all_offsets = std::all_of(required_offset_streams.begin(), required_offset_streams.end(), - [&](const auto & stream_name) { return offset_streams.contains(stream_name); }); + size_t i = 0; + for (; i < required_offsets_streams.size(); ++i) + { + if (!offsets_streams_set.contains(required_offsets_streams[i])) + break; + } - if (has_all_offsets) - return data_part->getColumnPosition(part_column.name); + if (i && (!position || i > max_matched_streams)) + { + max_matched_streams = i; + position = data_part->getColumnPosition(part_column.name); + } } - return {}; + return position; } void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index fa1bf988fbf..c2fab33bcf9 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -93,6 +93,8 @@ protected: using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const NameAndTypePair & column) const; + NameSet partially_read_columns; + private: /// Alter conversions, which must be applied on fly if required MergeTreeData::AlterConversions alter_conversions; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 87e0d75a871..440879a9340 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -142,6 +142,8 @@ void MergeTreeReaderCompact::fillColumnPositions() } column_positions[i] = std::move(position); + if (read_only_offsets[i]) + partially_read_columns.insert(column_to_read.name); } } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index abc3816c386..f1124c674c1 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -159,12 +159,18 @@ void MergeTreeReaderWide::addStreams( const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { + bool has_any_stream = false; + bool has_all_streams = true; + ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) { String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); if (streams.contains(stream_name)) + { + has_any_stream = true; return; + } bool data_file_exists = data_part->checksums.files.contains(stream_name + DATA_FILE_EXTENSION); @@ -172,8 +178,12 @@ void MergeTreeReaderWide::addStreams( * It is necessary since it allows to add new column to structure of the table without creating new files for old parts. */ if (!data_file_exists) + { + has_all_streams = false; return; + } + has_any_stream = true; bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; streams.emplace(stream_name, std::make_unique( @@ -185,6 +195,9 @@ void MergeTreeReaderWide::addStreams( }; serialization->enumerateStreams(callback); + + if (has_any_stream && !has_all_streams) + partially_read_columns.insert(name_and_type.name); } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index d8a46b07102..e4dbfe15095 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -95,7 +95,7 @@ protected: ++name_and_type; } - fillMissingColumns(columns, src.rows(), column_names_and_types, column_names_and_types, /*metadata_snapshot=*/ nullptr); + fillMissingColumns(columns, src.rows(), column_names_and_types, column_names_and_types, {}, nullptr); assert(std::all_of(columns.begin(), columns.end(), [](const auto & column) { return column != nullptr; })); return Chunk(std::move(columns), src.rows()); diff --git a/tests/queries/0_stateless/01825_type_json_17.reference b/tests/queries/0_stateless/01825_type_json_17.reference index 96e58224f32..0f97bfed5bc 100644 --- a/tests/queries/0_stateless/01825_type_json_17.reference +++ b/tests/queries/0_stateless/01825_type_json_17.reference @@ -3,5 +3,25 @@ Tuple(arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)), {"obj":{"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}],"id":2}} [['bbb','']] [['aaa','ccc']] [['ddd','']] [['','']] +1 [[0,0]] [[10,20]] +Tuple(arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))), id Int8) +{"obj":{"arr":[{"k1":[{"k2":"aaa","k3":[]}]}],"id":1}} +{"obj":{"arr":[{"k1":[{"k2":"bbb","k3":[{"k4":10}]},{"k2":"ccc","k3":[{"k4":20}]}]}],"id":2}} +[['aaa']] [[[]]] +[['bbb','ccc']] [[[10],[20]]] +1 +[[[]]] +[[[10],[20]]] +Tuple(arr Nested(k1 Nested(k2 String, k4 Nested(k5 Int8)), k3 String), id Int8) +{"obj":{"arr":[{"k1":[],"k3":"qqq"},{"k1":[],"k3":"www"}],"id":1}} +{"obj":{"arr":[{"k1":[{"k2":"aaa","k4":[]}],"k3":"eee"}],"id":2}} +{"obj":{"arr":[{"k1":[{"k2":"bbb","k4":[{"k5":10}]},{"k2":"ccc","k4":[{"k5":20}]}],"k3":"rrr"}],"id":3}} +['qqq','www'] [[],[]] [[],[]] +['eee'] [['aaa']] [[[]]] +['rrr'] [['bbb','ccc']] [[[10],[20]]] +1 +[[],[]] +[[[]]] +[[[10],[20]]] diff --git a/tests/queries/0_stateless/01825_type_json_17.sql b/tests/queries/0_stateless/01825_type_json_17.sql index b34357f8ef1..e3c0c83322b 100644 --- a/tests/queries/0_stateless/01825_type_json_17.sql +++ b/tests/queries/0_stateless/01825_type_json_17.sql @@ -7,12 +7,42 @@ SET output_format_json_named_tuples_as_objects = 1; CREATE TABLE t_json_17(obj JSON) ENGINE = MergeTree ORDER BY tuple(); +DROP FUNCTION IF EXISTS hasValidSizes17; +CREATE FUNCTION hasValidSizes17 AS (arr1, arr2) -> length(arr1) = length(arr2) AND arrayAll((x, y) -> length(x) = length(y), arr1, arr2); + +SYSTEM STOP MERGES t_json_17; + INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; SELECT obj FROM t_json_17 ORDER BY obj.id FORMAT JSONEachRow; SELECT obj.arr.k1.k3, obj.arr.k1.k2 FROM t_json_17 ORDER BY obj.id; +SELECT sum(hasValidSizes17(obj.arr.k1.k3, obj.arr.k1.k2)) == count() FROM t_json_17; SELECT obj.arr.k1.k4 FROM t_json_17 ORDER BY obj.id; -DROP TABLE IF EXISTS t_json_17; +TRUNCATE TABLE t_json_17; + +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa"}]}]} +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k2": "bbb", "k3": [{"k4": 10}]}, {"k2": "ccc", "k3": [{"k4": 20}]}]}]} + +SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; +SELECT obj FROM t_json_17 ORDER BY obj.id FORMAT JSONEachRow; +SELECT obj.arr.k1.k2, obj.arr.k1.k3.k4 FROM t_json_17 ORDER BY obj.id; +SELECT sum(hasValidSizes17(obj.arr.k1.k2, obj.arr.k1.k3.k4)) == count() FROM t_json_17; +SELECT obj.arr.k1.k3.k4 FROM t_json_17 ORDER BY obj.id; + +TRUNCATE TABLE t_json_17; + +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k3": "qqq"}, {"k3": "www"}]} +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k2": "aaa"}], "k3": "eee"}]} +INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 3, "arr": [{"k1": [{"k2": "bbb", "k4": [{"k5": 10}]}, {"k2": "ccc", "k4": [{"k5": 20}]}], "k3": "rrr"}]} + +SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; +SELECT obj FROM t_json_17 ORDER BY obj.id FORMAT JSONEachRow; +SELECT obj.arr.k3, obj.arr.k1.k2, obj.arr.k1.k4.k5 FROM t_json_17 ORDER BY obj.id; +SELECT sum(hasValidSizes17(obj.arr.k1.k2, obj.arr.k1.k4.k5)) == count() FROM t_json_17; +SELECT obj.arr.k1.k4.k5 FROM t_json_17 ORDER BY obj.id; + +DROP FUNCTION hasValidSizes17; +DROP TABLE t_json_17; From 6ced4131caf290bf6b5e47f9decac43b5a261320 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 6 Sep 2022 22:00:00 +0800 Subject: [PATCH 407/582] exception safe Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index c05d3385bc8..9efb278a670 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -283,10 +283,6 @@ TracingContextHolder::TracingContextHolder( this->root_span.start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - /// This object is created to initialize tracing context on a new thread, - /// it's helpful to record the thread_id so that we know the thread switching from the span log - this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); - /// set up trace context on current thread current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; @@ -306,6 +302,18 @@ TracingContextHolder::~TracingContextHolder() auto shared_span_log = current_thread_trace_context.span_log.lock(); if (shared_span_log) { + try + { + /// This object is created to initialize tracing context on a new thread, + /// it's helpful to record the thread_id so that we know the thread switching from the span log + this->root_span.addAttribute("clickhouse.thread_id", getThreadId()); + } + catch (...) + { + /// It's acceptable that the attribute is not recorded in case of any exception, + /// so the exception is ignored to try to log the span. + } + this->root_span.finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); From f21ab12d8e364e0fce760bb5228a8ef00c8c0c66 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Tue, 6 Sep 2022 22:21:31 +0800 Subject: [PATCH 408/582] Use sleep to wait for flush --- ...7_opentelemetry_insert_on_distributed_table.sh | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index 6f766e9f3bb..b9b5dd2d424 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -33,15 +33,11 @@ SET insert_distributed_sync=0; INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1 VALUES(1),(2); " -# Wait complete of ASYNC INSERT on distributed table -wait +# Wait 10s to complete of ASYNC INSERT on distributed table and flush of system.opentelemetry_span_log +sleep 10 # Check log ${CLICKHOUSE_CLIENT} -nq " --- Flush opentelemetry span log on all nodes -SET distributed_ddl_output_mode = 'none'; -SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; - -- Above INSERT will insert data to two shards respectively, so there will be two spans generated SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%writeToLocal%'; SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%processFile%'; @@ -64,12 +60,11 @@ SET insert_distributed_sync=1; INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1 VALUES(1),(2); " +# Wait 10s to flush system.opentelemetry_span_log +sleep 10 + # Check log ${CLICKHOUSE_CLIENT} -nq " --- Flush opentelemetry span log on all nodes -SET distributed_ddl_output_mode = 'none'; -SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; - -- Above INSERT will insert data to two shards in the same flow, so there should be two spans generated with the same operation name SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%runWritingJob%'; " From 487bc0fba32fcd81a44aeb974c2470dceac8a33f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Sep 2022 23:10:03 +0200 Subject: [PATCH 409/582] Fix heap use after free --- programs/local/LocalServer.cpp | 11 +++++------ src/Client/ClientBase.h | 8 +++++--- src/Client/LocalConnection.cpp | 3 --- src/Client/LocalConnection.h | 1 - src/Interpreters/Context.cpp | 14 +++++++++++--- 5 files changed, 21 insertions(+), 16 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index fa43c9f1283..fa84199ea92 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -227,6 +227,10 @@ void LocalServer::cleanup() global_context.reset(); } + /// thread status should be destructed before shared context because it relies on process list. + thread_status.reset(); + shared_context.reset(); + status.reset(); // Delete the temporary directory if needed. @@ -366,12 +370,7 @@ int LocalServer::main(const std::vector & /*args*/) try { UseSSL use_ssl; - ThreadStatus thread_status; - SCOPE_EXIT_SAFE({ - /// Context should not live longer than thread_status. - global_context.reset(); - shared_context.reset(); - }); + thread_status.emplace(); StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true)); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 6b19c1b8e02..219d35d87cd 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -176,9 +176,6 @@ protected: bool stderr_is_a_tty = false; /// stderr is a terminal. uint64_t terminal_width = 0; - ServerConnectionPtr connection; - ConnectionParameters connection_parameters; - String format; /// Query results output format. bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering. bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering. @@ -199,6 +196,11 @@ protected: SharedContextHolder shared_context; ContextMutablePtr global_context; + std::optional thread_status; + + ServerConnectionPtr connection; + ConnectionParameters connection_parameters; + /// Buffer that reads from stdin in batch mode. ReadBufferFromFileDescriptor std_in{STDIN_FILENO}; /// Console output. diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index b10e24f1ae4..7ac68324915 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -31,9 +31,6 @@ LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool /// Authenticate and create a context to execute queries. session.authenticate("default", "", Poco::Net::SocketAddress{}); session.makeSessionContext(); - - if (!CurrentThread::isInitialized()) - thread_status.emplace(); } LocalConnection::~LocalConnection() diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 7967874d11f..7a1a73006ac 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -156,7 +156,6 @@ private: String description = "clickhouse-local"; std::optional state; - std::optional thread_status; /// Last "server" packet. std::optional next_packet_type; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d39c39cdb15..87c2a997afa 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -143,7 +143,7 @@ namespace ErrorCodes /** Set of known objects (environment), that could be used in query. * Shared (global) part. Order of members (especially, order of destruction) is very important. */ -struct ContextSharedPart +struct ContextSharedPart : boost::noncopyable { Poco::Logger * log = &Poco::Logger::get("Context"); @@ -314,11 +314,19 @@ struct ContextSharedPart ~ContextSharedPart() { - /// Wait for thread pool for background writes, - /// since it may use per-user MemoryTracker which will be destroyed here. try { + /// Wait for thread pool for background writes, + /// since it may use per-user MemoryTracker which will be destroyed here. IObjectStorage::getThreadPoolWriter().wait(); + /// Make sure that threadpool is destructed before this->process_list + /// because thread_status, which was created for threads inside threadpool, + /// relies on it. + if (load_marks_threadpool) + { + load_marks_threadpool->wait(); + load_marks_threadpool.reset(); + } } catch (...) { From ce9b76f41649269cc05a680fbb26f1bf690d520f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Sep 2022 15:01:47 +0000 Subject: [PATCH 410/582] fix Nested in in-memory parts --- src/Storages/MergeTree/MergeTreeReaderInMemory.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 568e1cecf02..06f5785c868 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -37,8 +37,13 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. if (!part_in_memory->block.has(column_to_read.name) && typeid_cast(column_to_read.type.get())) - if (auto offset_position = findColumnForOffsets(column_to_read)) - positions_for_offsets[column_to_read.name] = *offset_position; + { + if (auto offsets_position = findColumnForOffsets(column_to_read)) + { + positions_for_offsets[column_to_read.name] = *offsets_position; + partially_read_columns.insert(column_to_read.name); + } + } } } From 63e992d52d3e71fc8e3491a995732e68c9be204f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 6 Sep 2022 13:47:07 +0200 Subject: [PATCH 411/582] Edit test configs. --- src/Common/tests/gtest_merge_configs.cpp | 14 ++++------- .../configs/config.d/path.yaml | 24 +++++-------------- .../configs/users.yaml | 1 - .../configs/config.d/path.yaml | 24 +++++-------------- .../test_config_yaml_full/configs/users.yaml | 1 - .../test_config_yaml_main/configs/users.yaml | 1 - 6 files changed, 16 insertions(+), 49 deletions(-) diff --git a/src/Common/tests/gtest_merge_configs.cpp b/src/Common/tests/gtest_merge_configs.cpp index 293de60f7f6..2cc7f4a99af 100644 --- a/src/Common/tests/gtest_merge_configs.cpp +++ b/src/Common/tests/gtest_merge_configs.cpp @@ -43,11 +43,8 @@ clickhouse: text_log: database: system table: text_log - partition_by: - "@remove": "1" - engine: - - "@replace" : "1" - - "ENGINE MergeTree" + partition_by: {"@remove": "1"} + engine: "ENGINE MergeTree" flush_interval_milliseconds: 7500 level: debug )YAML"; @@ -112,11 +109,8 @@ clickhouse: text_log : database: system table: text_log - partition_by: - "@remove": "1" - engine: - - "@replace" : "1" - - "ENGINE MergeTree" + partition_by: {"@remove": "1"} + engine: "ENGINE MergeTree" flush_interval_milliseconds: 7500 level: debug )YAML"; diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.d/path.yaml b/tests/integration/test_config_xml_yaml_mix/configs/config.d/path.yaml index 7fd5b1a0478..de0fefd2f48 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/config.d/path.yaml +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.d/path.yaml @@ -1,18 +1,6 @@ -path: - - /var/lib/clickhouse - - "@replace": replace -tmp_path: - - /var/lib/clickhouse/tmp/ - - "@replace": replace -user_files_path: - - /var/lib/clickhouse/user_files/ - - "@replace": replace -format_schema_path: - - /var/lib/clickhouse/format_schemas/ - - "@replace": replace -access_control_path: - - /var/lib/clickhouse/access/ - - "@replace": replace -top_level_domains_path: - - /var/lib/clickhouse/top_level_domains/ - - "@replace": replace +path: /var/lib/clickhouse +tmp_path: /var/lib/clickhouse/tmp/ +user_files_path: /var/lib/clickhouse/user_files/ +format_schema_path: /var/lib/clickhouse/format_schemas/ +access_control_path: /var/lib/clickhouse/access/ +top_level_domains_path: /var/lib/clickhouse/top_level_domains/ diff --git a/tests/integration/test_config_xml_yaml_mix/configs/users.yaml b/tests/integration/test_config_xml_yaml_mix/configs/users.yaml index a87a8c82819..7a28807949f 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/users.yaml +++ b/tests/integration/test_config_xml_yaml_mix/configs/users.yaml @@ -6,7 +6,6 @@ users: default: password: '' networks: - "@replace": replace ip: '::/0' profile: default diff --git a/tests/integration/test_config_yaml_full/configs/config.d/path.yaml b/tests/integration/test_config_yaml_full/configs/config.d/path.yaml index 7fd5b1a0478..de0fefd2f48 100644 --- a/tests/integration/test_config_yaml_full/configs/config.d/path.yaml +++ b/tests/integration/test_config_yaml_full/configs/config.d/path.yaml @@ -1,18 +1,6 @@ -path: - - /var/lib/clickhouse - - "@replace": replace -tmp_path: - - /var/lib/clickhouse/tmp/ - - "@replace": replace -user_files_path: - - /var/lib/clickhouse/user_files/ - - "@replace": replace -format_schema_path: - - /var/lib/clickhouse/format_schemas/ - - "@replace": replace -access_control_path: - - /var/lib/clickhouse/access/ - - "@replace": replace -top_level_domains_path: - - /var/lib/clickhouse/top_level_domains/ - - "@replace": replace +path: /var/lib/clickhouse +tmp_path: /var/lib/clickhouse/tmp/ +user_files_path: /var/lib/clickhouse/user_files/ +format_schema_path: /var/lib/clickhouse/format_schemas/ +access_control_path: /var/lib/clickhouse/access/ +top_level_domains_path: /var/lib/clickhouse/top_level_domains/ diff --git a/tests/integration/test_config_yaml_full/configs/users.yaml b/tests/integration/test_config_yaml_full/configs/users.yaml index a87a8c82819..7a28807949f 100644 --- a/tests/integration/test_config_yaml_full/configs/users.yaml +++ b/tests/integration/test_config_yaml_full/configs/users.yaml @@ -6,7 +6,6 @@ users: default: password: '' networks: - "@replace": replace ip: '::/0' profile: default diff --git a/tests/integration/test_config_yaml_main/configs/users.yaml b/tests/integration/test_config_yaml_main/configs/users.yaml index a87a8c82819..7a28807949f 100644 --- a/tests/integration/test_config_yaml_main/configs/users.yaml +++ b/tests/integration/test_config_yaml_main/configs/users.yaml @@ -6,7 +6,6 @@ users: default: password: '' networks: - "@replace": replace ip: '::/0' profile: default From 7032a1b267cb4d25fdc8325100b8443124a74fd5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 6 Sep 2022 11:14:55 -0400 Subject: [PATCH 412/582] move title to frontmatter --- docs/en/operations/backup.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 2faa23908e4..d26d8f27820 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -2,10 +2,9 @@ slug: /en/operations/backup sidebar_position: 49 sidebar_label: Data backup and restore +title: Data backup and restore --- -# Data backup and restore - While [replication](../engines/table-engines/mergetree-family/replication.md) provides protection from hardware failures, it does not protect against human errors: accidental deletion of data, deletion of the wrong table or a table on the wrong cluster, and software bugs that result in incorrect data processing or data corruption. In many cases mistakes like these will affect all replicas. ClickHouse has built-in safeguards to prevent some types of mistakes — for example, by default [you can’t just drop tables with a MergeTree-like engine containing more than 50 Gb of data](server-configuration-parameters/settings.md#max-table-size-to-drop). However, these safeguards do not cover all possible cases and can be circumvented. In order to effectively mitigate possible human errors, you should carefully prepare a strategy for backing up and restoring your data **in advance**. From cc1bd3ac362151a2e8f57025440d12b29efa5c23 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Sep 2022 16:15:50 +0000 Subject: [PATCH 413/582] fix: disable vectorscan when building w/o SSE >=3 --- contrib/vectorscan-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index bc17105be99..f9f46d9a8cf 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ # We use vectorscan, a portable and API/ABI-compatible drop-in replacement for hyperscan. -if (ARCH_AMD64) +if (ARCH_AMD64 AND NOT NO_SSE3_OR_HIGHER) option (ENABLE_VECTORSCAN "Enable vectorscan library" ${ENABLE_LIBRARIES}) endif() From ceed9f418bb22319308bac2f5fa134e249008390 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Sep 2022 18:22:44 +0200 Subject: [PATCH 414/582] Return better errors handling --- src/IO/ReadBufferFromS3.cpp | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 38b59dae186..e8f2b555bd4 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -130,14 +130,24 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); break; } - catch (const S3Exception & e) + catch (const Exception & e) { watch.stop(); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1); - /// It doesn't make sense to retry Access Denied or No Such Key - if (!e.isRetryableError()) + if (const auto * s3_exception = dynamic_cast(&e)) + { + /// It doesn't make sense to retry Access Denied or No Such Key + if (!s3_exception->isRetryableError()) + { + tryLogCurrentException(log); + throw; + } + } + + /// It doesn't make sense to retry allocator errors + if (e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY) { tryLogCurrentException(log); throw; From 09e97a638152b9f267d916067ce8a42a48ed33dc Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Sep 2022 18:38:34 +0200 Subject: [PATCH 415/582] Fix style --- src/IO/ReadBufferFromS3.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index e8f2b555bd4..7e02addd21c 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -34,6 +34,7 @@ namespace ErrorCodes extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; extern const int LOGICAL_ERROR; + extern const int CANNOT_ALLOCATE_MEMORY; } From b778b9f37f2e484678142411f1ca22234ffd9b6f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Sep 2022 19:25:58 +0200 Subject: [PATCH 416/582] Improve logging better --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 9 ++++++ src/Storages/MergeTree/MergeTreeData.cpp | 7 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 28 +++++++++++++++++-- 3 files changed, 39 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d89e3637a6f..1bc73c82dbe 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1409,7 +1409,10 @@ std::pair IMergeTreeDataPart::canRemovePart() const { /// NOTE: It's needed for zero-copy replication if (force_keep_shared_data) + { + LOG_DEBUG(storage.log, "Blobs for part {} cannot be removed because it's forced to be keeped", name); return std::make_pair(false, NameSet{}); + } return storage.unlockSharedData(*this); } @@ -1433,6 +1436,12 @@ void IMergeTreeDataPart::remove() const auto [can_remove, files_not_to_remove] = canRemovePart(); + if (!can_remove) + LOG_TRACE(storage.log, "Blobs of part {} cannot be removed", name); + + if (!files_not_to_remove.empty()) + LOG_TRACE(storage.log, "Some blobs ({}) of part {} cannot be removed", fmt::join(files_not_to_remove, ", "), name); + if (!isStoredOnDisk()) return; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e6f770ce7c0..f37f14c6924 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1901,7 +1901,10 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts, bool void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_to_remove, NameSet * part_names_successed) { const auto settings = getSettings(); - if (parts_to_remove.size() > 1 && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold) + if (parts_to_remove.size() > 1 + && !settings->allow_remote_fs_zero_copy_replication + && settings->max_part_removal_threads > 1 + && parts_to_remove.size() > settings->concurrent_part_removal_threshold) { /// Parallel parts removal. size_t num_threads = std::min(settings->max_part_removal_threads, parts_to_remove.size()); @@ -1916,7 +1919,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t if (thread_group) CurrentThread::attachToIfDetached(thread_group); - LOG_DEBUG(log, "Removing part from filesystem {}", part->name); + LOG_DEBUG(log, "Removing part from filesystem {} (concurrently)", part->name); part->remove(); if (part_names_successed) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 421aa33db97..b17fea7b9f9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7538,21 +7538,39 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, std::pair StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const { - if (!part.data_part_storage || !part.isStoredOnDisk()) + auto settings = getSettings(); + if (!settings->allow_remote_fs_zero_copy_replication) return std::make_pair(true, NameSet{}); - if (!part.data_part_storage || !part.data_part_storage->supportZeroCopyReplication()) + if (!part.data_part_storage || !part.isStoredOnDisk()) + { + LOG_TRACE(log, "Part {} is not stored on disk, blobs can be removed", part.name); return std::make_pair(true, NameSet{}); + } + + if (!part.data_part_storage || !part.data_part_storage->supportZeroCopyReplication()) + { + LOG_TRACE(log, "Part {} is not stored on zero-copy replicaed disk, blobs can be removed", part.name); + return std::make_pair(true, NameSet{}); + } /// If part is temporary refcount file may be absent if (part.data_part_storage->exists(IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK)) { auto ref_count = part.data_part_storage->getRefCount(IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK); if (ref_count > 0) /// Keep part shard info for frozen backups + { + LOG_TRACE(log, "Part {} has more than zero local references ({}), blobs cannot be removed", part.name, ref_count); return std::make_pair(false, NameSet{}); + } + else + { + LOG_TRACE(log, "Part {} local references is zero, will check blobs can be removed in zookeeper", part.name); + } } else { + LOG_TRACE(log, "Part {} looks temporary, because checksums file doesn't exists, blobs can be removed", part.name); /// Temporary part with some absent file cannot be locked in shared mode return std::make_pair(true, NameSet{}); } @@ -7600,10 +7618,14 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!children.empty()) { - LOG_TRACE(logger, "Found {} ({}) zookeper locks for {}", zookeeper_part_uniq_node, children.size(), fmt::join(children, ", ")); + LOG_TRACE(logger, "Found {} ({}) zookeper locks for {}", children.size(), fmt::join(children, ", "), zookeeper_part_uniq_node); part_has_no_more_locks = false; continue; } + else + { + LOG_TRACE(logger, "No more children left for for {}, will try to remove the whole node", zookeeper_part_uniq_node); + } auto error_code = zookeeper_ptr->tryRemove(zookeeper_part_uniq_node); From 9def86779508bc4b086b57a1874825ca755e2293 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Sep 2022 17:38:51 +0000 Subject: [PATCH 417/582] fix reading of subcolumns from in-memory parts --- src/Storages/MergeTree/MergeTreeReaderInMemory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 06f5785c868..ad425a10d30 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -36,7 +36,8 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( { /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. - if (!part_in_memory->block.has(column_to_read.name) && typeid_cast(column_to_read.type.get())) + if (typeid_cast(column_to_read.type.get()) + && !tryGetColumnFromBlock(part_in_memory->block, column_to_read)) { if (auto offsets_position = findColumnForOffsets(column_to_read)) { From d054ffd1109ab93ec366a64558c7c983cd23a11e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Sep 2022 18:23:19 +0000 Subject: [PATCH 418/582] fix: don't force-inline SSE3 code into generic code Force-inlining code compiled for SSE3 into "generic" (non-platform-specific) code works for standard x86 builds where everything is compiled with SSE 4.2 (and smaller). It no longer works if we compile everything only with SSE 2. --- src/Functions/GatherUtils/sliceHasImplAnyAll.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/GatherUtils/sliceHasImplAnyAll.h b/src/Functions/GatherUtils/sliceHasImplAnyAll.h index 21c80b742fd..3ca8c6be9a7 100644 --- a/src/Functions/GatherUtils/sliceHasImplAnyAll.h +++ b/src/Functions/GatherUtils/sliceHasImplAnyAll.h @@ -419,7 +419,7 @@ DECLARE_SSE42_SPECIFIC_CODE ( // SSE4.2 Int64, UInt64 specialization template requires (std::is_same_v || std::is_same_v) -inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64( +inline bool sliceHasImplAnyAllImplInt64( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, @@ -495,7 +495,7 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64( // SSE4.2 Int32, UInt32 specialization template requires (std::is_same_v || std::is_same_v) -inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32( +inline bool sliceHasImplAnyAllImplInt32( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, @@ -580,7 +580,7 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32( // SSE4.2 Int16, UInt16 specialization template requires (std::is_same_v || std::is_same_v) -inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16( +inline bool sliceHasImplAnyAllImplInt16( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, @@ -682,7 +682,7 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16( // SSE2 Int8, UInt8 specialization template requires (std::is_same_v || std::is_same_v) -inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt8( +inline bool sliceHasImplAnyAllImplInt8( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, From 0e363ca6e3b8f0823755f1a248884e03a85c5418 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Sep 2022 21:15:19 +0000 Subject: [PATCH 419/582] chore: regenerate vectorscan assets with ragel/colm head --- contrib/vectorscan-cmake/CMakeLists.txt | 2 +- .../vectorscan-cmake/rageled_files/Parser.cpp | 16174 ++++++++++------ .../rageled_files/control_verbs.cpp | 934 +- 3 files changed, 11167 insertions(+), 5943 deletions(-) diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index bc17105be99..b19215ba6f3 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ # We use vectorscan, a portable and API/ABI-compatible drop-in replacement for hyperscan. -if (ARCH_AMD64) +if (ARCH_AMD64 OR ARCH_AARCH64) option (ENABLE_VECTORSCAN "Enable vectorscan library" ${ENABLE_LIBRARIES}) endif() diff --git a/contrib/vectorscan-cmake/rageled_files/Parser.cpp b/contrib/vectorscan-cmake/rageled_files/Parser.cpp index aebbd7ace1e..b956d8f5801 100644 --- a/contrib/vectorscan-cmake/rageled_files/Parser.cpp +++ b/contrib/vectorscan-cmake/rageled_files/Parser.cpp @@ -1,36 +1,35 @@ - -#line 1 "Parser.rl" +#line 1 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" /* - * Copyright (c) 2015-2017, Intel Corporation - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Intel Corporation nor the names of its contributors - * may be used to endorse or promote products derived from this software - * without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ +* Copyright (c) 2015-2017, Intel Corporation +* +* Redistribution and use in source and binary forms, with or without +* modification, are permitted provided that the following conditions are met: +* +* * Redistributions of source code must retain the above copyright notice, +* this list of conditions and the following disclaimer. +* * Redistributions in binary form must reproduce the above copyright +* notice, this list of conditions and the following disclaimer in the +* documentation and/or other materials provided with the distribution. +* * Neither the name of Intel Corporation nor the names of its contributors +* may be used to endorse or promote products derived from this software +* without specific prior written permission. +* +* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +* POSSIBILITY OF SUCH DAMAGE. +*/ /** \file - * \brief Parser code (generated with Ragel from Parser.rl). - */ +* \brief Parser code (generated with Ragel from Parser.rl). +*/ #include "config.h" @@ -71,5535 +70,10656 @@ using namespace std; namespace ue2 { - + #define PUSH_SEQUENCE do {\ - sequences.push_back(ExprState(currentSeq, (size_t)(ts - ptr), \ - mode)); \ - } while(0) + sequences.push_back(ExprState(currentSeq, (size_t)(ts - ptr), \ + mode)); \ + } while(0) #define POP_SEQUENCE do {\ - currentSeq = sequences.back().seq; \ - mode = sequences.back().mode; \ - sequences.pop_back(); \ - } while(0) - -namespace { - -/** \brief Structure representing current state as we're parsing (current - * sequence, current options). Stored in the 'sequences' vector. */ -struct ExprState { - ExprState(ComponentSequence *seq_in, size_t offset, - const ParseMode &mode_in) : - seq(seq_in), seqOffset(offset), mode(mode_in) {} - - ComponentSequence *seq; //!< current sequence - size_t seqOffset; //!< offset seq was entered, for error reporting - ParseMode mode; //!< current mode flags -}; - -} // namespace - -static -unsigned parseAsDecimal(unsigned oct) { - // The input was parsed as octal, but should have been parsed as decimal. - // Deconstruct the octal number and reconstruct into decimal - unsigned ret = 0; - unsigned multiplier = 1; - while (oct) { - ret += (oct & 0x7) * multiplier; - oct >>= 3; - multiplier *= 10; - } - return ret; -} - -/** \brief Maximum value for a positive integer. We use INT_MAX, as that's what - * PCRE uses. */ -static constexpr u32 MAX_NUMBER = INT_MAX; - -static -void pushDec(u32 *acc, char raw_digit) { - assert(raw_digit >= '0' && raw_digit <= '9'); - u32 digit_val = raw_digit - '0'; - - // Ensure that we don't overflow. - u64a val = ((u64a)*acc * 10) + digit_val; - if (val > MAX_NUMBER) { - throw LocatedParseError("Number is too big"); - } - - *acc = verify_u32(val); -} - -static -void pushOct(u32 *acc, char raw_digit) { - assert(raw_digit >= '0' && raw_digit <= '7'); - u32 digit_val = raw_digit - '0'; - - // Ensure that we don't overflow. - u64a val = ((u64a)*acc * 8) + digit_val; - if (val > MAX_NUMBER) { - throw LocatedParseError("Number is too big"); - } - - *acc = verify_u32(val); -} - -static -void throwInvalidRepeat(void) { - throw LocatedParseError("Invalid repeat"); -} - -static -void throwInvalidUtf8(void) { - throw ParseError("Expression is not valid UTF-8."); -} - -/** - * Adds the given child component to the parent sequence, returning a pointer - * to the new (child) "current sequence". - */ -static -ComponentSequence *enterSequence(ComponentSequence *parent, - unique_ptr child) { - assert(parent); - assert(child); - - ComponentSequence *seq = child.get(); - parent->addComponent(move(child)); - return seq; -} - -static -void addLiteral(ComponentSequence *currentSeq, char c, const ParseMode &mode) { - if (mode.utf8 && mode.caseless) { - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - assert(cc); - cc->add(c); - cc->finalize(); - currentSeq->addComponent(move(cc)); - } else { - currentSeq->addComponent(getLiteralComponentClass(c, mode.caseless)); - } -} - -static -void addEscaped(ComponentSequence *currentSeq, unichar accum, - const ParseMode &mode, const char *err_msg) { - if (mode.utf8) { - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - assert(cc); - cc->add(accum); - cc->finalize(); - currentSeq->addComponent(move(cc)); - } else { - if (accum > 255) { - throw LocatedParseError(err_msg); - } - addLiteral(currentSeq, (char)accum, mode); - } -} - -static -void addEscapedOctal(ComponentSequence *currentSeq, unichar accum, - const ParseMode &mode) { - addEscaped(currentSeq, accum, mode, "Octal value is greater than \\377"); -} - -static -void addEscapedHex(ComponentSequence *currentSeq, unichar accum, - const ParseMode &mode) { - addEscaped(currentSeq, accum, mode, - "Hexadecimal value is greater than \\xFF"); -} - + currentSeq = sequences.back().seq; \ + mode = sequences.back().mode; \ + sequences.pop_back(); \ + } while(0) + + namespace { + + /** \brief Structure representing current state as we're parsing (current + * sequence, current options). Stored in the 'sequences' vector. */ + struct ExprState { + ExprState(ComponentSequence *seq_in, size_t offset, + const ParseMode &mode_in) : + seq(seq_in), seqOffset(offset), mode(mode_in) {} + + ComponentSequence *seq; //!< current sequence + size_t seqOffset; //!< offset seq was entered, for error reporting + ParseMode mode; //!< current mode flags + }; + + } // namespace + + static + unsigned parseAsDecimal(unsigned oct) { + // The input was parsed as octal, but should have been parsed as decimal. + // Deconstruct the octal number and reconstruct into decimal + unsigned ret = 0; + unsigned multiplier = 1; + while (oct) { + ret += (oct & 0x7) * multiplier; + oct >>= 3; + multiplier *= 10; + } + return ret; + } + + /** \brief Maximum value for a positive integer. We use INT_MAX, as that's what + * PCRE uses. */ + static constexpr u32 MAX_NUMBER = INT_MAX; + + static + void pushDec(u32 *acc, char raw_digit) { + assert(raw_digit >= '0' && raw_digit <= '9'); + u32 digit_val = raw_digit - '0'; + + // Ensure that we don't overflow. + u64a val = ((u64a)*acc * 10) + digit_val; + if (val > MAX_NUMBER) { + throw LocatedParseError("Number is too big"); + } + + *acc = verify_u32(val); + } + + static + void pushOct(u32 *acc, char raw_digit) { + assert(raw_digit >= '0' && raw_digit <= '7'); + u32 digit_val = raw_digit - '0'; + + // Ensure that we don't overflow. + u64a val = ((u64a)*acc * 8) + digit_val; + if (val > MAX_NUMBER) { + throw LocatedParseError("Number is too big"); + } + + *acc = verify_u32(val); + } + + static + void throwInvalidRepeat(void) { + throw LocatedParseError("Invalid repeat"); + } + + static + void throwInvalidUtf8(void) { + throw ParseError("Expression is not valid UTF-8."); + } + + /** + * Adds the given child component to the parent sequence, returning a pointer + * to the new (child) "current sequence". + */ + static + ComponentSequence *enterSequence(ComponentSequence *parent, + unique_ptr child) { + assert(parent); + assert(child); + + ComponentSequence *seq = child.get(); + parent->addComponent(move(child)); + return seq; + } + + static + void addLiteral(ComponentSequence *currentSeq, char c, const ParseMode &mode) { + if (mode.utf8 && mode.caseless) { + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + assert(cc); + cc->add(c); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } else { + currentSeq->addComponent(getLiteralComponentClass(c, mode.caseless)); + } + } + + static + void addEscaped(ComponentSequence *currentSeq, unichar accum, + const ParseMode &mode, const char *err_msg) { + if (mode.utf8) { + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + assert(cc); + cc->add(accum); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } else { + if (accum > 255) { + throw LocatedParseError(err_msg); + } + addLiteral(currentSeq, (char)accum, mode); + } + } + + static + void addEscapedOctal(ComponentSequence *currentSeq, unichar accum, + const ParseMode &mode) { + addEscaped(currentSeq, accum, mode, "Octal value is greater than \\377"); + } + + static + void addEscapedHex(ComponentSequence *currentSeq, unichar accum, + const ParseMode &mode) { + addEscaped(currentSeq, accum, mode, + "Hexadecimal value is greater than \\xFF"); + } + #define SLASH_C_ERROR "\\c must be followed by an ASCII character" + + static + u8 decodeCtrl(char raw) { + if (raw & 0x80) { + throw LocatedParseError(SLASH_C_ERROR); + } + return mytoupper(raw) ^ 0x40; + } + + static + unichar readUtf8CodePoint2c(const char *s) { + auto *ts = (const u8 *)s; + assert(ts[0] >= 0xc0 && ts[0] < 0xe0); + assert(ts[1] >= 0x80 && ts[1] < 0xc0); + unichar val = ts[0] & 0x1f; + val <<= 6; + val |= ts[1] & 0x3f; + DEBUG_PRINTF("utf8 %02hhx %02hhx ->\\x{%x}\n", ts[0], + ts[1], val); + return val; + } + + static + unichar readUtf8CodePoint3c(const char *s) { + auto *ts = (const u8 *)s; + assert(ts[0] >= 0xe0 && ts[0] < 0xf0); + assert(ts[1] >= 0x80 && ts[1] < 0xc0); + assert(ts[2] >= 0x80 && ts[2] < 0xc0); + unichar val = ts[0] & 0x0f; + val <<= 6; + val |= ts[1] & 0x3f; + val <<= 6; + val |= ts[2] & 0x3f; + DEBUG_PRINTF("utf8 %02hhx %02hhx %02hhx ->\\x{%x}\n", ts[0], + ts[1], ts[2], val); + return val; + } + + static + unichar readUtf8CodePoint4c(const char *s) { + auto *ts = (const u8 *)s; + assert(ts[0] >= 0xf0 && ts[0] < 0xf8); + assert(ts[1] >= 0x80 && ts[1] < 0xc0); + assert(ts[2] >= 0x80 && ts[2] < 0xc0); + assert(ts[3] >= 0x80 && ts[3] < 0xc0); + unichar val = ts[0] & 0x07; + val <<= 6; + val |= ts[1] & 0x3f; + val <<= 6; + val |= ts[2] & 0x3f; + val <<= 6; + val |= ts[3] & 0x3f; + DEBUG_PRINTF("utf8 %02hhx %02hhx %02hhx %02hhx ->\\x{%x}\n", ts[0], + ts[1], ts[2], ts[3], val); + return val; + } + + +#line 1909 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" -static -u8 decodeCtrl(char raw) { - if (raw & 0x80) { - throw LocatedParseError(SLASH_C_ERROR); - } - return mytoupper(raw) ^ 0x40; -} + -static -unichar readUtf8CodePoint2c(const char *s) { - auto *ts = (const u8 *)s; - assert(ts[0] >= 0xc0 && ts[0] < 0xe0); - assert(ts[1] >= 0x80 && ts[1] < 0xc0); - unichar val = ts[0] & 0x1f; - val <<= 6; - val |= ts[1] & 0x3f; - DEBUG_PRINTF("utf8 %02hhx %02hhx ->\\x{%x}\n", ts[0], - ts[1], val); - return val; -} - -static -unichar readUtf8CodePoint3c(const char *s) { - auto *ts = (const u8 *)s; - assert(ts[0] >= 0xe0 && ts[0] < 0xf0); - assert(ts[1] >= 0x80 && ts[1] < 0xc0); - assert(ts[2] >= 0x80 && ts[2] < 0xc0); - unichar val = ts[0] & 0x0f; - val <<= 6; - val |= ts[1] & 0x3f; - val <<= 6; - val |= ts[2] & 0x3f; - DEBUG_PRINTF("utf8 %02hhx %02hhx %02hhx ->\\x{%x}\n", ts[0], - ts[1], ts[2], val); - return val; -} - -static -unichar readUtf8CodePoint4c(const char *s) { - auto *ts = (const u8 *)s; - assert(ts[0] >= 0xf0 && ts[0] < 0xf8); - assert(ts[1] >= 0x80 && ts[1] < 0xc0); - assert(ts[2] >= 0x80 && ts[2] < 0xc0); - assert(ts[3] >= 0x80 && ts[3] < 0xc0); - unichar val = ts[0] & 0x07; - val <<= 6; - val |= ts[1] & 0x3f; - val <<= 6; - val |= ts[2] & 0x3f; - val <<= 6; - val |= ts[3] & 0x3f; - DEBUG_PRINTF("utf8 %02hhx %02hhx %02hhx %02hhx ->\\x{%x}\n", ts[0], - ts[1], ts[2], ts[3], val); - return val; -} - - -#line 1909 "Parser.rl" - - - -#line 281 "Parser.cpp" +#line 277 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" static const short _regex_actions[] = { - 0, 1, 0, 1, 1, 1, 2, 1, - 3, 1, 4, 1, 7, 1, 8, 1, - 9, 1, 10, 1, 11, 1, 12, 1, - 13, 1, 15, 1, 16, 1, 17, 1, - 18, 1, 19, 1, 20, 1, 21, 1, - 22, 1, 23, 1, 24, 1, 25, 1, - 26, 1, 27, 1, 28, 1, 29, 1, - 30, 1, 31, 1, 32, 1, 33, 1, - 34, 1, 35, 1, 36, 1, 37, 1, - 38, 1, 39, 1, 40, 1, 41, 1, - 42, 1, 43, 1, 44, 1, 45, 1, - 46, 1, 47, 1, 48, 1, 49, 1, - 50, 1, 51, 1, 52, 1, 53, 1, - 54, 1, 55, 1, 56, 1, 57, 1, - 58, 1, 59, 1, 60, 1, 61, 1, - 62, 1, 63, 1, 64, 1, 65, 1, - 66, 1, 67, 1, 68, 1, 69, 1, - 70, 1, 71, 1, 72, 1, 73, 1, - 74, 1, 75, 1, 76, 1, 77, 1, - 78, 1, 79, 1, 80, 1, 81, 1, - 82, 1, 83, 1, 84, 1, 85, 1, - 86, 1, 87, 1, 88, 1, 89, 1, - 90, 1, 91, 1, 92, 1, 93, 1, - 94, 1, 95, 1, 96, 1, 97, 1, - 98, 1, 99, 1, 100, 1, 101, 1, - 102, 1, 103, 1, 104, 1, 105, 1, - 106, 1, 107, 1, 108, 1, 109, 1, - 110, 1, 111, 1, 112, 1, 113, 1, - 114, 1, 115, 1, 116, 1, 117, 1, - 118, 1, 119, 1, 120, 1, 121, 1, - 122, 1, 123, 1, 124, 1, 125, 1, - 126, 1, 127, 1, 128, 1, 129, 1, - 130, 1, 131, 1, 132, 1, 133, 1, - 134, 1, 135, 1, 136, 1, 137, 1, - 138, 1, 139, 1, 140, 1, 141, 1, - 142, 1, 143, 1, 144, 1, 145, 1, - 146, 1, 147, 1, 148, 1, 149, 1, - 150, 1, 151, 1, 152, 1, 153, 1, - 154, 1, 155, 1, 156, 1, 157, 1, - 158, 1, 159, 1, 160, 1, 161, 1, - 162, 1, 163, 1, 164, 1, 165, 1, - 166, 1, 167, 1, 168, 1, 169, 1, - 170, 1, 171, 1, 172, 1, 173, 1, - 174, 1, 175, 1, 176, 1, 177, 1, - 178, 1, 179, 1, 180, 1, 181, 1, - 182, 1, 183, 1, 184, 1, 185, 1, - 186, 1, 187, 1, 188, 1, 189, 1, - 190, 1, 191, 1, 192, 1, 193, 1, - 194, 1, 195, 1, 196, 1, 197, 1, - 198, 1, 199, 1, 200, 1, 201, 1, - 202, 1, 203, 1, 204, 1, 205, 1, - 206, 1, 207, 1, 208, 1, 209, 1, - 210, 1, 211, 1, 212, 1, 213, 1, - 214, 1, 215, 1, 216, 1, 217, 1, - 218, 1, 219, 1, 220, 1, 221, 1, - 222, 1, 223, 1, 224, 1, 225, 1, - 226, 1, 227, 1, 228, 1, 229, 1, - 230, 1, 231, 1, 232, 1, 233, 1, - 234, 1, 235, 1, 236, 1, 237, 1, - 240, 1, 242, 1, 243, 1, 244, 1, - 245, 1, 246, 1, 247, 1, 248, 1, - 249, 1, 250, 1, 251, 1, 252, 1, - 253, 1, 254, 1, 255, 1, 256, 1, - 257, 1, 258, 1, 259, 1, 260, 1, - 261, 1, 262, 1, 263, 1, 264, 1, - 265, 1, 266, 1, 267, 1, 268, 1, - 269, 1, 270, 1, 271, 1, 272, 1, - 273, 1, 274, 1, 275, 1, 276, 1, - 277, 1, 278, 1, 279, 1, 280, 1, - 281, 1, 282, 1, 283, 1, 284, 1, - 285, 1, 286, 1, 287, 1, 288, 1, - 289, 1, 290, 1, 291, 1, 292, 1, - 293, 1, 294, 1, 295, 1, 296, 1, - 297, 1, 298, 1, 299, 1, 300, 1, - 301, 1, 302, 1, 303, 1, 307, 1, - 308, 1, 309, 1, 310, 1, 311, 1, - 312, 1, 313, 1, 314, 1, 315, 1, - 316, 1, 317, 1, 318, 1, 319, 1, - 320, 1, 321, 1, 322, 1, 323, 1, - 324, 1, 325, 1, 326, 1, 327, 1, - 328, 1, 329, 1, 330, 1, 331, 1, - 332, 1, 333, 1, 334, 1, 335, 1, - 336, 1, 337, 1, 338, 1, 342, 1, - 343, 1, 344, 1, 345, 1, 346, 1, - 347, 1, 348, 1, 349, 1, 350, 1, - 352, 1, 353, 1, 354, 1, 355, 1, - 356, 1, 357, 1, 358, 1, 359, 1, - 360, 1, 361, 1, 362, 1, 363, 1, - 364, 1, 365, 1, 366, 1, 367, 1, - 368, 1, 369, 1, 370, 1, 371, 1, - 372, 1, 373, 1, 374, 1, 375, 1, - 376, 1, 377, 1, 378, 1, 379, 1, - 380, 1, 381, 1, 382, 1, 383, 1, - 384, 1, 385, 1, 386, 1, 387, 1, - 388, 1, 389, 1, 390, 1, 391, 1, - 392, 1, 393, 1, 394, 1, 395, 1, - 396, 1, 397, 1, 398, 1, 399, 1, - 400, 1, 401, 1, 402, 1, 403, 1, - 404, 1, 405, 1, 406, 1, 407, 1, - 408, 1, 409, 1, 410, 1, 411, 1, - 412, 1, 413, 1, 414, 1, 415, 1, - 416, 1, 417, 1, 418, 1, 419, 1, - 420, 1, 421, 1, 422, 1, 423, 1, - 424, 1, 425, 1, 426, 1, 427, 1, - 428, 1, 429, 1, 430, 1, 431, 1, - 432, 1, 433, 1, 434, 1, 435, 1, - 436, 2, 3, 0, 2, 4, 5, 2, - 5, 1, 2, 9, 10, 2, 9, 238, - 2, 9, 239, 2, 9, 339, 2, 10, - 1, 2, 10, 340, 2, 10, 341, 2, - 11, 241, 2, 11, 351, 2, 12, 241, - 2, 12, 351, 2, 13, 241, 2, 13, - 351, 2, 14, 375, 2, 14, 376, 2, - 25, 0, 2, 25, 3, 2, 25, 6, - 2, 25, 14, 3, 25, 5, 306, 3, - 25, 10, 305, 3, 25, 14, 15, 4, - 25, 9, 304, 10 -}; + 0, 1, 0, 1, 1, 1, 2, 1, + 3, 1, 4, 1, 7, 1, 8, 1, + 9, 1, 10, 1, 11, 1, 12, 1, + 13, 1, 15, 1, 16, 1, 17, 1, + 18, 1, 19, 1, 20, 1, 21, 1, + 22, 1, 23, 1, 24, 1, 25, 1, + 26, 1, 27, 1, 28, 1, 29, 1, + 30, 1, 31, 1, 32, 1, 33, 1, + 34, 1, 35, 1, 36, 1, 37, 1, + 38, 1, 39, 1, 40, 1, 41, 1, + 42, 1, 43, 1, 44, 1, 45, 1, + 46, 1, 47, 1, 48, 1, 49, 1, + 50, 1, 51, 1, 52, 1, 53, 1, + 54, 1, 55, 1, 56, 1, 57, 1, + 58, 1, 59, 1, 60, 1, 61, 1, + 62, 1, 63, 1, 64, 1, 65, 1, + 66, 1, 67, 1, 68, 1, 69, 1, + 70, 1, 71, 1, 72, 1, 73, 1, + 74, 1, 75, 1, 76, 1, 77, 1, + 78, 1, 79, 1, 80, 1, 81, 1, + 82, 1, 83, 1, 84, 1, 85, 1, + 86, 1, 87, 1, 88, 1, 89, 1, + 90, 1, 91, 1, 92, 1, 93, 1, + 94, 1, 95, 1, 96, 1, 97, 1, + 98, 1, 99, 1, 100, 1, 101, 1, + 102, 1, 103, 1, 104, 1, 105, 1, + 106, 1, 107, 1, 108, 1, 109, 1, + 110, 1, 111, 1, 112, 1, 113, 1, + 114, 1, 115, 1, 116, 1, 117, 1, + 118, 1, 119, 1, 120, 1, 121, 1, + 122, 1, 123, 1, 124, 1, 125, 1, + 126, 1, 127, 1, 128, 1, 129, 1, + 130, 1, 131, 1, 132, 1, 133, 1, + 134, 1, 135, 1, 136, 1, 137, 1, + 138, 1, 139, 1, 140, 1, 141, 1, + 142, 1, 143, 1, 144, 1, 145, 1, + 146, 1, 147, 1, 148, 1, 149, 1, + 150, 1, 151, 1, 152, 1, 153, 1, + 154, 1, 155, 1, 156, 1, 157, 1, + 158, 1, 159, 1, 160, 1, 161, 1, + 162, 1, 163, 1, 164, 1, 165, 1, + 166, 1, 167, 1, 168, 1, 169, 1, + 170, 1, 171, 1, 172, 1, 173, 1, + 174, 1, 175, 1, 176, 1, 177, 1, + 178, 1, 179, 1, 180, 1, 181, 1, + 182, 1, 183, 1, 184, 1, 185, 1, + 186, 1, 187, 1, 188, 1, 189, 1, + 190, 1, 191, 1, 192, 1, 193, 1, + 194, 1, 195, 1, 196, 1, 197, 1, + 198, 1, 199, 1, 200, 1, 201, 1, + 202, 1, 203, 1, 204, 1, 205, 1, + 206, 1, 207, 1, 208, 1, 209, 1, + 210, 1, 211, 1, 212, 1, 213, 1, + 214, 1, 215, 1, 216, 1, 217, 1, + 218, 1, 219, 1, 220, 1, 221, 1, + 222, 1, 223, 1, 224, 1, 225, 1, + 226, 1, 227, 1, 228, 1, 229, 1, + 230, 1, 231, 1, 232, 1, 233, 1, + 234, 1, 235, 1, 236, 1, 237, 1, + 240, 1, 242, 1, 243, 1, 244, 1, + 245, 1, 246, 1, 247, 1, 248, 1, + 249, 1, 250, 1, 251, 1, 252, 1, + 253, 1, 254, 1, 255, 1, 256, 1, + 257, 1, 258, 1, 259, 1, 260, 1, + 261, 1, 262, 1, 263, 1, 264, 1, + 265, 1, 266, 1, 267, 1, 268, 1, + 269, 1, 270, 1, 271, 1, 272, 1, + 273, 1, 274, 1, 275, 1, 276, 1, + 277, 1, 278, 1, 279, 1, 280, 1, + 281, 1, 282, 1, 283, 1, 284, 1, + 285, 1, 286, 1, 287, 1, 288, 1, + 289, 1, 290, 1, 291, 1, 292, 1, + 293, 1, 294, 1, 295, 1, 296, 1, + 297, 1, 298, 1, 299, 1, 300, 1, + 301, 1, 302, 1, 303, 1, 307, 1, + 308, 1, 309, 1, 310, 1, 311, 1, + 312, 1, 313, 1, 314, 1, 315, 1, + 316, 1, 317, 1, 318, 1, 319, 1, + 320, 1, 321, 1, 322, 1, 323, 1, + 324, 1, 325, 1, 326, 1, 327, 1, + 328, 1, 329, 1, 330, 1, 331, 1, + 332, 1, 333, 1, 334, 1, 335, 1, + 336, 1, 337, 1, 338, 1, 342, 1, + 343, 1, 344, 1, 345, 1, 346, 1, + 347, 1, 348, 1, 349, 1, 350, 1, + 352, 1, 353, 1, 354, 1, 355, 1, + 356, 1, 357, 1, 358, 1, 359, 1, + 360, 1, 361, 1, 362, 1, 363, 1, + 364, 1, 365, 1, 366, 1, 367, 1, + 368, 1, 369, 1, 370, 1, 371, 1, + 372, 1, 373, 1, 374, 1, 375, 1, + 376, 1, 377, 1, 378, 1, 379, 1, + 380, 1, 381, 1, 382, 1, 383, 1, + 384, 1, 385, 1, 386, 1, 387, 1, + 388, 1, 389, 1, 390, 1, 391, 1, + 392, 1, 393, 1, 394, 1, 395, 1, + 396, 1, 397, 1, 398, 1, 399, 1, + 400, 1, 401, 1, 402, 1, 403, 1, + 404, 1, 405, 1, 406, 1, 407, 1, + 408, 1, 409, 1, 410, 1, 411, 1, + 412, 1, 413, 1, 414, 1, 415, 1, + 416, 1, 417, 1, 418, 1, 419, 1, + 420, 1, 421, 1, 422, 1, 423, 1, + 424, 1, 425, 1, 426, 1, 427, 1, + 428, 1, 429, 1, 430, 1, 431, 1, + 432, 1, 433, 1, 434, 1, 435, 1, + 436, 2, 3, 0, 2, 4, 5, 2, + 5, 1, 2, 9, 10, 2, 9, 238, + 2, 9, 239, 2, 9, 339, 2, 10, + 1, 2, 10, 340, 2, 10, 341, 2, + 11, 241, 2, 11, 351, 2, 12, 241, + 2, 12, 351, 2, 13, 241, 2, 13, + 351, 2, 14, 375, 2, 14, 376, 2, + 25, 0, 2, 25, 3, 2, 25, 6, + 2, 25, 14, 3, 25, 5, 306, 3, + 25, 10, 305, 3, 25, 14, 15, 4, + 25, 9, 304, 10, 0 + }; + + static const short _regex_key_offsets[] = { + 0, 0, 1, 23, 31, 39, 46, 54, + 55, 63, 71, 79, 86, 94, 97, 99, + 108, 115, 123, 131, 134, 140, 148, 151, + 158, 165, 173, 180, 184, 191, 194, 197, + 199, 202, 205, 207, 210, 213, 215, 216, + 218, 219, 227, 229, 232, 235, 236, 244, + 252, 260, 268, 275, 283, 290, 298, 305, + 313, 315, 318, 325, 329, 332, 335, 337, + 339, 341, 342, 344, 345, 347, 349, 350, + 351, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 369, + 370, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 384, 385, + 386, 387, 388, 389, 390, 392, 393, 394, + 395, 396, 397, 399, 400, 401, 402, 403, + 404, 405, 406, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 429, + 430, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 443, 444, 445, + 446, 447, 448, 450, 451, 452, 453, 454, + 455, 456, 457, 458, 459, 461, 462, 463, + 464, 465, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, + 512, 513, 514, 515, 516, 517, 519, 520, + 521, 522, 523, 524, 525, 526, 527, 528, + 529, 530, 531, 532, 533, 534, 535, 536, + 537, 538, 539, 540, 541, 542, 543, 544, + 545, 546, 547, 548, 549, 550, 551, 552, + 553, 554, 555, 556, 557, 558, 559, 561, + 562, 563, 564, 565, 566, 567, 568, 569, + 570, 571, 572, 573, 574, 575, 576, 577, + 578, 579, 580, 582, 583, 584, 585, 586, + 587, 588, 589, 590, 591, 592, 593, 594, + 595, 596, 597, 601, 602, 603, 604, 605, + 606, 607, 608, 609, 610, 611, 612, 613, + 614, 615, 616, 617, 618, 620, 621, 622, + 623, 624, 625, 626, 627, 628, 629, 631, + 632, 633, 634, 635, 636, 637, 640, 641, + 642, 643, 644, 645, 646, 647, 648, 650, + 651, 652, 653, 654, 655, 656, 658, 659, + 660, 661, 662, 663, 664, 665, 666, 667, + 668, 669, 670, 671, 672, 673, 674, 675, + 676, 677, 678, 679, 680, 681, 682, 683, + 684, 685, 686, 687, 688, 689, 690, 691, + 692, 693, 694, 695, 696, 697, 698, 699, + 700, 701, 702, 704, 705, 706, 707, 708, + 709, 710, 714, 715, 716, 717, 718, 719, + 720, 721, 722, 723, 724, 725, 726, 727, + 728, 729, 730, 731, 732, 733, 734, 735, + 736, 737, 738, 739, 740, 741, 742, 743, + 744, 745, 746, 747, 748, 749, 750, 752, + 753, 754, 755, 756, 757, 758, 759, 760, + 761, 762, 763, 764, 765, 766, 767, 768, + 769, 770, 771, 773, 774, 775, 776, 777, + 778, 779, 780, 781, 782, 783, 784, 785, + 786, 787, 788, 789, 790, 791, 792, 793, + 794, 795, 796, 797, 798, 799, 800, 801, + 802, 803, 805, 806, 807, 808, 809, 810, + 811, 812, 813, 814, 815, 816, 817, 820, + 822, 823, 824, 825, 826, 827, 828, 829, + 830, 833, 834, 835, 836, 837, 838, 839, + 840, 841, 842, 843, 844, 845, 846, 847, + 849, 850, 851, 853, 854, 855, 856, 857, + 858, 859, 860, 861, 862, 863, 864, 865, + 866, 867, 868, 869, 870, 871, 872, 873, + 874, 875, 876, 877, 880, 883, 885, 900, + 903, 906, 908, 922, 927, 932, 936, 940, + 943, 946, 950, 954, 957, 960, 964, 968, + 972, 975, 978, 982, 986, 990, 994, 997, + 1000, 1004, 1008, 1012, 1016, 1019, 1022, 1026, + 1030, 1034, 1038, 1041, 1044, 1048, 1052, 1056, + 1060, 1063, 1066, 1070, 1074, 1078, 1082, 1085, + 1088, 1093, 1097, 1101, 1105, 1108, 1111, 1115, + 1119, 1123, 1126, 1129, 1133, 1137, 1141, 1145, + 1148, 1151, 1155, 1159, 1163, 1167, 1170, 1173, + 1177, 1181, 1185, 1188, 1191, 1195, 1199, 1203, + 1207, 1211, 1214, 1217, 1222, 1227, 1231, 1235, + 1238, 1241, 1245, 1249, 1252, 1255, 1259, 1263, + 1267, 1270, 1273, 1277, 1281, 1285, 1289, 1292, + 1295, 1299, 1303, 1307, 1311, 1314, 1317, 1321, + 1325, 1329, 1333, 1336, 1339, 1343, 1347, 1351, + 1355, 1358, 1361, 1365, 1369, 1373, 1377, 1380, + 1383, 1388, 1392, 1396, 1400, 1403, 1406, 1410, + 1414, 1418, 1421, 1424, 1428, 1432, 1436, 1440, + 1443, 1446, 1450, 1454, 1458, 1462, 1465, 1468, + 1472, 1476, 1480, 1483, 1486, 1490, 1494, 1498, + 1502, 1506, 1509, 1512, 1515, 1518, 1520, 1522, + 1525, 1532, 1534, 1536, 1538, 1540, 1542, 1544, + 1546, 1548, 1550, 1577, 1579, 1586, 1593, 1607, + 1609, 1615, 1618, 1627, 1628, 1631, 1634, 1641, + 1643, 1645, 1647, 1650, 1695, 1697, 1699, 1703, + 1707, 1709, 1710, 1710, 1716, 1718, 1720, 1722, + 1724, 1727, 1728, 1729, 1736, 1742, 1748, 1750, + 1752, 1754, 1756, 1758, 1760, 1761, 1764, 1787, + 1790, 1795, 1804, 1806, 1807, 1809, 1814, 1817, + 1819, 1821, 1822, 1824, 1834, 1840, 1841, 1846, + 1850, 1858, 1860, 1869, 1873, 1874, 1875, 1879, + 1880, 1883, 1883, 1890, 1904, 1907, 1946, 1948, + 1950, 1952, 1954, 1955, 1955, 1956, 1957, 1964, + 1970, 1976, 1978, 1980, 1982, 1985, 1987, 1998, + 1999, 2001, 2003, 2005, 2016, 2017, 2019, 2021, + 2023, 2024, 0 + }; + + static const char _regex_trans_keys[] = { + 41u, 33u, 35u, 38u, 39u, 40u, 41u, 43u, + 45u, 58u, 60u, 61u, 62u, 63u, 67u, 80u, + 105u, 109u, 115u, 120u, 123u, 48u, 57u, 41u, + 95u, 48u, 57u, 65u, 90u, 97u, 122u, 39u, + 95u, 48u, 57u, 65u, 90u, 97u, 122u, 95u, + 48u, 57u, 65u, 90u, 97u, 122u, 39u, 95u, + 48u, 57u, 65u, 90u, 97u, 122u, 41u, 41u, + 95u, 48u, 57u, 65u, 90u, 97u, 122u, 41u, + 95u, 48u, 57u, 65u, 90u, 97u, 122u, 41u, + 95u, 48u, 57u, 65u, 90u, 97u, 122u, 95u, + 48u, 57u, 65u, 90u, 97u, 122u, 62u, 95u, + 48u, 57u, 65u, 90u, 97u, 122u, 33u, 60u, + 61u, 33u, 61u, 38u, 41u, 95u, 48u, 57u, + 65u, 90u, 97u, 122u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 41u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 41u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 41u, 48u, 57u, 41u, 58u, + 105u, 109u, 115u, 120u, 62u, 95u, 48u, 57u, + 65u, 90u, 97u, 122u, 41u, 48u, 57u, 95u, + 48u, 57u, 65u, 90u, 97u, 122u, 95u, 48u, + 57u, 65u, 90u, 97u, 122u, 41u, 95u, 48u, + 57u, 65u, 90u, 97u, 122u, 95u, 48u, 57u, + 65u, 90u, 97u, 122u, 105u, 109u, 115u, 120u, + 41u, 45u, 58u, 105u, 109u, 115u, 120u, 46u, + 92u, 93u, 46u, 92u, 93u, 46u, 92u, 58u, + 92u, 93u, 58u, 92u, 93u, 58u, 92u, 61u, + 92u, 93u, 61u, 92u, 93u, 61u, 92u, 39u, + 48u, 57u, 62u, 45u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 48u, 57u, 125u, 48u, 57u, + 125u, 48u, 57u, 125u, 95u, 125u, 48u, 57u, + 65u, 90u, 97u, 122u, 95u, 125u, 48u, 57u, + 65u, 90u, 97u, 122u, 95u, 125u, 48u, 57u, + 65u, 90u, 97u, 122u, 95u, 125u, 48u, 57u, + 65u, 90u, 97u, 122u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 39u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 95u, 48u, 57u, 65u, 90u, + 97u, 122u, 62u, 95u, 48u, 57u, 65u, 90u, + 97u, 122u, 95u, 48u, 57u, 65u, 90u, 97u, + 122u, 95u, 125u, 48u, 57u, 65u, 90u, 97u, + 122u, 48u, 55u, 125u, 48u, 55u, 125u, 48u, + 57u, 65u, 70u, 97u, 102u, 44u, 125u, 48u, + 57u, 125u, 48u, 57u, 125u, 48u, 57u, 128u, + 191u, 128u, 191u, 128u, 191u, 41u, 41u, 80u, + 41u, 41u, 70u, 41u, 56u, 41u, 121u, 97u, + 109u, 98u, 105u, 99u, 101u, 110u, 105u, 97u, + 110u, 101u, 115u, 116u, 97u, 110u, 108u, 109u, + 116u, 105u, 110u, 101u, 115u, 101u, 117u, 109u, + 97u, 107u, 110u, 103u, 97u, 108u, 105u, 112u, + 111u, 109u, 111u, 102u, 111u, 97u, 104u, 105u, + 109u, 105u, 108u, 108u, 101u, 103u, 104u, 105u, + 110u, 101u, 115u, 101u, 105u, 100u, 110u, 114u, + 97u, 100u, 105u, 97u, 110u, 95u, 65u, 98u, + 111u, 114u, 105u, 103u, 105u, 110u, 97u, 108u, + 105u, 97u, 110u, 97u, 101u, 109u, 114u, 111u, + 107u, 101u, 101u, 109u, 111u, 110u, 116u, 105u, + 99u, 110u, 101u, 105u, 102u, 111u, 114u, 109u, + 112u, 114u, 114u, 105u, 111u, 116u, 105u, 108u, + 108u, 105u, 99u, 115u, 118u, 101u, 114u, 101u, + 116u, 97u, 110u, 97u, 103u, 97u, 114u, 105u, + 121u, 112u, 116u, 105u, 97u, 110u, 95u, 72u, + 105u, 101u, 114u, 111u, 103u, 108u, 121u, 112u, + 104u, 115u, 104u, 105u, 111u, 112u, 105u, 99u, + 111u, 114u, 103u, 105u, 97u, 110u, 97u, 103u, + 111u, 108u, 105u, 116u, 105u, 99u, 116u, 104u, + 105u, 99u, 101u, 101u, 107u, 106u, 114u, 97u, + 114u, 97u, 116u, 105u, 109u, 117u, 107u, 104u, + 105u, 110u, 117u, 108u, 110u, 111u, 111u, 98u, + 114u, 101u, 119u, 114u, 97u, 103u, 97u, 110u, + 97u, 112u, 101u, 114u, 105u, 97u, 108u, 95u, + 65u, 114u, 97u, 109u, 97u, 105u, 99u, 104u, + 115u, 101u, 114u, 105u, 116u, 101u, 100u, 99u, + 114u, 105u, 112u, 116u, 105u, 111u, 110u, 97u, + 108u, 95u, 80u, 97u, 104u, 114u, 108u, 97u, + 118u, 105u, 116u, 104u, 105u, 97u, 110u, 118u, + 97u, 110u, 101u, 115u, 101u, 105u, 110u, 116u, + 121u, 116u, 104u, 105u, 110u, 97u, 100u, 97u, + 97u, 107u, 97u, 110u, 97u, 97u, 104u, 95u, + 76u, 105u, 97u, 109u, 114u, 111u, 115u, 104u, + 116u, 104u, 105u, 101u, 114u, 111u, 116u, 105u, + 110u, 112u, 99u, 104u, 97u, 109u, 110u, 115u, + 98u, 117u, 101u, 97u, 114u, 95u, 66u, 117u, + 99u, 100u, 105u, 97u, 110u, 105u, 97u, 110u, + 108u, 110u, 97u, 121u, 97u, 108u, 97u, 109u, + 100u, 97u, 105u, 99u, 116u, 101u, 105u, 95u, + 77u, 97u, 121u, 101u, 107u, 110u, 103u, 111u, + 108u, 105u, 97u, 110u, 97u, 110u, 109u, 97u, + 114u, 119u, 95u, 84u, 97u, 105u, 95u, 76u, + 117u, 101u, 111u, 104u, 97u, 109u, 95u, 100u, + 67u, 104u, 105u, 107u, 105u, 95u, 73u, 80u, + 83u, 84u, 116u, 97u, 108u, 105u, 99u, 101u, + 114u, 115u, 105u, 97u, 110u, 111u, 117u, 116u, + 104u, 95u, 65u, 114u, 97u, 98u, 105u, 97u, + 110u, 117u, 114u, 107u, 105u, 99u, 105u, 121u, + 97u, 109u, 97u, 110u, 121u, 97u, 97u, 111u, + 103u, 115u, 95u, 80u, 97u, 101u, 110u, 105u, + 99u, 105u, 97u, 110u, 106u, 97u, 110u, 103u, + 110u, 105u, 99u, 109u, 117u, 97u, 114u, 105u, + 116u, 97u, 110u, 114u, 97u, 115u, 104u, 116u, + 114u, 97u, 97u, 118u, 105u, 97u, 110u, 110u, + 104u, 97u, 108u, 97u, 110u, 100u, 97u, 110u, + 101u, 115u, 101u, 108u, 114u, 111u, 116u, 105u, + 95u, 78u, 97u, 103u, 114u, 105u, 105u, 97u, + 99u, 103u, 105u, 109u, 97u, 98u, 108u, 111u, + 103u, 97u, 110u, 119u, 97u, 95u, 76u, 84u, + 86u, 101u, 104u, 97u, 109u, 105u, 101u, 116u, + 105u, 108u, 108u, 117u, 103u, 117u, 97u, 97u, + 105u, 110u, 97u, 98u, 102u, 101u, 116u, 97u, + 110u, 105u, 110u, 97u, 103u, 104u, 97u, 114u, + 105u, 116u, 105u, 99u, 105u, 110u, 115u, 112u, + 100u, 123u, 94u, 125u, 94u, 46u, 92u, 93u, + 46u, 92u, 93u, 46u, 92u, 58u, 92u, 93u, + 94u, 97u, 98u, 99u, 100u, 103u, 108u, 112u, + 115u, 117u, 119u, 120u, 58u, 92u, 93u, 58u, + 92u, 93u, 58u, 92u, 58u, 92u, 93u, 97u, + 98u, 99u, 100u, 103u, 108u, 112u, 115u, 117u, + 119u, 120u, 58u, 92u, 93u, 108u, 115u, 58u, + 92u, 93u, 110u, 112u, 58u, 92u, 93u, 117u, + 58u, 92u, 93u, 109u, 58u, 92u, 93u, 58u, + 92u, 93u, 58u, 92u, 93u, 104u, 58u, 92u, + 93u, 97u, 58u, 92u, 93u, 58u, 92u, 93u, + 58u, 92u, 93u, 99u, 58u, 92u, 93u, 105u, + 58u, 92u, 93u, 105u, 58u, 92u, 93u, 58u, + 92u, 93u, 58u, 92u, 93u, 108u, 58u, 92u, + 93u, 97u, 58u, 92u, 93u, 110u, 58u, 92u, + 93u, 107u, 58u, 92u, 93u, 58u, 92u, 93u, + 58u, 92u, 93u, 110u, 58u, 92u, 93u, 116u, + 58u, 92u, 93u, 114u, 58u, 92u, 93u, 108u, + 58u, 92u, 93u, 58u, 92u, 93u, 58u, 92u, + 93u, 105u, 58u, 92u, 93u, 103u, 58u, 92u, + 93u, 105u, 58u, 92u, 93u, 116u, 58u, 92u, + 93u, 58u, 92u, 93u, 58u, 92u, 93u, 114u, + 58u, 92u, 93u, 97u, 58u, 92u, 93u, 112u, + 58u, 92u, 93u, 104u, 58u, 92u, 93u, 58u, + 92u, 93u, 58u, 92u, 93u, 111u, 58u, 92u, + 93u, 119u, 58u, 92u, 93u, 101u, 58u, 92u, + 93u, 114u, 58u, 92u, 93u, 58u, 92u, 93u, + 58u, 92u, 93u, 114u, 117u, 58u, 92u, 93u, + 105u, 58u, 92u, 93u, 110u, 58u, 92u, 93u, + 116u, 58u, 92u, 93u, 58u, 92u, 93u, 58u, + 92u, 93u, 110u, 58u, 92u, 93u, 99u, 58u, + 92u, 93u, 116u, 58u, 92u, 93u, 58u, 92u, + 93u, 58u, 92u, 93u, 112u, 58u, 92u, 93u, + 97u, 58u, 92u, 93u, 99u, 58u, 92u, 93u, + 101u, 58u, 92u, 93u, 58u, 92u, 93u, 58u, + 92u, 93u, 112u, 58u, 92u, 93u, 112u, 58u, + 92u, 93u, 101u, 58u, 92u, 93u, 114u, 58u, + 92u, 93u, 58u, 92u, 93u, 58u, 92u, 93u, + 111u, 58u, 92u, 93u, 114u, 58u, 92u, 93u, + 100u, 58u, 92u, 93u, 58u, 92u, 93u, 58u, + 92u, 93u, 100u, 58u, 92u, 93u, 105u, 58u, + 92u, 93u, 103u, 58u, 92u, 93u, 105u, 58u, + 92u, 93u, 116u, 58u, 92u, 93u, 58u, 92u, + 93u, 58u, 92u, 93u, 108u, 115u, 58u, 92u, + 93u, 110u, 112u, 58u, 92u, 93u, 117u, 58u, + 92u, 93u, 109u, 58u, 92u, 93u, 58u, 92u, + 93u, 58u, 92u, 93u, 104u, 58u, 92u, 93u, + 97u, 58u, 92u, 93u, 58u, 92u, 93u, 58u, + 92u, 93u, 99u, 58u, 92u, 93u, 105u, 58u, + 92u, 93u, 105u, 58u, 92u, 93u, 58u, 92u, + 93u, 58u, 92u, 93u, 108u, 58u, 92u, 93u, + 97u, 58u, 92u, 93u, 110u, 58u, 92u, 93u, + 107u, 58u, 92u, 93u, 58u, 92u, 93u, 58u, + 92u, 93u, 110u, 58u, 92u, 93u, 116u, 58u, + 92u, 93u, 114u, 58u, 92u, 93u, 108u, 58u, + 92u, 93u, 58u, 92u, 93u, 58u, 92u, 93u, + 105u, 58u, 92u, 93u, 103u, 58u, 92u, 93u, + 105u, 58u, 92u, 93u, 116u, 58u, 92u, 93u, + 58u, 92u, 93u, 58u, 92u, 93u, 114u, 58u, + 92u, 93u, 97u, 58u, 92u, 93u, 112u, 58u, + 92u, 93u, 104u, 58u, 92u, 93u, 58u, 92u, + 93u, 58u, 92u, 93u, 111u, 58u, 92u, 93u, + 119u, 58u, 92u, 93u, 101u, 58u, 92u, 93u, + 114u, 58u, 92u, 93u, 58u, 92u, 93u, 58u, + 92u, 93u, 114u, 117u, 58u, 92u, 93u, 105u, + 58u, 92u, 93u, 110u, 58u, 92u, 93u, 116u, + 58u, 92u, 93u, 58u, 92u, 93u, 58u, 92u, + 93u, 110u, 58u, 92u, 93u, 99u, 58u, 92u, + 93u, 116u, 58u, 92u, 93u, 58u, 92u, 93u, + 58u, 92u, 93u, 112u, 58u, 92u, 93u, 97u, + 58u, 92u, 93u, 99u, 58u, 92u, 93u, 101u, + 58u, 92u, 93u, 58u, 92u, 93u, 58u, 92u, + 93u, 112u, 58u, 92u, 93u, 112u, 58u, 92u, + 93u, 101u, 58u, 92u, 93u, 114u, 58u, 92u, + 93u, 58u, 92u, 93u, 58u, 92u, 93u, 111u, + 58u, 92u, 93u, 114u, 58u, 92u, 93u, 100u, + 58u, 92u, 93u, 58u, 92u, 93u, 58u, 92u, + 93u, 100u, 58u, 92u, 93u, 105u, 58u, 92u, + 93u, 103u, 58u, 92u, 93u, 105u, 58u, 92u, + 93u, 116u, 58u, 92u, 93u, 58u, 92u, 93u, + 61u, 92u, 93u, 61u, 92u, 93u, 61u, 92u, + 48u, 55u, 125u, 48u, 55u, 125u, 48u, 57u, + 65u, 70u, 97u, 102u, 128u, 191u, 128u, 191u, + 128u, 191u, 128u, 191u, 128u, 191u, 128u, 191u, + 128u, 191u, 128u, 191u, 128u, 191u, 0u, 32u, + 35u, 36u, 40u, 41u, 42u, 43u, 46u, 63u, + 91u, 92u, 94u, 123u, 124u, 9u, 13u, 128u, + 191u, 192u, 223u, 224u, 239u, 240u, 247u, 248u, + 255u, 42u, 63u, 95u, 48u, 57u, 65u, 90u, + 97u, 122u, 95u, 48u, 57u, 65u, 90u, 97u, + 122u, 39u, 48u, 60u, 63u, 82u, 95u, 49u, + 55u, 56u, 57u, 65u, 90u, 97u, 122u, 48u, + 57u, 105u, 109u, 115u, 120u, 48u, 57u, 41u, + 48u, 57u, 33u, 61u, 95u, 48u, 57u, 65u, + 90u, 97u, 122u, 123u, 41u, 48u, 57u, 60u, + 61u, 62u, 41u, 45u, 58u, 105u, 109u, 115u, + 120u, 43u, 63u, 43u, 63u, 43u, 63u, 46u, + 58u, 61u, 48u, 65u, 66u, 67u, 68u, 69u, + 71u, 72u, 75u, 76u, 78u, 80u, 81u, 82u, + 83u, 85u, 86u, 87u, 88u, 90u, 97u, 98u, + 99u, 100u, 101u, 102u, 103u, 104u, 107u, 108u, + 110u, 111u, 112u, 114u, 115u, 116u, 117u, 118u, + 119u, 120u, 122u, 49u, 55u, 56u, 57u, 48u, + 55u, 48u, 55u, 48u, 55u, 56u, 57u, 48u, + 55u, 56u, 57u, 48u, 57u, 123u, 39u, 45u, + 60u, 123u, 48u, 57u, 48u, 57u, 48u, 57u, + 48u, 57u, 48u, 57u, 39u, 60u, 123u, 123u, + 123u, 123u, 48u, 57u, 65u, 70u, 97u, 102u, + 48u, 57u, 65u, 70u, 97u, 102u, 48u, 57u, + 65u, 70u, 97u, 102u, 48u, 57u, 43u, 63u, + 128u, 191u, 128u, 191u, 128u, 191u, 41u, 85u, + 41u, 41u, 67u, 84u, 65u, 66u, 67u, 68u, + 69u, 71u, 72u, 73u, 74u, 75u, 76u, 77u, + 78u, 79u, 80u, 82u, 83u, 84u, 85u, 86u, + 88u, 89u, 90u, 110u, 114u, 118u, 97u, 101u, + 111u, 114u, 117u, 97u, 99u, 102u, 104u, 110u, + 111u, 115u, 117u, 121u, 109u, 112u, 101u, 103u, + 116u, 101u, 108u, 111u, 114u, 117u, 97u, 101u, + 105u, 103u, 117u, 109u, 110u, 97u, 97u, 104u, + 38u, 97u, 101u, 105u, 108u, 109u, 111u, 116u, + 117u, 121u, 97u, 99u, 101u, 110u, 111u, 121u, + 101u, 100u, 101u, 107u, 108u, 111u, 103u, 108u, + 114u, 115u, 99u, 100u, 101u, 102u, 104u, 105u, + 111u, 115u, 101u, 117u, 97u, 99u, 104u, 105u, + 107u, 109u, 111u, 117u, 121u, 97u, 101u, 104u, + 105u, 103u, 97u, 97u, 112u, 115u, 119u, 105u, + 108u, 112u, 115u, 67u, 76u, 77u, 78u, 80u, + 83u, 90u, 45u, 91u, 92u, 93u, 128u, 191u, + 192u, 223u, 224u, 239u, 240u, 247u, 248u, 255u, + 46u, 58u, 61u, 48u, 68u, 69u, 72u, 76u, + 78u, 80u, 81u, 83u, 85u, 86u, 87u, 97u, + 98u, 99u, 100u, 101u, 102u, 103u, 104u, 108u, + 110u, 111u, 112u, 114u, 115u, 116u, 117u, 118u, + 119u, 120u, 49u, 55u, 56u, 57u, 65u, 90u, + 105u, 122u, 48u, 55u, 48u, 55u, 48u, 55u, + 48u, 55u, 123u, 123u, 123u, 123u, 48u, 57u, + 65u, 70u, 97u, 102u, 48u, 57u, 65u, 70u, + 97u, 102u, 48u, 57u, 65u, 70u, 97u, 102u, + 128u, 191u, 128u, 191u, 128u, 191u, 92u, 93u, + 94u, 69u, 81u, 92u, 128u, 191u, 192u, 223u, + 224u, 239u, 240u, 247u, 248u, 255u, 69u, 128u, + 191u, 128u, 191u, 128u, 191u, 92u, 128u, 191u, + 192u, 223u, 224u, 239u, 240u, 247u, 248u, 255u, + 69u, 128u, 191u, 128u, 191u, 128u, 191u, 41u, + 10u, 0u + }; + + static const signed char _regex_single_lengths[] = { + 0, 1, 20, 2, 2, 1, 2, 1, + 2, 2, 2, 1, 2, 3, 2, 3, + 1, 2, 2, 1, 6, 2, 1, 1, + 1, 2, 1, 4, 7, 3, 3, 2, + 3, 3, 2, 3, 3, 2, 1, 0, + 1, 2, 0, 1, 1, 1, 2, 2, + 2, 2, 1, 2, 1, 2, 1, 2, + 0, 1, 1, 2, 1, 1, 0, 0, + 0, 1, 2, 1, 2, 2, 1, 1, + 2, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 3, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 2, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 4, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 3, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 2, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 4, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 2, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 3, 2, + 1, 1, 1, 1, 1, 1, 1, 1, + 3, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 2, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 3, 3, 2, 15, 3, + 3, 2, 14, 5, 5, 4, 4, 3, + 3, 4, 4, 3, 3, 4, 4, 4, + 3, 3, 4, 4, 4, 4, 3, 3, + 4, 4, 4, 4, 3, 3, 4, 4, + 4, 4, 3, 3, 4, 4, 4, 4, + 3, 3, 4, 4, 4, 4, 3, 3, + 5, 4, 4, 4, 3, 3, 4, 4, + 4, 3, 3, 4, 4, 4, 4, 3, + 3, 4, 4, 4, 4, 3, 3, 4, + 4, 4, 3, 3, 4, 4, 4, 4, + 4, 3, 3, 5, 5, 4, 4, 3, + 3, 4, 4, 3, 3, 4, 4, 4, + 3, 3, 4, 4, 4, 4, 3, 3, + 4, 4, 4, 4, 3, 3, 4, 4, + 4, 4, 3, 3, 4, 4, 4, 4, + 3, 3, 4, 4, 4, 4, 3, 3, + 5, 4, 4, 4, 3, 3, 4, 4, + 4, 3, 3, 4, 4, 4, 4, 3, + 3, 4, 4, 4, 4, 3, 3, 4, + 4, 4, 3, 3, 4, 4, 4, 4, + 4, 3, 3, 3, 3, 2, 0, 1, + 1, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 15, 2, 1, 1, 6, 0, + 4, 1, 3, 1, 1, 3, 7, 2, + 2, 2, 3, 41, 0, 0, 0, 0, + 0, 1, 0, 4, 0, 0, 0, 0, + 3, 1, 1, 1, 0, 0, 0, 2, + 0, 0, 0, 2, 1, 3, 23, 3, + 5, 9, 2, 1, 2, 5, 3, 2, + 2, 1, 2, 10, 6, 1, 5, 4, + 8, 2, 9, 4, 1, 1, 4, 1, + 3, 0, 7, 4, 3, 31, 0, 0, + 0, 0, 1, 0, 1, 1, 1, 0, + 0, 0, 0, 0, 3, 2, 1, 1, + 0, 0, 0, 1, 1, 0, 0, 0, + 1, 1, 0 + }; + + static const signed char _regex_range_lengths[] = { + 0, 0, 1, 3, 3, 3, 3, 0, + 3, 3, 3, 3, 3, 0, 0, 3, + 3, 3, 3, 1, 0, 3, 1, 3, + 3, 3, 3, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1, + 0, 3, 1, 1, 1, 0, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, + 1, 1, 3, 1, 1, 1, 1, 1, + 1, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1, 1, + 3, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 6, 0, 3, 3, 4, 1, + 1, 1, 3, 0, 1, 0, 0, 0, + 0, 0, 0, 2, 1, 1, 2, 2, + 1, 0, 0, 1, 1, 1, 1, 1, + 0, 0, 0, 3, 3, 3, 1, 0, + 1, 1, 1, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 5, 0, 4, 1, 1, + 1, 1, 0, 0, 0, 0, 3, 3, + 3, 1, 1, 1, 0, 0, 5, 0, + 1, 1, 1, 5, 0, 1, 1, 1, + 0, 0, 0 + }; + + static const short _regex_index_offsets[] = { + 0, 0, 2, 24, 30, 36, 41, 47, + 49, 55, 61, 67, 72, 78, 82, 85, + 92, 97, 103, 109, 112, 119, 125, 128, + 133, 138, 144, 149, 154, 162, 166, 170, + 173, 177, 181, 184, 188, 192, 195, 197, + 199, 201, 207, 209, 212, 215, 217, 223, + 229, 235, 241, 246, 252, 257, 263, 268, + 274, 276, 279, 284, 288, 291, 294, 296, + 298, 300, 302, 305, 307, 310, 313, 315, + 317, 320, 322, 324, 326, 328, 330, 332, + 334, 336, 338, 340, 342, 344, 346, 350, + 352, 354, 356, 358, 360, 362, 364, 366, + 368, 370, 372, 374, 376, 378, 380, 382, + 384, 386, 388, 390, 392, 395, 397, 399, + 401, 403, 405, 408, 410, 412, 414, 416, + 418, 420, 422, 425, 427, 429, 431, 433, + 435, 437, 439, 441, 443, 445, 447, 449, + 451, 453, 455, 457, 459, 461, 463, 466, + 468, 470, 472, 474, 476, 478, 480, 482, + 484, 486, 488, 490, 492, 494, 496, 498, + 500, 502, 504, 507, 509, 511, 513, 515, + 517, 519, 521, 523, 525, 528, 530, 532, + 534, 536, 538, 540, 542, 544, 546, 548, + 550, 552, 554, 556, 558, 560, 562, 564, + 566, 568, 570, 572, 574, 576, 578, 580, + 582, 584, 586, 588, 590, 592, 594, 596, + 598, 600, 602, 604, 606, 608, 610, 612, + 614, 616, 618, 620, 622, 624, 626, 628, + 630, 632, 634, 636, 638, 640, 643, 645, + 647, 649, 651, 653, 655, 657, 659, 661, + 663, 665, 667, 669, 671, 673, 675, 677, + 679, 681, 683, 685, 687, 689, 691, 693, + 695, 697, 699, 701, 703, 705, 707, 709, + 711, 713, 715, 717, 719, 721, 723, 726, + 728, 730, 732, 734, 736, 738, 740, 742, + 744, 746, 748, 750, 752, 754, 756, 758, + 760, 762, 764, 767, 769, 771, 773, 775, + 777, 779, 781, 783, 785, 787, 789, 791, + 793, 795, 797, 802, 804, 806, 808, 810, + 812, 814, 816, 818, 820, 822, 824, 826, + 828, 830, 832, 834, 836, 839, 841, 843, + 845, 847, 849, 851, 853, 855, 857, 860, + 862, 864, 866, 868, 870, 872, 876, 878, + 880, 882, 884, 886, 888, 890, 892, 895, + 897, 899, 901, 903, 905, 907, 910, 912, + 914, 916, 918, 920, 922, 924, 926, 928, + 930, 932, 934, 936, 938, 940, 942, 944, + 946, 948, 950, 952, 954, 956, 958, 960, + 962, 964, 966, 968, 970, 972, 974, 976, + 978, 980, 982, 984, 986, 988, 990, 992, + 994, 996, 998, 1001, 1003, 1005, 1007, 1009, + 1011, 1013, 1018, 1020, 1022, 1024, 1026, 1028, + 1030, 1032, 1034, 1036, 1038, 1040, 1042, 1044, + 1046, 1048, 1050, 1052, 1054, 1056, 1058, 1060, + 1062, 1064, 1066, 1068, 1070, 1072, 1074, 1076, + 1078, 1080, 1082, 1084, 1086, 1088, 1090, 1093, + 1095, 1097, 1099, 1101, 1103, 1105, 1107, 1109, + 1111, 1113, 1115, 1117, 1119, 1121, 1123, 1125, + 1127, 1129, 1131, 1134, 1136, 1138, 1140, 1142, + 1144, 1146, 1148, 1150, 1152, 1154, 1156, 1158, + 1160, 1162, 1164, 1166, 1168, 1170, 1172, 1174, + 1176, 1178, 1180, 1182, 1184, 1186, 1188, 1190, + 1192, 1194, 1197, 1199, 1201, 1203, 1205, 1207, + 1209, 1211, 1213, 1215, 1217, 1219, 1221, 1225, + 1228, 1230, 1232, 1234, 1236, 1238, 1240, 1242, + 1244, 1248, 1250, 1252, 1254, 1256, 1258, 1260, + 1262, 1264, 1266, 1268, 1270, 1272, 1274, 1276, + 1279, 1281, 1283, 1286, 1288, 1290, 1292, 1294, + 1296, 1298, 1300, 1302, 1304, 1306, 1308, 1310, + 1312, 1314, 1316, 1318, 1320, 1322, 1324, 1326, + 1328, 1330, 1332, 1334, 1338, 1342, 1345, 1361, + 1365, 1369, 1372, 1387, 1393, 1399, 1404, 1409, + 1413, 1417, 1422, 1427, 1431, 1435, 1440, 1445, + 1450, 1454, 1458, 1463, 1468, 1473, 1478, 1482, + 1486, 1491, 1496, 1501, 1506, 1510, 1514, 1519, + 1524, 1529, 1534, 1538, 1542, 1547, 1552, 1557, + 1562, 1566, 1570, 1575, 1580, 1585, 1590, 1594, + 1598, 1604, 1609, 1614, 1619, 1623, 1627, 1632, + 1637, 1642, 1646, 1650, 1655, 1660, 1665, 1670, + 1674, 1678, 1683, 1688, 1693, 1698, 1702, 1706, + 1711, 1716, 1721, 1725, 1729, 1734, 1739, 1744, + 1749, 1754, 1758, 1762, 1768, 1774, 1779, 1784, + 1788, 1792, 1797, 1802, 1806, 1810, 1815, 1820, + 1825, 1829, 1833, 1838, 1843, 1848, 1853, 1857, + 1861, 1866, 1871, 1876, 1881, 1885, 1889, 1894, + 1899, 1904, 1909, 1913, 1917, 1922, 1927, 1932, + 1937, 1941, 1945, 1950, 1955, 1960, 1965, 1969, + 1973, 1979, 1984, 1989, 1994, 1998, 2002, 2007, + 2012, 2017, 2021, 2025, 2030, 2035, 2040, 2045, + 2049, 2053, 2058, 2063, 2068, 2073, 2077, 2081, + 2086, 2091, 2096, 2100, 2104, 2109, 2114, 2119, + 2124, 2129, 2133, 2137, 2141, 2145, 2148, 2150, + 2153, 2158, 2160, 2162, 2164, 2166, 2168, 2170, + 2172, 2174, 2176, 2198, 2201, 2206, 2211, 2222, + 2224, 2230, 2233, 2240, 2242, 2245, 2249, 2257, + 2260, 2263, 2266, 2270, 2314, 2316, 2318, 2321, + 2324, 2326, 2328, 2329, 2335, 2337, 2339, 2341, + 2343, 2347, 2349, 2351, 2356, 2360, 2364, 2366, + 2369, 2371, 2373, 2375, 2378, 2380, 2384, 2408, + 2412, 2418, 2428, 2431, 2433, 2436, 2442, 2446, + 2449, 2452, 2454, 2457, 2468, 2475, 2477, 2483, + 2488, 2497, 2500, 2510, 2515, 2517, 2519, 2524, + 2526, 2530, 2531, 2539, 2549, 2553, 2589, 2591, + 2593, 2595, 2597, 2599, 2600, 2602, 2604, 2609, + 2613, 2617, 2619, 2621, 2623, 2627, 2630, 2637, + 2639, 2641, 2643, 2645, 2652, 2654, 2656, 2658, + 2660, 2662, 0 + }; + + static const signed char _regex_trans_cond_spaces[] = { + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 0, -1, + 0, -1, 0, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 0, -1, + 0, -1, 0, -1, 0, -1, 0, -1, + 0, -1, 0, -1, 0, -1, 0, -1, + -1, -1, 1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + 0, 0, 0, 0, 0, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, 0, -1, 0, -1, 0, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 0, + 0, 0, 0, 0, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, 0, -1, 0, -1, 0, -1, -1, + 2, 2, -1, -1, -1, -1, -1, 0, + 0, 0, 0, 0, -1, -1, -1, 0, + -1, 0, -1, 0, -1, -1, 0, 0, + 0, 0, 0, -1, -1, -1, 0, -1, + 0, -1, 0, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 0 + }; + + static const short _regex_trans_offsets[] = { + 0, 1, 2, 3, 4, 5, 6, 7, + 8, 9, 10, 11, 12, 13, 14, 15, + 16, 17, 18, 19, 20, 21, 22, 23, + 24, 25, 26, 27, 28, 29, 30, 31, + 32, 33, 34, 35, 36, 37, 38, 39, + 40, 41, 42, 43, 44, 45, 46, 47, + 48, 49, 50, 51, 52, 53, 54, 55, + 56, 57, 58, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, + 72, 73, 74, 75, 76, 77, 78, 79, + 80, 81, 82, 83, 84, 85, 86, 87, + 88, 89, 90, 91, 92, 93, 94, 95, + 96, 97, 98, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, 115, 116, 117, 118, 119, + 120, 121, 122, 123, 124, 125, 126, 127, + 128, 129, 130, 131, 132, 133, 134, 135, + 136, 137, 138, 139, 140, 141, 142, 143, + 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, 159, + 160, 161, 162, 163, 164, 165, 166, 167, + 168, 169, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, + 192, 193, 194, 195, 196, 197, 198, 199, + 200, 201, 202, 203, 204, 205, 206, 207, + 208, 209, 210, 211, 212, 213, 214, 215, + 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 227, 228, 229, 230, 231, + 232, 233, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 277, 278, 279, + 280, 281, 282, 283, 284, 285, 286, 287, + 288, 289, 290, 291, 292, 293, 294, 296, + 297, 299, 300, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, 321, 322, + 323, 324, 325, 326, 327, 328, 329, 330, + 331, 332, 333, 334, 335, 336, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, + 443, 444, 445, 446, 447, 448, 449, 450, + 451, 452, 453, 454, 455, 456, 457, 458, + 459, 460, 461, 462, 463, 464, 465, 466, + 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 501, 502, 503, 504, 505, 506, + 507, 508, 509, 510, 511, 512, 513, 514, + 515, 516, 517, 518, 519, 520, 521, 522, + 523, 524, 525, 526, 527, 528, 529, 530, + 531, 532, 533, 534, 535, 536, 537, 538, + 539, 540, 541, 542, 543, 544, 545, 546, + 547, 548, 549, 550, 551, 552, 553, 554, + 555, 556, 557, 558, 559, 560, 561, 562, + 563, 564, 565, 566, 567, 568, 569, 570, + 571, 572, 573, 574, 575, 576, 577, 578, + 579, 580, 581, 582, 583, 584, 585, 586, + 587, 588, 589, 590, 591, 592, 593, 594, + 595, 596, 597, 598, 599, 600, 601, 602, + 603, 604, 605, 606, 607, 608, 609, 610, + 611, 612, 613, 614, 615, 616, 617, 618, + 619, 620, 621, 622, 623, 624, 625, 626, + 627, 628, 629, 630, 631, 632, 633, 634, + 635, 636, 637, 638, 639, 640, 641, 642, + 643, 644, 645, 646, 647, 648, 649, 650, + 651, 652, 653, 654, 655, 656, 657, 658, + 659, 660, 661, 662, 663, 664, 665, 666, + 667, 668, 669, 670, 671, 672, 673, 674, + 675, 676, 677, 678, 679, 680, 681, 682, + 683, 684, 685, 686, 687, 688, 689, 690, + 691, 692, 693, 694, 695, 696, 697, 698, + 699, 700, 701, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 713, 714, + 715, 716, 717, 718, 719, 720, 721, 722, + 723, 724, 725, 726, 727, 728, 729, 730, + 731, 732, 733, 734, 735, 736, 737, 738, + 739, 740, 741, 742, 743, 744, 745, 746, + 747, 748, 749, 750, 751, 752, 753, 754, + 755, 756, 757, 758, 759, 760, 761, 762, + 763, 764, 765, 766, 767, 768, 769, 770, + 771, 772, 773, 774, 775, 776, 777, 778, + 779, 780, 781, 782, 783, 784, 785, 786, + 787, 788, 789, 790, 791, 792, 793, 794, + 795, 796, 797, 798, 799, 800, 801, 802, + 803, 804, 805, 806, 807, 808, 809, 810, + 811, 812, 813, 814, 815, 816, 817, 818, + 819, 820, 821, 822, 823, 824, 825, 826, + 827, 828, 829, 830, 831, 832, 833, 834, + 835, 836, 837, 838, 839, 840, 841, 842, + 843, 844, 845, 846, 847, 848, 849, 850, + 851, 852, 853, 854, 855, 856, 857, 858, + 859, 860, 861, 862, 863, 864, 865, 866, + 867, 868, 869, 870, 871, 872, 873, 874, + 875, 876, 877, 878, 879, 880, 881, 882, + 883, 884, 885, 886, 887, 888, 889, 890, + 891, 892, 893, 894, 895, 896, 897, 898, + 899, 900, 901, 902, 903, 904, 905, 906, + 907, 908, 909, 910, 911, 912, 913, 914, + 915, 916, 917, 918, 919, 920, 921, 922, + 923, 924, 925, 926, 927, 928, 929, 930, + 931, 932, 933, 934, 935, 936, 937, 938, + 939, 940, 941, 942, 943, 944, 945, 946, + 947, 948, 949, 950, 951, 952, 953, 954, + 955, 956, 957, 958, 959, 960, 961, 962, + 963, 964, 965, 966, 967, 968, 969, 970, + 971, 972, 973, 974, 975, 976, 977, 978, + 979, 980, 981, 982, 983, 984, 985, 986, + 987, 988, 989, 990, 991, 992, 993, 994, + 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, + 1011, 1012, 1013, 1014, 1015, 1016, 1017, 1018, + 1019, 1020, 1021, 1022, 1023, 1024, 1025, 1026, + 1027, 1028, 1029, 1030, 1031, 1032, 1033, 1034, + 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, + 1043, 1044, 1045, 1046, 1047, 1048, 1049, 1050, + 1051, 1052, 1053, 1054, 1055, 1056, 1057, 1058, + 1059, 1060, 1061, 1062, 1063, 1064, 1065, 1066, + 1067, 1068, 1069, 1070, 1071, 1072, 1073, 1074, + 1075, 1076, 1077, 1078, 1079, 1080, 1081, 1082, + 1083, 1084, 1085, 1086, 1087, 1088, 1089, 1090, + 1091, 1092, 1093, 1094, 1095, 1096, 1097, 1098, + 1099, 1100, 1101, 1102, 1103, 1104, 1105, 1106, + 1107, 1108, 1109, 1110, 1111, 1112, 1113, 1114, + 1115, 1116, 1117, 1118, 1119, 1120, 1121, 1122, + 1123, 1124, 1125, 1126, 1127, 1128, 1129, 1130, + 1131, 1132, 1133, 1134, 1135, 1136, 1137, 1138, + 1139, 1140, 1141, 1142, 1143, 1144, 1145, 1146, + 1147, 1148, 1149, 1150, 1151, 1152, 1153, 1154, + 1155, 1156, 1157, 1158, 1159, 1160, 1161, 1162, + 1163, 1164, 1165, 1166, 1167, 1168, 1169, 1170, + 1171, 1172, 1173, 1174, 1175, 1176, 1177, 1178, + 1179, 1180, 1181, 1182, 1183, 1184, 1185, 1186, + 1187, 1188, 1189, 1190, 1191, 1192, 1193, 1194, + 1195, 1196, 1197, 1198, 1199, 1200, 1201, 1202, + 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, + 1211, 1212, 1213, 1214, 1215, 1216, 1217, 1218, + 1219, 1220, 1221, 1222, 1223, 1224, 1225, 1226, + 1227, 1228, 1229, 1230, 1231, 1232, 1233, 1234, + 1235, 1236, 1237, 1238, 1239, 1240, 1241, 1242, + 1243, 1244, 1245, 1246, 1247, 1248, 1249, 1250, + 1251, 1252, 1253, 1254, 1255, 1256, 1257, 1258, + 1259, 1260, 1261, 1262, 1263, 1264, 1265, 1266, + 1267, 1268, 1269, 1270, 1271, 1272, 1273, 1274, + 1275, 1276, 1277, 1278, 1279, 1280, 1281, 1282, + 1283, 1284, 1285, 1286, 1287, 1288, 1289, 1290, + 1291, 1292, 1293, 1294, 1295, 1296, 1297, 1298, + 1299, 1300, 1301, 1302, 1303, 1304, 1305, 1306, + 1307, 1308, 1309, 1310, 1311, 1312, 1313, 1314, + 1315, 1316, 1317, 1318, 1319, 1320, 1321, 1322, + 1323, 1324, 1325, 1326, 1327, 1328, 1329, 1330, + 1331, 1332, 1333, 1334, 1335, 1336, 1337, 1338, + 1339, 1340, 1341, 1342, 1343, 1344, 1345, 1346, + 1347, 1348, 1349, 1350, 1351, 1352, 1353, 1354, + 1355, 1356, 1357, 1358, 1359, 1360, 1361, 1362, + 1363, 1364, 1365, 1366, 1367, 1368, 1369, 1370, + 1371, 1372, 1373, 1374, 1375, 1376, 1377, 1378, + 1379, 1380, 1381, 1382, 1383, 1384, 1385, 1386, + 1387, 1388, 1389, 1390, 1391, 1392, 1393, 1394, + 1395, 1396, 1397, 1398, 1399, 1400, 1401, 1402, + 1403, 1404, 1405, 1406, 1407, 1408, 1409, 1410, + 1411, 1412, 1413, 1414, 1415, 1416, 1417, 1418, + 1419, 1420, 1421, 1422, 1423, 1424, 1425, 1426, + 1427, 1428, 1429, 1430, 1431, 1432, 1433, 1434, + 1435, 1436, 1437, 1438, 1439, 1440, 1441, 1442, + 1443, 1444, 1445, 1446, 1447, 1448, 1449, 1450, + 1451, 1452, 1453, 1454, 1455, 1456, 1457, 1458, + 1459, 1460, 1461, 1462, 1463, 1464, 1465, 1466, + 1467, 1468, 1469, 1470, 1471, 1472, 1473, 1474, + 1475, 1476, 1477, 1478, 1479, 1480, 1481, 1482, + 1483, 1484, 1485, 1486, 1487, 1488, 1489, 1490, + 1491, 1492, 1493, 1494, 1495, 1496, 1497, 1498, + 1499, 1500, 1501, 1502, 1503, 1504, 1505, 1506, + 1507, 1508, 1509, 1510, 1511, 1512, 1513, 1514, + 1515, 1516, 1517, 1518, 1519, 1520, 1521, 1522, + 1523, 1524, 1525, 1526, 1527, 1528, 1529, 1530, + 1531, 1532, 1533, 1534, 1535, 1536, 1537, 1538, + 1539, 1540, 1541, 1542, 1543, 1544, 1545, 1546, + 1547, 1548, 1549, 1550, 1551, 1552, 1553, 1554, + 1555, 1556, 1557, 1558, 1559, 1560, 1561, 1562, + 1563, 1564, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 1572, 1573, 1574, 1575, 1576, 1577, 1578, + 1579, 1580, 1581, 1582, 1583, 1584, 1585, 1586, + 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, + 1595, 1596, 1597, 1598, 1599, 1600, 1601, 1602, + 1603, 1604, 1605, 1606, 1607, 1608, 1609, 1610, + 1611, 1612, 1613, 1614, 1615, 1616, 1617, 1618, + 1619, 1620, 1621, 1622, 1623, 1624, 1625, 1626, + 1627, 1628, 1629, 1630, 1631, 1632, 1633, 1634, + 1635, 1636, 1637, 1638, 1639, 1640, 1641, 1642, + 1643, 1644, 1645, 1646, 1647, 1648, 1649, 1650, + 1651, 1652, 1653, 1654, 1655, 1656, 1657, 1658, + 1659, 1660, 1661, 1662, 1663, 1664, 1665, 1666, + 1667, 1668, 1669, 1670, 1671, 1672, 1673, 1674, + 1675, 1676, 1677, 1678, 1679, 1680, 1681, 1682, + 1683, 1684, 1685, 1686, 1687, 1688, 1689, 1690, + 1691, 1692, 1693, 1694, 1695, 1696, 1697, 1698, + 1699, 1700, 1701, 1702, 1703, 1704, 1705, 1706, + 1707, 1708, 1709, 1710, 1711, 1712, 1713, 1714, + 1715, 1716, 1717, 1718, 1719, 1720, 1721, 1722, + 1723, 1724, 1725, 1726, 1727, 1728, 1729, 1730, + 1731, 1732, 1733, 1734, 1735, 1736, 1737, 1738, + 1739, 1740, 1741, 1742, 1743, 1744, 1745, 1746, + 1747, 1748, 1749, 1750, 1751, 1752, 1753, 1754, + 1755, 1756, 1757, 1758, 1759, 1760, 1761, 1762, + 1763, 1764, 1765, 1766, 1767, 1768, 1769, 1770, + 1771, 1772, 1773, 1774, 1775, 1776, 1777, 1778, + 1779, 1780, 1781, 1782, 1783, 1784, 1785, 1786, + 1787, 1788, 1789, 1790, 1791, 1792, 1793, 1794, + 1795, 1796, 1797, 1798, 1799, 1800, 1801, 1802, + 1803, 1804, 1805, 1806, 1807, 1808, 1809, 1810, + 1811, 1812, 1813, 1814, 1815, 1816, 1817, 1818, + 1819, 1820, 1821, 1822, 1823, 1824, 1825, 1826, + 1827, 1828, 1829, 1830, 1831, 1832, 1833, 1834, + 1835, 1836, 1837, 1838, 1839, 1840, 1841, 1842, + 1843, 1844, 1845, 1846, 1847, 1848, 1849, 1850, + 1851, 1852, 1853, 1854, 1855, 1856, 1857, 1858, + 1859, 1860, 1861, 1862, 1863, 1864, 1865, 1866, + 1867, 1868, 1869, 1870, 1871, 1872, 1873, 1874, + 1875, 1876, 1877, 1878, 1879, 1880, 1881, 1882, + 1883, 1884, 1885, 1886, 1887, 1888, 1889, 1890, + 1891, 1892, 1893, 1894, 1895, 1896, 1897, 1898, + 1899, 1900, 1901, 1902, 1903, 1904, 1905, 1906, + 1907, 1908, 1909, 1910, 1911, 1912, 1913, 1914, + 1915, 1916, 1917, 1918, 1919, 1920, 1921, 1922, + 1923, 1924, 1925, 1926, 1927, 1928, 1929, 1930, + 1931, 1932, 1933, 1934, 1935, 1936, 1937, 1938, + 1939, 1940, 1941, 1942, 1943, 1944, 1945, 1946, + 1947, 1948, 1949, 1950, 1951, 1952, 1953, 1954, + 1955, 1956, 1957, 1958, 1959, 1960, 1961, 1962, + 1963, 1964, 1965, 1966, 1967, 1968, 1969, 1970, + 1971, 1972, 1973, 1974, 1975, 1976, 1977, 1978, + 1979, 1980, 1981, 1982, 1983, 1984, 1985, 1986, + 1987, 1988, 1989, 1990, 1991, 1992, 1993, 1994, + 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002, + 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, + 2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, + 2019, 2020, 2021, 2022, 2023, 2024, 2025, 2026, + 2027, 2028, 2029, 2030, 2031, 2032, 2033, 2034, + 2035, 2036, 2037, 2038, 2039, 2040, 2041, 2042, + 2043, 2044, 2045, 2046, 2047, 2048, 2049, 2050, + 2051, 2052, 2053, 2054, 2055, 2056, 2057, 2058, + 2059, 2060, 2061, 2062, 2063, 2064, 2065, 2066, + 2067, 2068, 2069, 2070, 2071, 2072, 2073, 2074, + 2075, 2076, 2077, 2078, 2079, 2080, 2081, 2082, + 2083, 2084, 2085, 2086, 2087, 2088, 2089, 2090, + 2091, 2092, 2093, 2094, 2095, 2096, 2097, 2098, + 2099, 2100, 2101, 2102, 2103, 2104, 2105, 2106, + 2107, 2108, 2109, 2110, 2111, 2112, 2113, 2114, + 2115, 2116, 2117, 2118, 2119, 2120, 2121, 2122, + 2123, 2124, 2125, 2126, 2127, 2128, 2129, 2130, + 2131, 2132, 2133, 2134, 2135, 2136, 2137, 2138, + 2139, 2140, 2141, 2142, 2143, 2144, 2145, 2146, + 2147, 2148, 2149, 2150, 2151, 2152, 2153, 2154, + 2155, 2156, 2157, 2158, 2159, 2160, 2161, 2163, + 2164, 2166, 2167, 2169, 2170, 2172, 2173, 2175, + 2176, 2178, 2179, 2181, 2182, 2184, 2185, 2187, + 2188, 2189, 2190, 2192, 2193, 2194, 2195, 2196, + 2197, 2198, 2199, 2200, 2201, 2202, 2203, 2204, + 2205, 2207, 2209, 2211, 2213, 2215, 2216, 2217, + 2218, 2219, 2220, 2221, 2222, 2223, 2224, 2225, + 2226, 2227, 2228, 2229, 2230, 2231, 2232, 2233, + 2234, 2235, 2236, 2237, 2238, 2239, 2240, 2241, + 2242, 2243, 2244, 2245, 2246, 2247, 2248, 2249, + 2250, 2251, 2252, 2253, 2254, 2255, 2256, 2257, + 2258, 2259, 2260, 2261, 2262, 2263, 2264, 2265, + 2266, 2267, 2268, 2269, 2270, 2271, 2272, 2273, + 2274, 2275, 2276, 2277, 2278, 2279, 2280, 2281, + 2282, 2283, 2284, 2285, 2286, 2287, 2288, 2289, + 2290, 2291, 2292, 2293, 2294, 2295, 2296, 2297, + 2298, 2299, 2300, 2301, 2302, 2303, 2304, 2305, + 2306, 2307, 2308, 2309, 2310, 2311, 2312, 2313, + 2314, 2315, 2316, 2317, 2318, 2319, 2320, 2321, + 2322, 2323, 2324, 2325, 2326, 2327, 2328, 2329, + 2330, 2331, 2332, 2333, 2334, 2335, 2336, 2337, + 2338, 2339, 2340, 2341, 2342, 2343, 2344, 2345, + 2346, 2347, 2348, 2349, 2350, 2351, 2352, 2353, + 2354, 2355, 2356, 2357, 2358, 2359, 2360, 2361, + 2362, 2363, 2364, 2365, 2366, 2367, 2368, 2369, + 2370, 2371, 2372, 2373, 2374, 2375, 2376, 2377, + 2378, 2379, 2380, 2381, 2382, 2383, 2384, 2385, + 2386, 2387, 2389, 2390, 2392, 2393, 2395, 2396, + 2397, 2398, 2399, 2400, 2401, 2402, 2403, 2404, + 2405, 2406, 2407, 2408, 2409, 2410, 2411, 2412, + 2413, 2414, 2415, 2416, 2417, 2418, 2419, 2420, + 2421, 2422, 2423, 2424, 2425, 2426, 2427, 2428, + 2429, 2430, 2431, 2432, 2433, 2434, 2435, 2436, + 2437, 2438, 2439, 2440, 2441, 2442, 2443, 2444, + 2445, 2446, 2447, 2448, 2449, 2450, 2451, 2452, + 2453, 2454, 2455, 2456, 2457, 2458, 2459, 2460, + 2461, 2462, 2463, 2464, 2465, 2466, 2467, 2468, + 2469, 2470, 2471, 2472, 2473, 2474, 2475, 2476, + 2477, 2478, 2479, 2480, 2481, 2482, 2483, 2484, + 2485, 2486, 2487, 2488, 2489, 2490, 2491, 2492, + 2493, 2494, 2495, 2496, 2497, 2498, 2499, 2500, + 2501, 2502, 2503, 2504, 2505, 2506, 2507, 2508, + 2509, 2510, 2511, 2512, 2513, 2514, 2515, 2516, + 2517, 2518, 2519, 2520, 2521, 2522, 2523, 2524, + 2525, 2526, 2527, 2528, 2529, 2530, 2531, 2532, + 2533, 2534, 2535, 2536, 2537, 2538, 2539, 2540, + 2541, 2542, 2543, 2544, 2545, 2546, 2547, 2548, + 2549, 2550, 2551, 2552, 2553, 2554, 2555, 2556, + 2557, 2558, 2559, 2560, 2561, 2562, 2563, 2564, + 2566, 2568, 2570, 2572, 2574, 2575, 2576, 2577, + 2578, 2579, 2580, 2581, 2582, 2583, 2584, 2585, + 2586, 2587, 2588, 2589, 2590, 2591, 2592, 2593, + 2594, 2595, 2596, 2597, 2598, 2599, 2600, 2601, + 2602, 2603, 2604, 2605, 2606, 2607, 2608, 2609, + 2610, 2611, 2612, 2613, 2614, 2615, 2616, 2617, + 2618, 2619, 2620, 2621, 2622, 2623, 2624, 2625, + 2626, 2627, 2628, 2629, 2630, 2631, 2632, 2633, + 2634, 2635, 2636, 2637, 2638, 2639, 2640, 2641, + 2642, 2643, 2645, 2646, 2648, 2649, 2651, 2652, + 2653, 2655, 2657, 2658, 2659, 2660, 2661, 2662, + 2664, 2666, 2668, 2670, 2672, 2673, 2674, 2675, + 2677, 2678, 2680, 2681, 2683, 2684, 2685, 2687, + 2689, 2691, 2693, 2695, 2696, 2697, 2698, 2700, + 2701, 2703, 2704, 2706, 2707, 2708, 2709, 2710, + 2711, 2712, 2713, 2714, 2715, 2716, 2717, 2718, + 2719, 2720, 2721, 2722, 2723, 2724, 2725, 2726, + 2727, 2728, 2729, 2730, 2731, 2732, 2733, 2734, + 2735, 2736, 2737, 2738, 2739, 2740, 2741, 2742, + 2743, 2744, 2745, 2746, 2747, 2748, 2749, 2750, + 2751, 2752, 2753, 2754, 2755, 2756, 2757, 2758, + 2759, 2760, 2761, 2762, 2763, 2764, 2765, 2766, + 2767, 2768, 2769, 2770, 2771, 2772, 2773, 2774, + 2775, 2776, 2777, 2778, 2779, 2780, 2781, 2782, + 2783, 2784, 2785, 2786, 2787, 2788, 2789, 2790, + 2791, 2792, 2793, 2794, 2795, 2796, 2797, 2798, + 2799, 2800, 2801, 2802, 2803, 2804, 2805, 2806, + 2807, 2808, 2809, 2810, 2811, 2812, 2813, 2814, + 2815, 2816, 2817, 2818, 2819, 2820, 2821, 2822, + 2823, 2824, 2825, 2826, 2827, 2828, 2829, 2830, + 2831, 2832, 2833, 2834, 2835, 2836, 2837, 2838, + 2839, 2840, 2841, 2842, 2843, 2844, 2845, 2846, + 2847, 2848, 2849, 2850, 2851, 2852, 2853, 2854, + 2855, 2856, 2857, 2858, 2859, 2860, 2861, 2862, + 2863, 2864, 2865, 2866, 2867, 2868, 2869, 2870, + 2871, 2872, 2873, 2874, 2875, 2876, 2877, 2878, + 2879, 2880, 2881, 2882, 2883, 2884, 2885, 2886, + 2887, 2888, 2889, 2890, 2891, 2892, 2893, 2894, + 2895, 2896, 2897, 2898, 2899, 2900, 2901, 2902, + 2903, 2904, 2905, 2906, 2907, 2908, 2909, 2910, + 2911, 2912, 2913, 2914, 2915, 2916, 2917, 2918, + 2919, 2920, 2921, 2922, 2923, 2924, 2925, 2926, + 2927, 2928, 2929, 2930, 2931, 2932, 2933, 2934, + 2935, 2936, 2937, 2938, 2939, 2940, 2941, 2942, + 2943, 2944, 2945, 2946, 2947, 2948, 2949, 2950, + 2951, 2952, 2953, 2954, 2955, 2956, 2957, 2958, + 2959, 2960, 2961, 2962, 2963, 2964, 2965, 2966, + 2967, 2968, 2969, 2970, 2971, 2972, 2973, 2974, + 2975, 2976, 2977, 2978, 2979, 2980, 2981, 2982, + 2983, 2984, 2985, 2986, 2987, 2988, 2989, 2990, + 2991, 2992, 2993, 2994, 2995, 2996, 2997, 2998, + 2999, 3000, 3001, 3002, 3003, 3004, 3005, 3006, + 3007, 3008, 3009, 3010, 3011, 3012, 3013, 3014, + 3015, 3016, 3017, 3018, 3019, 3020, 3021, 3022, + 3023, 3024, 3025, 3026, 3027, 3028, 3029, 3030, + 3031, 3032, 3033, 3034, 3035, 3036, 3037, 3038, + 3039, 3040, 3041, 3042, 3043, 3044, 3045, 3046, + 3047, 3048, 3049, 3050, 3051, 3052, 3053, 3054, + 3055, 3056, 3057, 3058, 3059, 3060, 3061, 3062, + 3063, 3064, 3065, 3066, 3067, 3068, 3069, 3070, + 3071, 3072, 3073, 3074, 3075, 3076, 3077, 3078, + 3079, 3080, 3081, 3082, 3083, 3084, 3085, 3086, + 3087, 3088, 3089, 3090, 3091, 3092, 3093, 3094, + 3095, 3096, 3097, 3098, 3099, 3100, 3101, 3102, + 3103, 3104, 3105, 3106, 3107, 3108, 3109, 3110, + 3111, 3112, 3113, 3114, 3115, 3116, 3117, 3118, + 3119, 3120, 3121, 3122, 3123, 3124, 3125, 3126, + 3127, 3128, 3129, 3130, 3131, 3132, 3133, 3134, + 3135, 3136, 3137, 3138, 3139, 3140, 3141, 3142, + 3143, 3144, 3145, 3146, 3147, 3148, 3149, 3150, + 3151, 3152, 3153, 3154, 3155, 3156, 3157, 3158, + 3159, 3160, 3161, 3162, 3163, 3164, 3165, 3166, + 3167, 3168, 3169, 3170, 3171, 3172, 3173, 3174, + 3175, 3176, 3177, 3178, 3179, 3180, 3181, 3182, + 3183, 3184, 3185, 3186, 3187, 3188, 3189, 3190, + 3191, 3192, 3193, 3194, 3195, 3196, 3197, 3198, + 3199, 3200, 3201, 3202, 3203, 3204, 3205, 3206, + 3207, 3208, 3209, 3210, 3211, 3212, 3213, 3214, + 3215, 3216, 3217, 3218, 3219, 3220, 3221, 3222, + 3223, 3224, 3225, 3226, 3227, 3228, 3229, 3230, + 3231, 3232, 3233, 3234, 3235, 3236, 3237, 3238, + 3239, 3240, 3241, 3242, 3243, 3244, 3245, 3246, + 3247, 3248, 3249, 3250, 3251, 3252, 3253, 3254, + 3255, 3256, 3257, 3258, 3259, 3260, 3261, 3262, + 3263, 3264, 3265, 3266, 3267, 3268, 3269, 3270, + 3271, 3272, 3273, 3274, 3275, 3276, 3277, 3278, + 3279, 3280, 3281, 3282, 3283, 3284, 3285, 3286, + 3287, 3288, 3289, 3290, 3291, 3292, 3293, 3294, + 3295, 3296, 3297, 3298, 3299, 3300, 3301, 3302, + 3303, 3304, 3305, 3306, 3307, 3308, 3309, 3310, + 3311, 3312, 3313, 3314, 3315, 3316, 3317, 3318, + 3319, 3320, 3321, 3322, 3323, 3324, 3325, 3326, + 3327, 3328, 3329, 3330, 3331, 3332, 3333, 3334, + 3335, 3336, 3337, 3338, 3339, 3340, 3341, 3342, + 3343, 3344, 3345, 3346, 3347, 3348, 3349, 3350, + 3351, 3352, 3353, 3354, 3355, 3356, 3357, 3358, + 3359, 3360, 3361, 3362, 3363, 3364, 3365, 3366, + 3367, 3368, 3369, 3370, 3371, 3372, 3373, 3374, + 3375, 3376, 3377, 3378, 3379, 3380, 3381, 3382, + 3383, 3384, 3385, 3386, 3387, 3388, 3389, 3390, + 3391, 3392, 3393, 3394, 3395, 3396, 3397, 3398, + 3399, 3400, 3401, 3402, 3403, 3404, 3405, 3406, + 3407, 3408, 3409, 3410, 3411, 3412, 3413, 3414, + 3415, 3416, 3417, 3418, 3419, 3420, 3421, 3422, + 3423, 3424, 3425, 3426, 3427, 3428, 3429, 3430, + 3431, 3432, 3433, 3434, 3435, 3436, 3437, 3438, + 3439, 3440, 3441, 3442, 3443, 3444, 3445, 3446, + 3447, 3448, 3449, 3450, 3451, 3452, 3453, 3454, + 3455, 3456, 3457, 3458, 3459, 3460, 3461, 3462, + 3463, 3464, 3465, 3466, 3467, 3468, 3469, 3470, + 3471, 3472, 3473, 3474, 3475, 3476, 3477, 3478, + 3479, 3480, 3481, 3482, 3483, 3484, 3485, 3486, + 3487, 3488, 3489, 3490, 3491, 3492, 3493, 3494, + 3495, 3496, 3497, 3498, 3499, 3500, 3501, 3502, + 3503, 3504, 3505, 3506, 3507, 3508, 3509, 3510, + 3511, 3512, 3513, 3514, 3515, 3516, 3517, 3518, + 3519, 3520, 3521, 3522, 3523, 3524, 3525, 3526, + 3527, 3528, 3529, 3530, 3531, 3532, 3533, 3534, + 3535, 3536, 3537, 3538, 3539, 3540, 3541, 3542, + 3543, 3544, 3545, 3546, 3547, 3548, 3549, 3550, + 3551, 3552, 3553, 3554, 3555, 3556, 3557, 3558, + 3559, 3560, 0 + }; + + static const signed char _regex_trans_lengths[] = { + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 2, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 2, 1, 2, 1, 2, 1, 2, 1, + 2, 1, 2, 1, 2, 1, 2, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 2, 2, 2, 2, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 2, 1, 2, 1, 2, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 2, + 2, 2, 2, 2, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 2, 1, 2, 1, 2, 1, 1, + 2, 2, 1, 1, 1, 1, 1, 2, + 2, 2, 2, 2, 1, 1, 1, 2, + 1, 2, 1, 2, 1, 1, 2, 2, + 2, 2, 2, 1, 1, 1, 2, 1, + 2, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 0 + }; + + static const signed char _regex_cond_keys[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1, + 0, 0, 1, 0, 0, 1, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1, 0, 0, 1, 0, 0, + 1, 0, 0, 1, 0, 0, 1, 0, + 0, 1, 0, 0, 1, 0, 0, 1, + 0, 0, 1, 0, 0, 0, 0, 1, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1, 0, + 1, 0, 1, 0, 1, 0, 1, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 0, 0, 1, + 0, 0, 1, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1, 0, 1, + 0, 1, 0, 1, 0, 1, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 0, 0, 1, + 0, 0, 1, 0, 0, 0, 1, 0, + 1, 0, 0, 0, 0, 0, 0, 1, + 0, 1, 0, 1, 0, 1, 0, 1, + 0, 0, 0, 0, 1, 0, 0, 1, + 0, 0, 1, 0, 0, 0, 1, 0, + 1, 0, 1, 0, 1, 0, 1, 0, + 0, 0, 0, 1, 0, 0, 1, 0, + 0, 1, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0 + }; + + static const short _regex_cond_targs[] = { + 746, 746, 746, 746, 748, 749, 750, 746, + 751, 752, 746, 754, 746, 746, 755, 756, + 757, 758, 758, 758, 758, 746, 753, 746, + 746, 3, 3, 3, 3, 746, 746, 4, + 4, 4, 4, 746, 6, 6, 6, 6, + 746, 7, 6, 6, 6, 6, 746, 746, + 746, 746, 8, 8, 8, 8, 746, 746, + 8, 9, 8, 8, 746, 746, 8, 9, + 8, 8, 746, 12, 12, 12, 12, 746, + 7, 12, 12, 12, 12, 746, 746, 14, + 746, 746, 746, 746, 746, 16, 746, 8, + 18, 8, 8, 746, 17, 17, 17, 17, + 746, 746, 17, 17, 17, 17, 746, 746, + 8, 18, 8, 8, 746, 746, 19, 746, + 746, 746, 20, 20, 20, 20, 746, 746, + 21, 21, 21, 21, 746, 746, 22, 746, + 21, 21, 21, 21, 746, 25, 25, 25, + 25, 746, 746, 25, 25, 25, 25, 746, + 3, 3, 3, 3, 746, 20, 20, 20, + 20, 746, 746, 27, 746, 28, 28, 28, + 28, 746, 30, 31, 746, 29, 30, 31, + 746, 29, 30, 31, 29, 33, 34, 746, + 32, 33, 34, 746, 32, 33, 34, 32, + 36, 37, 746, 35, 36, 37, 746, 35, + 36, 37, 35, 746, 38, 772, 746, 746, + 40, 42, 49, 46, 49, 49, 746, 43, + 746, 746, 44, 746, 746, 45, 746, 746, + 746, 49, 746, 47, 49, 49, 746, 49, + 746, 48, 49, 49, 746, 49, 746, 49, + 49, 49, 746, 49, 746, 49, 49, 49, + 746, 51, 51, 51, 51, 746, 746, 51, + 51, 51, 51, 746, 53, 53, 53, 53, + 746, 746, 53, 53, 53, 53, 746, 55, + 55, 55, 55, 746, 55, 746, 55, 55, + 55, 746, 57, 746, 746, 57, 746, 746, + 58, 58, 58, 746, 60, 783, 59, 746, + 783, 61, 746, 783, 61, 746, 746, 746, + 746, 746, 64, 746, 746, 746, 746, 787, + 65, 787, 67, 65, 787, 65, 787, 69, + 65, 787, 70, 65, 787, 65, 790, 790, + 73, 76, 790, 74, 790, 75, 790, 790, + 790, 77, 790, 78, 790, 79, 790, 80, + 790, 790, 790, 82, 790, 83, 790, 84, + 790, 85, 790, 790, 790, 87, 92, 94, + 790, 88, 790, 89, 790, 90, 790, 91, + 790, 790, 790, 93, 790, 790, 790, 95, + 790, 790, 790, 97, 790, 98, 790, 99, + 790, 100, 790, 790, 790, 102, 790, 103, + 790, 104, 790, 105, 790, 106, 790, 790, + 790, 108, 790, 109, 111, 790, 110, 790, + 790, 790, 112, 790, 113, 790, 790, 790, + 115, 120, 790, 116, 790, 117, 790, 118, + 790, 119, 790, 790, 790, 121, 790, 790, + 790, 123, 139, 790, 124, 790, 125, 790, + 126, 790, 127, 790, 128, 790, 129, 790, + 130, 790, 131, 790, 132, 790, 133, 790, + 134, 790, 135, 790, 136, 790, 137, 790, + 138, 790, 790, 790, 140, 790, 141, 790, + 790, 790, 143, 144, 790, 790, 790, 145, + 790, 146, 790, 147, 790, 148, 790, 790, + 790, 150, 790, 151, 790, 790, 790, 153, + 790, 154, 790, 790, 790, 156, 790, 157, + 790, 158, 790, 159, 790, 160, 790, 161, + 790, 790, 790, 163, 167, 790, 164, 790, + 165, 790, 166, 790, 790, 790, 168, 790, + 169, 790, 170, 790, 171, 790, 790, 790, + 173, 177, 790, 174, 790, 175, 790, 176, + 790, 790, 790, 178, 790, 179, 790, 180, + 790, 181, 790, 182, 790, 183, 790, 790, + 790, 185, 790, 186, 790, 187, 790, 188, + 790, 189, 790, 190, 790, 191, 790, 192, + 790, 193, 790, 194, 790, 195, 790, 196, + 790, 197, 790, 198, 790, 199, 790, 200, + 790, 201, 790, 790, 790, 203, 790, 204, + 790, 205, 790, 206, 790, 207, 790, 790, + 790, 209, 790, 210, 790, 211, 790, 212, + 790, 213, 790, 790, 790, 215, 790, 216, + 790, 217, 790, 218, 790, 219, 790, 220, + 790, 221, 790, 790, 790, 223, 790, 224, + 790, 225, 790, 790, 790, 227, 790, 228, + 790, 790, 790, 230, 235, 790, 231, 790, + 232, 790, 233, 790, 234, 790, 790, 790, + 236, 790, 237, 790, 238, 790, 239, 790, + 790, 790, 799, 790, 242, 790, 790, 790, + 244, 790, 245, 790, 790, 790, 247, 790, + 248, 790, 249, 790, 790, 790, 251, 790, + 252, 790, 253, 790, 254, 790, 255, 790, + 790, 790, 257, 790, 258, 790, 259, 790, + 260, 790, 261, 790, 262, 790, 263, 790, + 264, 790, 265, 790, 266, 790, 267, 790, + 268, 790, 269, 790, 790, 790, 271, 277, + 790, 272, 790, 273, 790, 274, 790, 275, + 790, 276, 790, 790, 790, 278, 790, 279, + 790, 280, 790, 281, 790, 282, 790, 283, + 790, 284, 790, 285, 790, 286, 790, 287, + 790, 288, 790, 289, 790, 290, 790, 291, + 295, 790, 292, 790, 293, 790, 294, 790, + 790, 790, 296, 790, 297, 790, 298, 790, + 299, 790, 790, 790, 301, 790, 302, 790, + 303, 790, 304, 790, 305, 790, 790, 790, + 307, 310, 314, 319, 790, 308, 790, 309, + 790, 790, 790, 311, 790, 312, 790, 313, + 790, 790, 790, 315, 790, 316, 790, 317, + 790, 318, 790, 790, 790, 320, 790, 321, + 790, 322, 790, 323, 790, 790, 790, 325, + 332, 790, 326, 790, 327, 790, 328, 790, + 329, 790, 330, 790, 331, 790, 790, 790, + 333, 790, 790, 790, 790, 335, 790, 336, + 790, 790, 790, 338, 790, 339, 790, 340, + 790, 790, 790, 342, 344, 349, 790, 343, + 790, 790, 790, 345, 790, 346, 790, 347, + 790, 348, 790, 790, 790, 790, 790, 351, + 354, 790, 352, 790, 353, 790, 790, 790, + 355, 790, 356, 790, 790, 790, 358, 364, + 790, 359, 790, 360, 790, 361, 790, 362, + 790, 363, 790, 790, 790, 365, 790, 366, + 790, 367, 790, 790, 790, 369, 790, 370, + 790, 371, 790, 372, 790, 373, 790, 374, + 790, 375, 790, 376, 790, 790, 790, 378, + 790, 379, 790, 380, 790, 381, 790, 382, + 790, 383, 790, 790, 790, 385, 790, 386, + 790, 387, 790, 388, 790, 790, 790, 390, + 790, 391, 790, 392, 790, 393, 790, 394, + 790, 395, 790, 396, 790, 397, 790, 790, + 790, 790, 790, 400, 790, 401, 790, 790, + 790, 403, 408, 790, 404, 790, 405, 790, + 406, 790, 407, 790, 790, 790, 409, 790, + 410, 415, 421, 433, 790, 411, 790, 412, + 790, 413, 790, 414, 790, 790, 790, 416, + 790, 417, 790, 418, 790, 419, 790, 420, + 790, 790, 790, 422, 790, 423, 790, 424, + 790, 425, 790, 426, 790, 427, 790, 428, + 790, 429, 790, 430, 790, 431, 790, 432, + 790, 790, 790, 434, 790, 435, 790, 436, + 790, 437, 790, 790, 790, 439, 790, 440, + 790, 790, 790, 442, 790, 443, 790, 444, + 790, 445, 790, 790, 790, 447, 452, 790, + 448, 790, 449, 790, 450, 790, 451, 790, + 790, 790, 453, 790, 454, 790, 455, 790, + 456, 790, 457, 790, 458, 790, 790, 790, + 460, 790, 461, 790, 462, 790, 790, 790, + 464, 790, 465, 790, 790, 790, 467, 473, + 790, 468, 790, 469, 790, 470, 790, 471, + 790, 472, 790, 790, 790, 474, 790, 475, + 790, 476, 790, 477, 790, 478, 790, 479, + 790, 790, 790, 481, 790, 482, 790, 483, + 790, 484, 790, 790, 790, 486, 790, 487, + 790, 488, 790, 489, 790, 790, 790, 491, + 790, 492, 790, 493, 790, 494, 790, 495, + 790, 496, 790, 790, 790, 498, 507, 790, + 499, 790, 500, 790, 501, 790, 502, 790, + 503, 790, 504, 790, 505, 790, 506, 790, + 790, 790, 508, 790, 509, 790, 790, 790, + 511, 519, 528, 790, 512, 515, 790, 513, + 790, 514, 790, 790, 790, 516, 790, 517, + 790, 518, 790, 790, 790, 520, 790, 521, + 522, 525, 790, 790, 790, 523, 790, 524, + 790, 790, 790, 526, 790, 527, 790, 790, + 790, 529, 790, 790, 790, 531, 790, 532, + 790, 533, 790, 790, 790, 535, 790, 536, + 790, 790, 537, 790, 790, 790, 539, 543, + 790, 540, 790, 541, 790, 542, 790, 790, + 790, 544, 790, 545, 790, 546, 790, 547, + 790, 790, 790, 549, 790, 550, 790, 551, + 790, 552, 790, 553, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 560, 0, 562, 561, 817, 0, 0, + 561, 564, 565, 819, 563, 564, 565, 819, + 563, 564, 565, 563, 568, 569, 819, 570, + 651, 666, 672, 678, 684, 690, 696, 707, + 713, 719, 724, 567, 568, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 567, 568, + 569, 819, 571, 586, 592, 598, 604, 610, + 616, 627, 633, 639, 644, 567, 568, 569, + 819, 572, 581, 567, 568, 569, 819, 573, + 577, 567, 568, 569, 819, 574, 567, 568, + 569, 819, 575, 567, 576, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 578, + 567, 568, 569, 819, 579, 567, 580, 569, + 819, 567, 568, 569, 819, 567, 568, 569, + 819, 582, 567, 568, 569, 819, 583, 567, + 568, 569, 819, 584, 567, 585, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 587, 567, 568, 569, 819, 588, 567, 568, + 569, 819, 589, 567, 568, 569, 819, 590, + 567, 591, 569, 819, 567, 568, 569, 819, + 567, 568, 569, 819, 593, 567, 568, 569, + 819, 594, 567, 568, 569, 819, 595, 567, + 568, 569, 819, 596, 567, 597, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 599, 567, 568, 569, 819, 600, 567, 568, + 569, 819, 601, 567, 568, 569, 819, 602, + 567, 603, 569, 819, 567, 568, 569, 819, + 567, 568, 569, 819, 605, 567, 568, 569, + 819, 606, 567, 568, 569, 819, 607, 567, + 568, 569, 819, 608, 567, 609, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 611, 567, 568, 569, 819, 612, 567, 568, + 569, 819, 613, 567, 568, 569, 819, 614, + 567, 615, 569, 819, 567, 568, 569, 819, + 567, 568, 569, 819, 617, 622, 567, 568, + 569, 819, 618, 567, 568, 569, 819, 619, + 567, 568, 569, 819, 620, 567, 621, 569, + 819, 567, 568, 569, 819, 567, 568, 569, + 819, 623, 567, 568, 569, 819, 624, 567, + 568, 569, 819, 625, 567, 626, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 628, 567, 568, 569, 819, 629, 567, 568, + 569, 819, 630, 567, 568, 569, 819, 631, + 567, 632, 569, 819, 567, 568, 569, 819, + 567, 568, 569, 819, 634, 567, 568, 569, + 819, 635, 567, 568, 569, 819, 636, 567, + 568, 569, 819, 637, 567, 638, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 640, 567, 568, 569, 819, 641, 567, 568, + 569, 819, 642, 567, 643, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 645, + 567, 568, 569, 819, 646, 567, 568, 569, + 819, 647, 567, 568, 569, 819, 648, 567, + 568, 569, 819, 649, 567, 650, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 652, 661, 567, 568, 569, 819, 653, 657, + 567, 568, 569, 819, 654, 567, 568, 569, + 819, 655, 567, 656, 569, 819, 567, 568, + 569, 819, 567, 568, 569, 819, 658, 567, + 568, 569, 819, 659, 567, 660, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 662, 567, 568, 569, 819, 663, 567, 568, + 569, 819, 664, 567, 665, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 667, + 567, 568, 569, 819, 668, 567, 568, 569, + 819, 669, 567, 568, 569, 819, 670, 567, + 671, 569, 819, 567, 568, 569, 819, 567, + 568, 569, 819, 673, 567, 568, 569, 819, + 674, 567, 568, 569, 819, 675, 567, 568, + 569, 819, 676, 567, 677, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 679, + 567, 568, 569, 819, 680, 567, 568, 569, + 819, 681, 567, 568, 569, 819, 682, 567, + 683, 569, 819, 567, 568, 569, 819, 567, + 568, 569, 819, 685, 567, 568, 569, 819, + 686, 567, 568, 569, 819, 687, 567, 568, + 569, 819, 688, 567, 689, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 691, + 567, 568, 569, 819, 692, 567, 568, 569, + 819, 693, 567, 568, 569, 819, 694, 567, + 695, 569, 819, 567, 568, 569, 819, 567, + 568, 569, 819, 697, 702, 567, 568, 569, + 819, 698, 567, 568, 569, 819, 699, 567, + 568, 569, 819, 700, 567, 701, 569, 819, + 567, 568, 569, 819, 567, 568, 569, 819, + 703, 567, 568, 569, 819, 704, 567, 568, + 569, 819, 705, 567, 706, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 708, + 567, 568, 569, 819, 709, 567, 568, 569, + 819, 710, 567, 568, 569, 819, 711, 567, + 712, 569, 819, 567, 568, 569, 819, 567, + 568, 569, 819, 714, 567, 568, 569, 819, + 715, 567, 568, 569, 819, 716, 567, 568, + 569, 819, 717, 567, 718, 569, 819, 567, + 568, 569, 819, 567, 568, 569, 819, 720, + 567, 568, 569, 819, 721, 567, 568, 569, + 819, 722, 567, 723, 569, 819, 567, 568, + 569, 819, 567, 568, 569, 819, 725, 567, + 568, 569, 819, 726, 567, 568, 569, 819, + 727, 567, 568, 569, 819, 728, 567, 568, + 569, 819, 729, 567, 730, 569, 819, 567, + 568, 569, 819, 567, 732, 733, 819, 731, + 732, 733, 819, 731, 732, 733, 731, 735, + 819, 819, 735, 819, 819, 736, 736, 736, + 819, 819, 819, 819, 819, 739, 819, 819, + 819, 819, 838, 838, 838, 838, 742, 838, + 838, 838, 838, 843, 843, 843, 843, 745, + 843, 843, 843, 843, 746, 746, 746, 746, + 746, 747, 746, 759, 760, 746, 761, 762, + 763, 746, 782, 746, 746, 746, 746, 746, + 784, 746, 785, 746, 786, 746, 746, 746, + 1, 2, 746, 3, 3, 3, 3, 746, + 4, 4, 4, 4, 746, 5, 8, 11, + 13, 15, 8, 9, 10, 8, 8, 746, + 19, 746, 20, 20, 20, 20, 19, 746, + 746, 19, 746, 746, 746, 21, 21, 21, + 21, 746, 746, 746, 746, 22, 746, 23, + 24, 26, 746, 746, 27, 746, 28, 28, + 28, 28, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 29, 32, 35, 746, + 764, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 769, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 770, 746, + 746, 746, 771, 746, 776, 746, 746, 777, + 778, 746, 746, 746, 746, 746, 746, 779, + 746, 766, 768, 746, 765, 746, 746, 746, + 767, 768, 746, 768, 768, 746, 768, 746, + 746, 746, 746, 38, 39, 40, 41, 774, + 746, 773, 746, 746, 746, 775, 746, 746, + 746, 50, 52, 54, 746, 56, 746, 746, + 746, 781, 780, 780, 780, 746, 746, 746, + 746, 746, 58, 58, 58, 746, 59, 746, + 746, 746, 746, 746, 746, 746, 746, 62, + 746, 746, 63, 746, 787, 789, 788, 787, + 65, 787, 66, 68, 65, 791, 792, 793, + 795, 796, 797, 798, 800, 801, 802, 803, + 804, 806, 807, 808, 809, 810, 811, 812, + 813, 814, 815, 816, 790, 71, 72, 81, + 790, 86, 96, 101, 107, 114, 790, 122, + 790, 790, 142, 790, 794, 790, 155, 162, + 790, 149, 152, 790, 172, 790, 184, 202, + 790, 208, 214, 222, 226, 229, 790, 240, + 246, 250, 790, 241, 243, 790, 256, 270, + 790, 300, 790, 306, 324, 790, 790, 334, + 337, 341, 790, 790, 790, 790, 790, 350, + 790, 357, 790, 805, 790, 377, 384, 790, + 368, 790, 790, 389, 398, 790, 790, 790, + 399, 402, 438, 441, 790, 790, 790, 790, + 790, 446, 790, 790, 790, 790, 459, 463, + 790, 466, 790, 480, 485, 790, 790, 790, + 490, 497, 790, 510, 530, 534, 538, 790, + 548, 790, 554, 790, 555, 556, 557, 558, + 790, 790, 790, 790, 790, 790, 790, 0, + 818, 818, 818, 818, 818, 818, 818, 818, + 819, 820, 821, 819, 819, 819, 819, 833, + 819, 834, 819, 835, 819, 819, 819, 563, + 566, 731, 819, 822, 819, 819, 819, 819, + 819, 826, 819, 819, 819, 819, 819, 819, + 819, 827, 819, 819, 819, 819, 819, 819, + 819, 828, 829, 819, 819, 819, 819, 819, + 819, 830, 824, 819, 819, 819, 819, 823, + 819, 819, 819, 825, 819, 819, 819, 819, + 819, 819, 734, 819, 819, 819, 832, 831, + 831, 831, 819, 819, 819, 819, 819, 736, + 736, 736, 819, 819, 819, 819, 819, 737, + 819, 819, 738, 819, 837, 836, 836, 836, + 836, 836, 836, 836, 836, 839, 838, 838, + 838, 840, 838, 841, 838, 842, 838, 838, + 838, 838, 838, 838, 838, 838, 838, 740, + 838, 838, 741, 838, 844, 843, 843, 843, + 845, 843, 846, 843, 847, 843, 843, 843, + 843, 843, 843, 843, 843, 843, 743, 843, + 843, 744, 843, 848, 848, 849, 849, 0, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 787, 787, 787, 787, 787, 787, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 559, 560, + 561, 562, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 838, 838, 838, 843, 843, + 843, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 787, 787, 787, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 817, 818, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 836, 836, 838, 838, 838, + 838, 838, 843, 843, 843, 843, 843, 848, + 849, 0 + }; + + static const short _regex_cond_actions[] = { + 827, 631, 731, 723, 45, 903, 903, 897, + 45, 912, 900, 903, 729, 741, 0, 45, + 45, 923, 923, 923, 923, 737, 45, 765, + 747, 0, 0, 0, 0, 841, 743, 3, + 3, 3, 3, 841, 3, 3, 3, 3, + 839, 0, 3, 3, 3, 3, 839, 761, + 839, 761, 3, 3, 3, 3, 839, 759, + 3, 870, 3, 3, 839, 761, 3, 870, + 3, 3, 839, 3, 3, 3, 3, 839, + 0, 3, 3, 3, 3, 839, 751, 0, + 749, 839, 755, 753, 839, 0, 757, 3, + 3, 3, 3, 839, 0, 0, 0, 0, + 839, 757, 0, 0, 0, 0, 839, 757, + 3, 3, 3, 3, 839, 745, 0, 841, + 725, 727, 27, 27, 27, 27, 841, 743, + 3, 3, 3, 3, 841, 763, 0, 841, + 3, 3, 3, 3, 841, 3, 3, 3, + 3, 841, 673, 3, 3, 3, 3, 841, + 0, 0, 0, 0, 841, 27, 27, 27, + 27, 841, 725, 0, 727, 25, 25, 25, + 25, 841, 0, 0, 829, 0, 0, 0, + 603, 0, 0, 0, 0, 0, 0, 829, + 0, 0, 0, 601, 0, 0, 0, 0, + 0, 0, 829, 0, 0, 0, 603, 0, + 0, 0, 0, 675, 0, 17, 831, 675, + 0, 7, 3, 870, 3, 3, 831, 17, + 831, 663, 17, 831, 663, 17, 831, 663, + 831, 3, 661, 870, 3, 3, 831, 3, + 661, 870, 3, 3, 831, 3, 661, 3, + 3, 3, 831, 3, 665, 3, 3, 3, + 831, 3, 3, 3, 3, 837, 671, 3, + 3, 3, 3, 837, 3, 3, 3, 3, + 837, 669, 3, 3, 3, 3, 837, 3, + 3, 3, 3, 837, 3, 667, 3, 3, + 3, 837, 0, 833, 677, 0, 833, 679, + 0, 0, 0, 835, 0, 29, 11, 845, + 31, 13, 845, 0, 13, 845, 843, 769, + 843, 843, 0, 843, 843, 771, 843, 53, + 0, 53, 0, 0, 51, 0, 53, 0, + 0, 49, 0, 0, 47, 0, 315, 359, + 0, 0, 359, 0, 359, 0, 359, 127, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 129, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 131, 359, 0, 0, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 133, 359, 0, 359, 135, 359, 0, + 359, 137, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 139, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 141, + 359, 0, 359, 0, 0, 359, 0, 359, + 143, 359, 0, 359, 0, 359, 145, 359, + 0, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 147, 359, 0, 359, 149, + 359, 0, 0, 341, 0, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 0, 341, + 0, 341, 151, 341, 0, 341, 0, 341, + 153, 341, 0, 0, 341, 155, 341, 0, + 341, 0, 341, 0, 341, 0, 341, 157, + 341, 0, 343, 0, 343, 159, 343, 0, + 343, 0, 343, 161, 343, 0, 341, 0, + 341, 0, 341, 0, 341, 0, 341, 0, + 341, 163, 341, 0, 0, 341, 0, 341, + 0, 341, 0, 341, 165, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 167, 341, + 0, 0, 359, 0, 359, 0, 359, 0, + 359, 169, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 171, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 173, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 175, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 177, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 179, 359, 0, 359, 0, + 359, 0, 359, 181, 359, 0, 359, 0, + 359, 183, 359, 0, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 185, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 187, 359, 45, 359, 0, 357, 189, 357, + 0, 357, 0, 357, 191, 357, 0, 359, + 0, 359, 0, 359, 193, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 195, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 197, 359, 0, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 199, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 0, 359, 0, 359, 0, 359, 0, 359, + 201, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 203, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 205, 359, + 0, 0, 0, 0, 359, 0, 359, 0, + 359, 207, 359, 0, 359, 0, 359, 0, + 359, 209, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 211, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 213, 359, 0, + 0, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 215, 359, + 0, 359, 217, 359, 219, 0, 345, 0, + 345, 221, 345, 0, 345, 0, 345, 0, + 345, 223, 345, 0, 0, 0, 345, 0, + 345, 225, 345, 0, 345, 0, 345, 0, + 345, 0, 345, 227, 345, 229, 345, 0, + 0, 345, 0, 345, 0, 345, 231, 345, + 0, 345, 0, 345, 233, 345, 0, 0, + 347, 0, 347, 0, 347, 0, 347, 0, + 347, 0, 347, 235, 347, 0, 347, 0, + 347, 0, 347, 237, 347, 0, 349, 0, + 349, 0, 349, 0, 349, 0, 349, 0, + 349, 0, 349, 0, 349, 239, 349, 0, + 347, 0, 347, 0, 347, 0, 347, 0, + 347, 0, 347, 241, 347, 0, 347, 0, + 347, 0, 347, 0, 347, 243, 347, 0, + 351, 0, 351, 0, 351, 0, 351, 0, + 351, 0, 351, 0, 351, 0, 351, 245, + 351, 247, 351, 0, 359, 0, 359, 249, + 359, 0, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 251, 359, 0, 359, + 0, 0, 0, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 253, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 255, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 257, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 259, 359, 0, 359, 0, + 359, 261, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 263, 359, 0, 0, 353, + 0, 353, 0, 353, 0, 353, 0, 353, + 265, 353, 0, 353, 0, 353, 0, 353, + 0, 353, 0, 353, 0, 353, 267, 353, + 0, 359, 0, 359, 0, 359, 269, 359, + 0, 359, 0, 359, 271, 359, 0, 0, + 355, 0, 355, 0, 355, 0, 355, 0, + 355, 0, 355, 273, 355, 0, 355, 0, + 355, 0, 355, 0, 355, 0, 355, 0, + 355, 275, 355, 0, 355, 0, 355, 0, + 355, 0, 355, 277, 355, 0, 355, 0, + 355, 0, 355, 0, 355, 279, 355, 0, + 355, 0, 355, 0, 355, 0, 355, 0, + 355, 0, 355, 281, 355, 0, 0, 355, + 0, 355, 0, 355, 0, 355, 0, 355, + 0, 355, 0, 355, 0, 355, 0, 355, + 283, 355, 0, 355, 0, 355, 285, 355, + 0, 0, 0, 359, 0, 0, 359, 0, + 359, 0, 359, 287, 359, 0, 359, 0, + 359, 0, 359, 289, 359, 0, 359, 0, + 0, 0, 359, 291, 359, 0, 359, 0, + 359, 293, 359, 0, 359, 0, 359, 295, + 359, 0, 359, 297, 359, 0, 359, 0, + 359, 0, 359, 299, 359, 0, 359, 0, + 303, 359, 0, 359, 301, 359, 0, 0, + 359, 0, 359, 0, 359, 0, 359, 305, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 307, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 309, 359, 311, + 359, 119, 359, 121, 359, 123, 359, 125, + 359, 0, 39, 33, 35, 37, 39, 39, + 35, 0, 0, 539, 0, 0, 0, 377, + 0, 0, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 539, 0, + 0, 0, 435, 0, 0, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 381, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 385, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 389, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 393, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 397, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 401, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 405, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 409, + 0, 0, 0, 539, 0, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 413, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 417, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 421, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 425, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 429, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 433, 0, 0, 0, 539, + 0, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 379, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 383, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 387, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 391, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 395, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 399, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 403, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 407, 0, + 0, 0, 539, 0, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 411, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 415, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 419, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 423, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 427, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 431, 0, 0, 0, 539, 0, + 0, 0, 377, 0, 0, 0, 0, 0, + 533, 471, 0, 533, 475, 0, 0, 0, + 535, 537, 503, 537, 537, 0, 537, 537, + 505, 537, 569, 557, 569, 569, 0, 569, + 569, 559, 569, 587, 575, 587, 587, 0, + 587, 587, 577, 587, 633, 775, 777, 629, + 637, 45, 597, 0, 0, 609, 0, 45, + 0, 635, 909, 599, 775, 777, 773, 777, + 0, 777, 45, 777, 45, 777, 773, 777, + 0, 0, 779, 0, 0, 0, 0, 821, + 3, 3, 3, 3, 821, 1, 3, 1, + 0, 3, 3, 855, 855, 3, 3, 819, + 0, 821, 27, 27, 27, 27, 0, 821, + 745, 0, 821, 735, 733, 3, 3, 3, + 3, 821, 739, 821, 763, 0, 821, 1, + 1, 0, 821, 725, 0, 727, 25, 25, + 25, 25, 821, 615, 613, 783, 619, 617, + 785, 623, 621, 787, 0, 0, 0, 781, + 5, 639, 647, 611, 695, 607, 717, 699, + 715, 683, 683, 0, 605, 713, 691, 683, + 703, 687, 719, 641, 657, 645, 0, 693, + 659, 655, 906, 697, 45, 683, 651, 45, + 0, 653, 689, 649, 683, 701, 685, 7, + 643, 852, 915, 721, 15, 791, 867, 791, + 858, 919, 795, 927, 919, 793, 919, 847, + 709, 711, 681, 0, 7, 0, 849, 17, + 801, 17, 799, 876, 799, 17, 797, 873, + 797, 1, 1, 1, 815, 0, 803, 705, + 707, 45, 19, 23, 21, 805, 882, 894, + 888, 805, 0, 0, 0, 807, 11, 825, + 627, 625, 789, 823, 767, 823, 823, 0, + 823, 823, 0, 823, 55, 45, 45, 53, + 0, 53, 0, 0, 0, 45, 45, 45, + 45, 45, 45, 45, 45, 45, 45, 45, + 45, 45, 45, 45, 45, 45, 45, 45, + 45, 45, 0, 0, 317, 0, 0, 0, + 339, 0, 0, 0, 0, 0, 339, 0, + 61, 63, 0, 65, 45, 67, 0, 0, + 319, 0, 0, 321, 0, 339, 0, 0, + 339, 0, 0, 0, 0, 0, 339, 0, + 0, 0, 339, 0, 0, 337, 0, 0, + 339, 0, 339, 0, 0, 339, 79, 0, + 0, 0, 69, 71, 73, 75, 77, 0, + 323, 0, 81, 45, 83, 0, 0, 325, + 0, 327, 85, 0, 0, 87, 89, 329, + 0, 0, 0, 0, 339, 91, 93, 95, + 97, 0, 99, 101, 103, 331, 0, 0, + 339, 0, 105, 0, 0, 107, 109, 111, + 0, 0, 333, 0, 0, 0, 0, 339, + 0, 339, 0, 339, 0, 0, 0, 0, + 339, 313, 339, 113, 115, 117, 335, 39, + 361, 363, 365, 367, 369, 371, 373, 375, + 491, 45, 0, 511, 509, 507, 509, 0, + 509, 45, 509, 45, 509, 507, 509, 0, + 0, 0, 531, 5, 489, 439, 457, 493, + 493, 0, 437, 485, 493, 461, 481, 451, + 441, 0, 487, 453, 449, 495, 455, 493, + 445, 45, 0, 447, 483, 443, 493, 459, + 479, 7, 9, 473, 497, 497, 499, 15, + 517, 861, 517, 15, 519, 864, 519, 467, + 469, 477, 0, 521, 463, 465, 45, 19, + 23, 21, 523, 879, 891, 885, 523, 0, + 0, 0, 525, 529, 501, 529, 529, 0, + 529, 529, 0, 529, 0, 549, 543, 549, + 541, 549, 547, 545, 551, 0, 563, 561, + 563, 0, 563, 45, 563, 45, 563, 561, + 563, 553, 567, 565, 555, 565, 565, 0, + 565, 565, 0, 565, 0, 581, 579, 581, + 0, 581, 45, 581, 45, 581, 579, 581, + 571, 585, 583, 573, 583, 583, 0, 583, + 583, 0, 583, 589, 591, 593, 595, 0, + 827, 827, 841, 841, 839, 839, 839, 839, + 839, 839, 839, 839, 839, 839, 839, 839, + 839, 839, 841, 841, 841, 841, 841, 841, + 841, 841, 841, 841, 829, 829, 829, 829, + 829, 829, 829, 829, 829, 831, 831, 831, + 831, 831, 831, 831, 831, 831, 831, 831, + 831, 837, 837, 837, 837, 837, 837, 833, + 833, 835, 845, 845, 845, 843, 843, 843, + 59, 59, 59, 59, 59, 59, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 343, 343, 343, 343, + 343, 343, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 357, 357, 357, 357, 357, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 345, 345, 345, + 345, 345, 345, 345, 345, 345, 345, 345, + 345, 345, 345, 345, 345, 345, 345, 345, + 345, 345, 345, 345, 347, 347, 347, 347, + 347, 347, 347, 347, 347, 347, 347, 349, + 349, 349, 349, 349, 349, 349, 349, 349, + 347, 347, 347, 347, 347, 347, 347, 347, + 347, 347, 347, 347, 351, 351, 351, 351, + 351, 351, 351, 351, 351, 351, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 353, 353, 353, + 353, 353, 353, 353, 353, 353, 353, 353, + 353, 353, 359, 359, 359, 359, 359, 359, + 359, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 39, 39, + 39, 39, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 533, 533, 535, + 537, 537, 537, 569, 569, 569, 587, 587, + 587, 0, 779, 821, 821, 819, 821, 821, + 821, 821, 821, 821, 821, 821, 783, 785, + 787, 781, 817, 791, 791, 795, 793, 847, + 811, 809, 801, 799, 799, 797, 797, 815, + 803, 813, 805, 805, 807, 825, 789, 823, + 823, 823, 0, 57, 57, 0, 339, 339, + 319, 321, 339, 339, 339, 339, 337, 339, + 339, 339, 323, 325, 327, 329, 339, 331, + 339, 333, 339, 339, 339, 339, 339, 335, + 0, 0, 0, 531, 531, 517, 517, 519, + 519, 513, 527, 521, 515, 523, 523, 525, + 529, 529, 529, 0, 551, 0, 567, 565, + 565, 565, 0, 585, 583, 583, 583, 0, + 0, 0 + }; + + static const signed char _regex_to_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 41, + 0, 41, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 41, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 41, 0, 0, 41, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 41, 41, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 41, 0, 41, 0, + 0, 0, 0, 41, 0, 0, 0, 0, + 41, 41, 0 + }; + + static const signed char _regex_from_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 43, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 43, 0, 0, 43, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 43, 43, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 43, 0, 43, 0, + 0, 0, 0, 43, 0, 0, 0, 0, + 43, 43, 0 + }; + + static const short _regex_eof_trans[] = { + 2665, 2666, 2667, 2668, 2669, 2670, 2671, 2672, + 2673, 2674, 2675, 2676, 2677, 2678, 2679, 2680, + 2681, 2682, 2683, 2684, 2685, 2686, 2687, 2688, + 2689, 2690, 2691, 2692, 2693, 2694, 2695, 2696, + 2697, 2698, 2699, 2700, 2701, 2702, 2703, 2704, + 2705, 2706, 2707, 2708, 2709, 2710, 2711, 2712, + 2713, 2714, 2715, 2716, 2717, 2718, 2719, 2720, + 2721, 2722, 2723, 2724, 2725, 2726, 2727, 2728, + 2729, 2730, 2731, 2732, 2733, 2734, 2735, 2736, + 2737, 2738, 2739, 2740, 2741, 2742, 2743, 2744, + 2745, 2746, 2747, 2748, 2749, 2750, 2751, 2752, + 2753, 2754, 2755, 2756, 2757, 2758, 2759, 2760, + 2761, 2762, 2763, 2764, 2765, 2766, 2767, 2768, + 2769, 2770, 2771, 2772, 2773, 2774, 2775, 2776, + 2777, 2778, 2779, 2780, 2781, 2782, 2783, 2784, + 2785, 2786, 2787, 2788, 2789, 2790, 2791, 2792, + 2793, 2794, 2795, 2796, 2797, 2798, 2799, 2800, + 2801, 2802, 2803, 2804, 2805, 2806, 2807, 2808, + 2809, 2810, 2811, 2812, 2813, 2814, 2815, 2816, + 2817, 2818, 2819, 2820, 2821, 2822, 2823, 2824, + 2825, 2826, 2827, 2828, 2829, 2830, 2831, 2832, + 2833, 2834, 2835, 2836, 2837, 2838, 2839, 2840, + 2841, 2842, 2843, 2844, 2845, 2846, 2847, 2848, + 2849, 2850, 2851, 2852, 2853, 2854, 2855, 2856, + 2857, 2858, 2859, 2860, 2861, 2862, 2863, 2864, + 2865, 2866, 2867, 2868, 2869, 2870, 2871, 2872, + 2873, 2874, 2875, 2876, 2877, 2878, 2879, 2880, + 2881, 2882, 2883, 2884, 2885, 2886, 2887, 2888, + 2889, 2890, 2891, 2892, 2893, 2894, 2895, 2896, + 2897, 2898, 2899, 2900, 2901, 2902, 2903, 2904, + 2905, 2906, 2907, 2908, 2909, 2910, 2911, 2912, + 2913, 2914, 2915, 2916, 2917, 2918, 2919, 2920, + 2921, 2922, 2923, 2924, 2925, 2926, 2927, 2928, + 2929, 2930, 2931, 2932, 2933, 2934, 2935, 2936, + 2937, 2938, 2939, 2940, 2941, 2942, 2943, 2944, + 2945, 2946, 2947, 2948, 2949, 2950, 2951, 2952, + 2953, 2954, 2955, 2956, 2957, 2958, 2959, 2960, + 2961, 2962, 2963, 2964, 2965, 2966, 2967, 2968, + 2969, 2970, 2971, 2972, 2973, 2974, 2975, 2976, + 2977, 2978, 2979, 2980, 2981, 2982, 2983, 2984, + 2985, 2986, 2987, 2988, 2989, 2990, 2991, 2992, + 2993, 2994, 2995, 2996, 2997, 2998, 2999, 3000, + 3001, 3002, 3003, 3004, 3005, 3006, 3007, 3008, + 3009, 3010, 3011, 3012, 3013, 3014, 3015, 3016, + 3017, 3018, 3019, 3020, 3021, 3022, 3023, 3024, + 3025, 3026, 3027, 3028, 3029, 3030, 3031, 3032, + 3033, 3034, 3035, 3036, 3037, 3038, 3039, 3040, + 3041, 3042, 3043, 3044, 3045, 3046, 3047, 3048, + 3049, 3050, 3051, 3052, 3053, 3054, 3055, 3056, + 3057, 3058, 3059, 3060, 3061, 3062, 3063, 3064, + 3065, 3066, 3067, 3068, 3069, 3070, 3071, 3072, + 3073, 3074, 3075, 3076, 3077, 3078, 3079, 3080, + 3081, 3082, 3083, 3084, 3085, 3086, 3087, 3088, + 3089, 3090, 3091, 3092, 3093, 3094, 3095, 3096, + 3097, 3098, 3099, 3100, 3101, 3102, 3103, 3104, + 3105, 3106, 3107, 3108, 3109, 3110, 3111, 3112, + 3113, 3114, 3115, 3116, 3117, 3118, 3119, 3120, + 3121, 3122, 3123, 3124, 3125, 3126, 3127, 3128, + 3129, 3130, 3131, 3132, 3133, 3134, 3135, 3136, + 3137, 3138, 3139, 3140, 3141, 3142, 3143, 3144, + 3145, 3146, 3147, 3148, 3149, 3150, 3151, 3152, + 3153, 3154, 3155, 3156, 3157, 3158, 3159, 3160, + 3161, 3162, 3163, 3164, 3165, 3166, 3167, 3168, + 3169, 3170, 3171, 3172, 3173, 3174, 3175, 3176, + 3177, 3178, 3179, 3180, 3181, 3182, 3183, 3184, + 3185, 3186, 3187, 3188, 3189, 3190, 3191, 3192, + 3193, 3194, 3195, 3196, 3197, 3198, 3199, 3200, + 3201, 3202, 3203, 3204, 3205, 3206, 3207, 3208, + 3209, 3210, 3211, 3212, 3213, 3214, 3215, 3216, + 3217, 3218, 3219, 3220, 3221, 3222, 3223, 3224, + 3225, 3226, 3227, 3228, 3229, 3230, 3231, 3232, + 3233, 3234, 3235, 3236, 3237, 3238, 3239, 3240, + 3241, 3242, 3243, 3244, 3245, 3246, 3247, 3248, + 3249, 3250, 3251, 3252, 3253, 3254, 3255, 3256, + 3257, 3258, 3259, 3260, 3261, 3262, 3263, 3264, + 3265, 3266, 3267, 3268, 3269, 3270, 3271, 3272, + 3273, 3274, 3275, 3276, 3277, 3278, 3279, 3280, + 3281, 3282, 3283, 3284, 3285, 3286, 3287, 3288, + 3289, 3290, 3291, 3292, 3293, 3294, 3295, 3296, + 3297, 3298, 3299, 3300, 3301, 3302, 3303, 3304, + 3305, 3306, 3307, 3308, 3309, 3310, 3311, 3312, + 3313, 3314, 3315, 3316, 3317, 3318, 3319, 3320, + 3321, 3322, 3323, 3324, 3325, 3326, 3327, 3328, + 3329, 3330, 3331, 3332, 3333, 3334, 3335, 3336, + 3337, 3338, 3339, 3340, 3341, 3342, 3343, 3344, + 3345, 3346, 3347, 3348, 3349, 3350, 3351, 3352, + 3353, 3354, 3355, 3356, 3357, 3358, 3359, 3360, + 3361, 3362, 3363, 3364, 3365, 3366, 3367, 3368, + 3369, 3370, 3371, 3372, 3373, 3374, 3375, 3376, + 3377, 3378, 3379, 3380, 3381, 3382, 3383, 3384, + 3385, 3386, 3387, 3388, 3389, 3390, 3391, 3392, + 3393, 3394, 3395, 3396, 3397, 3398, 3399, 3400, + 3401, 3402, 3403, 3404, 3405, 3406, 3407, 3408, + 3409, 3410, 3411, 3412, 3413, 3414, 3415, 3416, + 3417, 3418, 3419, 3420, 3421, 3422, 3423, 3424, + 3425, 3426, 3427, 3428, 3429, 3430, 3431, 3432, + 3433, 3434, 3435, 3436, 3437, 3438, 3439, 3440, + 3441, 3442, 3443, 3444, 3445, 3446, 3447, 3448, + 3449, 3450, 3451, 3452, 3453, 3454, 3455, 3456, + 3457, 3458, 3459, 3460, 3461, 3462, 3463, 3464, + 3465, 3466, 3467, 3468, 3469, 3470, 3471, 3472, + 3473, 3474, 3475, 3476, 3477, 3478, 3479, 3480, + 3481, 3482, 3483, 3484, 3485, 3486, 3487, 3488, + 3489, 3490, 3491, 3492, 3493, 3494, 3495, 3496, + 3497, 3498, 3499, 3500, 3501, 3502, 3503, 3504, + 3505, 3506, 3507, 3508, 3509, 3510, 3511, 3512, + 3513, 3514, 0 + }; + + static const int regex_start = 746; + static const int regex_error = 0; + + static const int regex_en_readVerb = 787; + static const int regex_en_readUCP = 790; + static const int regex_en_readBracedUCP = 559; + static const int regex_en_readUCPSingle = 818; + static const int regex_en_charClassGuts = 819; + static const int regex_en_readClass = 836; + static const int regex_en_readQuotedLiteral = 838; + static const int regex_en_readQuotedClass = 843; + static const int regex_en_readComment = 848; + static const int regex_en_readNewlineTerminatedComment = 849; + static const int regex_en_main = 746; + + +#line 1911 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" -static const char _regex_cond_offsets[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1, - 2, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 4, 5, 6, 7, 8, 9, - 10, 11, 12, 18, 18, 18, 18, 18, - 18, 18, 18, 18, 18, 18, 18, 18, - 18, 18, 18, 18, 18, 18, 18, 18, - 18, 18, 18, 18, 18, 18, 18, 18, - 18, 18, 18, 18, 18, 18, 18, 18, - 18, 19, 20, 21, 21, 21, 21, 21, - 21, 21, 21, 21, 21, 21, 21, 21, - 21, 21, 21, 21, 21, 21, 21, 21, - 21, 21, 21, 21, 21, 21, 21, 21, - 21, 21, 21, 21, 26, 26, 26, 26, - 26, 26, 26, 26, 26, 26, 26, 26, - 26, 26, 27, 28, 29, 31, 31, 36, - 36, 37, 38, 39, 44, 44, 45, 46, - 47, 47 -}; + + /** \brief Main parser call, returns root Component or nullptr. */ + unique_ptr parse(const char *ptr, ParseMode &globalMode) { + assert(ptr); + + const char *p = ptr; + const char *pe = ptr + strlen(ptr); + + // First, read the control verbs, set any global mode flags and move the + // ptr forward. + p = read_control_verbs(p, pe, 0, globalMode); + + const char *eof = pe; + int cs; + UNUSED int act; + int top; + vector stack; + const char *ts, *te; + unichar accumulator = 0; + unichar octAccumulator = 0; /* required as we are also accumulating for + * back ref when looking for octals */ + unsigned repeatN = 0; + unsigned repeatM = 0; + string label; + + ParseMode mode = globalMode; + ParseMode newMode; + + bool negated = false; + bool inComment = false; + + // Stack of sequences and flags used to store state when we enter + // sub-sequences. + vector sequences; + + // Index of the next capturing group. Note that zero is reserved for the + // root sequence. + unsigned groupIndex = 1; + + // Set storing group names that are currently in use. + flat_set groupNames; + + // Root sequence. + unique_ptr rootSeq = std::make_unique(); + rootSeq->setCaptureIndex(0); + + // Current sequence being appended to + ComponentSequence *currentSeq = rootSeq.get(); + + // The current character class being appended to. This is used as the + // accumulator for both character class and UCP properties. + unique_ptr currentCls; + + // True if the machine is currently inside a character class, i.e. square + // brackets [..]. + bool inCharClass = false; + + // True if the machine is inside a character class but it has not processed + // any "real" elements yet, i.e. it's still processing meta-characters like + // '^'. + bool inCharClassEarly = false; + + // Location at which the current character class began. + const char *currentClsBegin = p; + + // We throw exceptions on various parsing failures beyond this point: we + // use a try/catch block here to clean up our allocated memory before we + // re-throw the exception to the caller. + try { + // Embed the Ragel machine here -static const char _regex_cond_lengths[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1, 1, - 1, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 6, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 1, 1, 1, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 5, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1, 1, 1, 2, 0, 5, 0, - 1, 1, 1, 5, 0, 1, 1, 1, - 0, 0 -}; - -static const short _regex_cond_keys[] = { - -128, -65, -128, -65, -128, -65, -128, -65, - -128, -65, -128, -65, -128, -65, -128, -65, - -128, -65, -128, -65, -128, -65, -128, -65, - -128, -65, -64, -33, -32, -17, -16, -9, - -8, -1, 35, 35, -128, -65, -128, -65, - -128, -65, -128, -65, -64, -33, -32, -17, - -16, -9, -8, -1, -128, -65, -128, -65, - -128, -65, 93, 93, 94, 94, -128, -65, - -64, -33, -32, -17, -16, -9, -8, -1, - -128, -65, -128, -65, -128, -65, -128, -65, - -64, -33, -32, -17, -16, -9, -8, -1, - -128, -65, -128, -65, -128, -65, 0 -}; - -static const char _regex_cond_spaces[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2, 2, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0 -}; - -static const short _regex_key_offsets[] = { - 0, 0, 1, 23, 31, 39, 46, 54, - 55, 63, 71, 79, 86, 94, 97, 99, - 108, 115, 123, 131, 134, 140, 148, 151, - 158, 165, 173, 180, 184, 191, 194, 197, - 199, 202, 205, 207, 210, 213, 215, 216, - 218, 219, 227, 229, 232, 235, 236, 244, - 252, 260, 268, 275, 283, 290, 298, 305, - 313, 315, 318, 325, 329, 332, 335, 337, - 339, 341, 342, 344, 345, 347, 349, 350, - 351, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 369, - 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 392, 393, 394, - 395, 396, 397, 399, 400, 401, 402, 403, - 404, 405, 406, 408, 409, 410, 411, 412, - 413, 414, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 429, - 430, 431, 432, 433, 434, 435, 436, 437, - 438, 439, 440, 441, 442, 443, 444, 445, - 446, 447, 448, 450, 451, 452, 453, 454, - 455, 456, 457, 458, 459, 461, 462, 463, - 464, 465, 466, 467, 468, 469, 470, 471, - 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 510, 511, - 512, 513, 514, 515, 516, 517, 519, 520, - 521, 522, 523, 524, 525, 526, 527, 528, - 529, 530, 531, 532, 533, 534, 535, 536, - 537, 538, 539, 540, 541, 542, 543, 544, - 545, 546, 547, 548, 549, 550, 551, 552, - 553, 554, 555, 556, 557, 558, 559, 561, - 562, 563, 564, 565, 566, 567, 568, 569, - 570, 571, 572, 573, 574, 575, 576, 577, - 578, 579, 580, 582, 583, 584, 585, 586, - 587, 588, 589, 590, 591, 592, 593, 594, - 595, 596, 597, 601, 602, 603, 604, 605, - 606, 607, 608, 609, 610, 611, 612, 613, - 614, 615, 616, 617, 618, 620, 621, 622, - 623, 624, 625, 626, 627, 628, 629, 631, - 632, 633, 634, 635, 636, 637, 640, 641, - 642, 643, 644, 645, 646, 647, 648, 650, - 651, 652, 653, 654, 655, 656, 658, 659, - 660, 661, 662, 663, 664, 665, 666, 667, - 668, 669, 670, 671, 672, 673, 674, 675, - 676, 677, 678, 679, 680, 681, 682, 683, - 684, 685, 686, 687, 688, 689, 690, 691, - 692, 693, 694, 695, 696, 697, 698, 699, - 700, 701, 702, 704, 705, 706, 707, 708, - 709, 710, 714, 715, 716, 717, 718, 719, - 720, 721, 722, 723, 724, 725, 726, 727, - 728, 729, 730, 731, 732, 733, 734, 735, - 736, 737, 738, 739, 740, 741, 742, 743, - 744, 745, 746, 747, 748, 749, 750, 752, - 753, 754, 755, 756, 757, 758, 759, 760, - 761, 762, 763, 764, 765, 766, 767, 768, - 769, 770, 771, 773, 774, 775, 776, 777, - 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 788, 789, 790, 791, 792, 793, - 794, 795, 796, 797, 798, 799, 800, 801, - 802, 803, 805, 806, 807, 808, 809, 810, - 811, 812, 813, 814, 815, 816, 817, 820, - 822, 823, 824, 825, 826, 827, 828, 829, - 830, 833, 834, 835, 836, 837, 838, 839, - 840, 841, 842, 843, 844, 845, 846, 847, - 849, 850, 851, 853, 854, 855, 856, 857, - 858, 859, 860, 861, 862, 863, 864, 865, - 866, 867, 868, 869, 870, 871, 872, 873, - 874, 875, 876, 877, 880, 883, 885, 900, - 903, 906, 908, 922, 927, 932, 936, 940, - 943, 946, 950, 954, 957, 960, 964, 968, - 972, 975, 978, 982, 986, 990, 994, 997, - 1000, 1004, 1008, 1012, 1016, 1019, 1022, 1026, - 1030, 1034, 1038, 1041, 1044, 1048, 1052, 1056, - 1060, 1063, 1066, 1070, 1074, 1078, 1082, 1085, - 1088, 1093, 1097, 1101, 1105, 1108, 1111, 1115, - 1119, 1123, 1126, 1129, 1133, 1137, 1141, 1145, - 1148, 1151, 1155, 1159, 1163, 1167, 1170, 1173, - 1177, 1181, 1185, 1188, 1191, 1195, 1199, 1203, - 1207, 1211, 1214, 1217, 1222, 1227, 1231, 1235, - 1238, 1241, 1245, 1249, 1252, 1255, 1259, 1263, - 1267, 1270, 1273, 1277, 1281, 1285, 1289, 1292, - 1295, 1299, 1303, 1307, 1311, 1314, 1317, 1321, - 1325, 1329, 1333, 1336, 1339, 1343, 1347, 1351, - 1355, 1358, 1361, 1365, 1369, 1373, 1377, 1380, - 1383, 1388, 1392, 1396, 1400, 1403, 1406, 1410, - 1414, 1418, 1421, 1424, 1428, 1432, 1436, 1440, - 1443, 1446, 1450, 1454, 1458, 1462, 1465, 1468, - 1472, 1476, 1480, 1483, 1486, 1490, 1494, 1498, - 1502, 1506, 1509, 1512, 1515, 1518, 1520, 1522, - 1525, 1532, 1534, 1536, 1538, 1540, 1542, 1544, - 1546, 1548, 1550, 1584, 1586, 1593, 1600, 1614, - 1616, 1622, 1625, 1634, 1635, 1638, 1641, 1648, - 1650, 1652, 1654, 1657, 1702, 1704, 1706, 1710, - 1714, 1716, 1717, 1717, 1723, 1725, 1727, 1729, - 1731, 1734, 1735, 1736, 1743, 1749, 1755, 1757, - 1759, 1761, 1763, 1765, 1767, 1768, 1771, 1794, - 1797, 1802, 1811, 1813, 1814, 1816, 1821, 1824, - 1826, 1828, 1829, 1831, 1841, 1847, 1848, 1853, - 1857, 1865, 1867, 1876, 1880, 1881, 1882, 1886, - 1887, 1890, 1890, 1897, 1913, 1916, 1955, 1957, - 1959, 1961, 1963, 1964, 1964, 1965, 1966, 1973, - 1979, 1985, 1987, 1989, 1991, 2000, 2002, 2015, - 2016, 2018, 2020, 2022, 2035, 2036, 2038, 2040, - 2042, 2043 -}; - -static const short _regex_trans_keys[] = { - 41, 33, 35, 38, 39, 40, 41, 43, - 45, 58, 60, 61, 62, 63, 67, 80, - 105, 109, 115, 120, 123, 48, 57, 41, - 95, 48, 57, 65, 90, 97, 122, 39, - 95, 48, 57, 65, 90, 97, 122, 95, - 48, 57, 65, 90, 97, 122, 39, 95, - 48, 57, 65, 90, 97, 122, 41, 41, - 95, 48, 57, 65, 90, 97, 122, 41, - 95, 48, 57, 65, 90, 97, 122, 41, - 95, 48, 57, 65, 90, 97, 122, 95, - 48, 57, 65, 90, 97, 122, 62, 95, - 48, 57, 65, 90, 97, 122, 33, 60, - 61, 33, 61, 38, 41, 95, 48, 57, - 65, 90, 97, 122, 95, 48, 57, 65, - 90, 97, 122, 41, 95, 48, 57, 65, - 90, 97, 122, 41, 95, 48, 57, 65, - 90, 97, 122, 41, 48, 57, 41, 58, - 105, 109, 115, 120, 62, 95, 48, 57, - 65, 90, 97, 122, 41, 48, 57, 95, - 48, 57, 65, 90, 97, 122, 95, 48, - 57, 65, 90, 97, 122, 41, 95, 48, - 57, 65, 90, 97, 122, 95, 48, 57, - 65, 90, 97, 122, 105, 109, 115, 120, - 41, 45, 58, 105, 109, 115, 120, 46, - 92, 93, 46, 92, 93, 46, 92, 58, - 92, 93, 58, 92, 93, 58, 92, 61, - 92, 93, 61, 92, 93, 61, 92, 39, - 48, 57, 62, 45, 95, 48, 57, 65, - 90, 97, 122, 48, 57, 125, 48, 57, - 125, 48, 57, 125, 95, 125, 48, 57, - 65, 90, 97, 122, 95, 125, 48, 57, - 65, 90, 97, 122, 95, 125, 48, 57, - 65, 90, 97, 122, 95, 125, 48, 57, - 65, 90, 97, 122, 95, 48, 57, 65, - 90, 97, 122, 39, 95, 48, 57, 65, - 90, 97, 122, 95, 48, 57, 65, 90, - 97, 122, 62, 95, 48, 57, 65, 90, - 97, 122, 95, 48, 57, 65, 90, 97, - 122, 95, 125, 48, 57, 65, 90, 97, - 122, 48, 55, 125, 48, 55, 125, 48, - 57, 65, 70, 97, 102, 44, 125, 48, - 57, 125, 48, 57, 125, 48, 57, 384, - 447, 384, 447, 384, 447, 41, 41, 80, - 41, 41, 70, 41, 56, 41, 121, 97, - 109, 98, 105, 99, 101, 110, 105, 97, - 110, 101, 115, 116, 97, 110, 108, 109, - 116, 105, 110, 101, 115, 101, 117, 109, - 97, 107, 110, 103, 97, 108, 105, 112, - 111, 109, 111, 102, 111, 97, 104, 105, - 109, 105, 108, 108, 101, 103, 104, 105, - 110, 101, 115, 101, 105, 100, 110, 114, - 97, 100, 105, 97, 110, 95, 65, 98, - 111, 114, 105, 103, 105, 110, 97, 108, - 105, 97, 110, 97, 101, 109, 114, 111, - 107, 101, 101, 109, 111, 110, 116, 105, - 99, 110, 101, 105, 102, 111, 114, 109, - 112, 114, 114, 105, 111, 116, 105, 108, - 108, 105, 99, 115, 118, 101, 114, 101, - 116, 97, 110, 97, 103, 97, 114, 105, - 121, 112, 116, 105, 97, 110, 95, 72, - 105, 101, 114, 111, 103, 108, 121, 112, - 104, 115, 104, 105, 111, 112, 105, 99, - 111, 114, 103, 105, 97, 110, 97, 103, - 111, 108, 105, 116, 105, 99, 116, 104, - 105, 99, 101, 101, 107, 106, 114, 97, - 114, 97, 116, 105, 109, 117, 107, 104, - 105, 110, 117, 108, 110, 111, 111, 98, - 114, 101, 119, 114, 97, 103, 97, 110, - 97, 112, 101, 114, 105, 97, 108, 95, - 65, 114, 97, 109, 97, 105, 99, 104, - 115, 101, 114, 105, 116, 101, 100, 99, - 114, 105, 112, 116, 105, 111, 110, 97, - 108, 95, 80, 97, 104, 114, 108, 97, - 118, 105, 116, 104, 105, 97, 110, 118, - 97, 110, 101, 115, 101, 105, 110, 116, - 121, 116, 104, 105, 110, 97, 100, 97, - 97, 107, 97, 110, 97, 97, 104, 95, - 76, 105, 97, 109, 114, 111, 115, 104, - 116, 104, 105, 101, 114, 111, 116, 105, - 110, 112, 99, 104, 97, 109, 110, 115, - 98, 117, 101, 97, 114, 95, 66, 117, - 99, 100, 105, 97, 110, 105, 97, 110, - 108, 110, 97, 121, 97, 108, 97, 109, - 100, 97, 105, 99, 116, 101, 105, 95, - 77, 97, 121, 101, 107, 110, 103, 111, - 108, 105, 97, 110, 97, 110, 109, 97, - 114, 119, 95, 84, 97, 105, 95, 76, - 117, 101, 111, 104, 97, 109, 95, 100, - 67, 104, 105, 107, 105, 95, 73, 80, - 83, 84, 116, 97, 108, 105, 99, 101, - 114, 115, 105, 97, 110, 111, 117, 116, - 104, 95, 65, 114, 97, 98, 105, 97, - 110, 117, 114, 107, 105, 99, 105, 121, - 97, 109, 97, 110, 121, 97, 97, 111, - 103, 115, 95, 80, 97, 101, 110, 105, - 99, 105, 97, 110, 106, 97, 110, 103, - 110, 105, 99, 109, 117, 97, 114, 105, - 116, 97, 110, 114, 97, 115, 104, 116, - 114, 97, 97, 118, 105, 97, 110, 110, - 104, 97, 108, 97, 110, 100, 97, 110, - 101, 115, 101, 108, 114, 111, 116, 105, - 95, 78, 97, 103, 114, 105, 105, 97, - 99, 103, 105, 109, 97, 98, 108, 111, - 103, 97, 110, 119, 97, 95, 76, 84, - 86, 101, 104, 97, 109, 105, 101, 116, - 105, 108, 108, 117, 103, 117, 97, 97, - 105, 110, 97, 98, 102, 101, 116, 97, - 110, 105, 110, 97, 103, 104, 97, 114, - 105, 116, 105, 99, 105, 110, 115, 112, - 100, 123, 94, 125, 94, 46, 92, 93, - 46, 92, 93, 46, 92, 58, 92, 93, - 94, 97, 98, 99, 100, 103, 108, 112, - 115, 117, 119, 120, 58, 92, 93, 58, - 92, 93, 58, 92, 58, 92, 93, 97, - 98, 99, 100, 103, 108, 112, 115, 117, - 119, 120, 58, 92, 93, 108, 115, 58, - 92, 93, 110, 112, 58, 92, 93, 117, - 58, 92, 93, 109, 58, 92, 93, 58, - 92, 93, 58, 92, 93, 104, 58, 92, - 93, 97, 58, 92, 93, 58, 92, 93, - 58, 92, 93, 99, 58, 92, 93, 105, - 58, 92, 93, 105, 58, 92, 93, 58, - 92, 93, 58, 92, 93, 108, 58, 92, - 93, 97, 58, 92, 93, 110, 58, 92, - 93, 107, 58, 92, 93, 58, 92, 93, - 58, 92, 93, 110, 58, 92, 93, 116, - 58, 92, 93, 114, 58, 92, 93, 108, - 58, 92, 93, 58, 92, 93, 58, 92, - 93, 105, 58, 92, 93, 103, 58, 92, - 93, 105, 58, 92, 93, 116, 58, 92, - 93, 58, 92, 93, 58, 92, 93, 114, - 58, 92, 93, 97, 58, 92, 93, 112, - 58, 92, 93, 104, 58, 92, 93, 58, - 92, 93, 58, 92, 93, 111, 58, 92, - 93, 119, 58, 92, 93, 101, 58, 92, - 93, 114, 58, 92, 93, 58, 92, 93, - 58, 92, 93, 114, 117, 58, 92, 93, - 105, 58, 92, 93, 110, 58, 92, 93, - 116, 58, 92, 93, 58, 92, 93, 58, - 92, 93, 110, 58, 92, 93, 99, 58, - 92, 93, 116, 58, 92, 93, 58, 92, - 93, 58, 92, 93, 112, 58, 92, 93, - 97, 58, 92, 93, 99, 58, 92, 93, - 101, 58, 92, 93, 58, 92, 93, 58, - 92, 93, 112, 58, 92, 93, 112, 58, - 92, 93, 101, 58, 92, 93, 114, 58, - 92, 93, 58, 92, 93, 58, 92, 93, - 111, 58, 92, 93, 114, 58, 92, 93, - 100, 58, 92, 93, 58, 92, 93, 58, - 92, 93, 100, 58, 92, 93, 105, 58, - 92, 93, 103, 58, 92, 93, 105, 58, - 92, 93, 116, 58, 92, 93, 58, 92, - 93, 58, 92, 93, 108, 115, 58, 92, - 93, 110, 112, 58, 92, 93, 117, 58, - 92, 93, 109, 58, 92, 93, 58, 92, - 93, 58, 92, 93, 104, 58, 92, 93, - 97, 58, 92, 93, 58, 92, 93, 58, - 92, 93, 99, 58, 92, 93, 105, 58, - 92, 93, 105, 58, 92, 93, 58, 92, - 93, 58, 92, 93, 108, 58, 92, 93, - 97, 58, 92, 93, 110, 58, 92, 93, - 107, 58, 92, 93, 58, 92, 93, 58, - 92, 93, 110, 58, 92, 93, 116, 58, - 92, 93, 114, 58, 92, 93, 108, 58, - 92, 93, 58, 92, 93, 58, 92, 93, - 105, 58, 92, 93, 103, 58, 92, 93, - 105, 58, 92, 93, 116, 58, 92, 93, - 58, 92, 93, 58, 92, 93, 114, 58, - 92, 93, 97, 58, 92, 93, 112, 58, - 92, 93, 104, 58, 92, 93, 58, 92, - 93, 58, 92, 93, 111, 58, 92, 93, - 119, 58, 92, 93, 101, 58, 92, 93, - 114, 58, 92, 93, 58, 92, 93, 58, - 92, 93, 114, 117, 58, 92, 93, 105, - 58, 92, 93, 110, 58, 92, 93, 116, - 58, 92, 93, 58, 92, 93, 58, 92, - 93, 110, 58, 92, 93, 99, 58, 92, - 93, 116, 58, 92, 93, 58, 92, 93, - 58, 92, 93, 112, 58, 92, 93, 97, - 58, 92, 93, 99, 58, 92, 93, 101, - 58, 92, 93, 58, 92, 93, 58, 92, - 93, 112, 58, 92, 93, 112, 58, 92, - 93, 101, 58, 92, 93, 114, 58, 92, - 93, 58, 92, 93, 58, 92, 93, 111, - 58, 92, 93, 114, 58, 92, 93, 100, - 58, 92, 93, 58, 92, 93, 58, 92, - 93, 100, 58, 92, 93, 105, 58, 92, - 93, 103, 58, 92, 93, 105, 58, 92, - 93, 116, 58, 92, 93, 58, 92, 93, - 61, 92, 93, 61, 92, 93, 61, 92, - 48, 55, 125, 48, 55, 125, 48, 57, - 65, 70, 97, 102, 384, 447, 384, 447, - 384, 447, 384, 447, 384, 447, 384, 447, - 384, 447, 384, 447, 384, 447, 0, 32, - 36, 40, 41, 42, 43, 46, 63, 91, - 92, 94, 123, 124, 1315, 1571, 1, 8, - 9, 13, 14, 34, 37, 255, 384, 447, - 448, 479, 480, 495, 496, 503, 504, 511, - 42, 63, 95, 48, 57, 65, 90, 97, - 122, 95, 48, 57, 65, 90, 97, 122, - 39, 48, 60, 63, 82, 95, 49, 55, - 56, 57, 65, 90, 97, 122, 48, 57, - 105, 109, 115, 120, 48, 57, 41, 48, - 57, 33, 61, 95, 48, 57, 65, 90, - 97, 122, 123, 41, 48, 57, 60, 61, - 62, 41, 45, 58, 105, 109, 115, 120, - 43, 63, 43, 63, 43, 63, 46, 58, - 61, 48, 65, 66, 67, 68, 69, 71, - 72, 75, 76, 78, 80, 81, 82, 83, - 85, 86, 87, 88, 90, 97, 98, 99, - 100, 101, 102, 103, 104, 107, 108, 110, - 111, 112, 114, 115, 116, 117, 118, 119, - 120, 122, 49, 55, 56, 57, 48, 55, - 48, 55, 48, 55, 56, 57, 48, 55, - 56, 57, 48, 57, 123, 39, 45, 60, - 123, 48, 57, 48, 57, 48, 57, 48, - 57, 48, 57, 39, 60, 123, 123, 123, - 123, 48, 57, 65, 70, 97, 102, 48, - 57, 65, 70, 97, 102, 48, 57, 65, - 70, 97, 102, 48, 57, 43, 63, 384, - 447, 384, 447, 384, 447, 41, 85, 41, - 41, 67, 84, 65, 66, 67, 68, 69, - 71, 72, 73, 74, 75, 76, 77, 78, - 79, 80, 82, 83, 84, 85, 86, 88, - 89, 90, 110, 114, 118, 97, 101, 111, - 114, 117, 97, 99, 102, 104, 110, 111, - 115, 117, 121, 109, 112, 101, 103, 116, - 101, 108, 111, 114, 117, 97, 101, 105, - 103, 117, 109, 110, 97, 97, 104, 38, - 97, 101, 105, 108, 109, 111, 116, 117, - 121, 97, 99, 101, 110, 111, 121, 101, - 100, 101, 107, 108, 111, 103, 108, 114, - 115, 99, 100, 101, 102, 104, 105, 111, - 115, 101, 117, 97, 99, 104, 105, 107, - 109, 111, 117, 121, 97, 101, 104, 105, - 103, 97, 97, 112, 115, 119, 105, 108, - 112, 115, 67, 76, 77, 78, 80, 83, - 90, 45, 91, 92, 93, 0, 255, 384, - 447, 448, 479, 480, 495, 496, 503, 504, - 511, 46, 58, 61, 48, 68, 69, 72, - 76, 78, 80, 81, 83, 85, 86, 87, - 97, 98, 99, 100, 101, 102, 103, 104, - 108, 110, 111, 112, 114, 115, 116, 117, - 118, 119, 120, 49, 55, 56, 57, 65, - 90, 105, 122, 48, 55, 48, 55, 48, - 55, 48, 55, 123, 123, 123, 123, 48, - 57, 65, 70, 97, 102, 48, 57, 65, - 70, 97, 102, 48, 57, 65, 70, 97, - 102, 384, 447, 384, 447, 384, 447, 92, - 1117, 1118, -128, 91, 95, 127, 861, 862, - 69, 81, 92, 0, 255, 384, 447, 448, - 479, 480, 495, 496, 503, 504, 511, 69, - 384, 447, 384, 447, 384, 447, 92, 0, - 255, 384, 447, 448, 479, 480, 495, 496, - 503, 504, 511, 69, 384, 447, 384, 447, - 384, 447, 41, 10, 0 -}; - -static const char _regex_single_lengths[] = { - 0, 1, 20, 2, 2, 1, 2, 1, - 2, 2, 2, 1, 2, 3, 2, 3, - 1, 2, 2, 1, 6, 2, 1, 1, - 1, 2, 1, 4, 7, 3, 3, 2, - 3, 3, 2, 3, 3, 2, 1, 0, - 1, 2, 0, 1, 1, 1, 2, 2, - 2, 2, 1, 2, 1, 2, 1, 2, - 0, 1, 1, 2, 1, 1, 0, 0, - 0, 1, 2, 1, 2, 2, 1, 1, - 2, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 3, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 2, 1, 1, 1, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 2, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 2, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 2, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 2, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 4, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 2, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 2, 1, - 1, 1, 1, 1, 1, 3, 1, 1, - 1, 1, 1, 1, 1, 1, 2, 1, - 1, 1, 1, 1, 1, 2, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 4, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 2, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 2, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 3, 2, - 1, 1, 1, 1, 1, 1, 1, 1, - 3, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 2, - 1, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 3, 3, 2, 15, 3, - 3, 2, 14, 5, 5, 4, 4, 3, - 3, 4, 4, 3, 3, 4, 4, 4, - 3, 3, 4, 4, 4, 4, 3, 3, - 4, 4, 4, 4, 3, 3, 4, 4, - 4, 4, 3, 3, 4, 4, 4, 4, - 3, 3, 4, 4, 4, 4, 3, 3, - 5, 4, 4, 4, 3, 3, 4, 4, - 4, 3, 3, 4, 4, 4, 4, 3, - 3, 4, 4, 4, 4, 3, 3, 4, - 4, 4, 3, 3, 4, 4, 4, 4, - 4, 3, 3, 5, 5, 4, 4, 3, - 3, 4, 4, 3, 3, 4, 4, 4, - 3, 3, 4, 4, 4, 4, 3, 3, - 4, 4, 4, 4, 3, 3, 4, 4, - 4, 4, 3, 3, 4, 4, 4, 4, - 3, 3, 4, 4, 4, 4, 3, 3, - 5, 4, 4, 4, 3, 3, 4, 4, - 4, 3, 3, 4, 4, 4, 4, 3, - 3, 4, 4, 4, 4, 3, 3, 4, - 4, 4, 3, 3, 4, 4, 4, 4, - 4, 3, 3, 3, 3, 2, 0, 1, - 1, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 16, 2, 1, 1, 6, 0, - 4, 1, 3, 1, 1, 3, 7, 2, - 2, 2, 3, 41, 0, 0, 0, 0, - 0, 1, 0, 4, 0, 0, 0, 0, - 3, 1, 1, 1, 0, 0, 0, 2, - 0, 0, 0, 2, 1, 3, 23, 3, - 5, 9, 2, 1, 2, 5, 3, 2, - 2, 1, 2, 10, 6, 1, 5, 4, - 8, 2, 9, 4, 1, 1, 4, 1, - 3, 0, 7, 4, 3, 31, 0, 0, - 0, 0, 1, 0, 1, 1, 1, 0, - 0, 0, 0, 0, 3, 2, 1, 1, - 0, 0, 0, 1, 1, 0, 0, 0, - 1, 1 -}; - -static const char _regex_range_lengths[] = { - 0, 0, 1, 3, 3, 3, 3, 0, - 3, 3, 3, 3, 3, 0, 0, 3, - 3, 3, 3, 1, 0, 3, 1, 3, - 3, 3, 3, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1, - 0, 3, 1, 1, 1, 0, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 1, 1, 3, 1, 1, 1, 1, 1, - 1, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1, 1, - 3, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 9, 0, 3, 3, 4, 1, - 1, 1, 3, 0, 1, 0, 0, 0, - 0, 0, 0, 2, 1, 1, 2, 2, - 1, 0, 0, 1, 1, 1, 1, 1, - 0, 0, 0, 3, 3, 3, 1, 0, - 1, 1, 1, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 6, 0, 4, 1, 1, - 1, 1, 0, 0, 0, 0, 3, 3, - 3, 1, 1, 1, 3, 0, 6, 0, - 1, 1, 1, 6, 0, 1, 1, 1, - 0, 0 -}; - -static const short _regex_index_offsets[] = { - 0, 0, 2, 24, 30, 36, 41, 47, - 49, 55, 61, 67, 72, 78, 82, 85, - 92, 97, 103, 109, 112, 119, 125, 128, - 133, 138, 144, 149, 154, 162, 166, 170, - 173, 177, 181, 184, 188, 192, 195, 197, - 199, 201, 207, 209, 212, 215, 217, 223, - 229, 235, 241, 246, 252, 257, 263, 268, - 274, 276, 279, 284, 288, 291, 294, 296, - 298, 300, 302, 305, 307, 310, 313, 315, - 317, 320, 322, 324, 326, 328, 330, 332, - 334, 336, 338, 340, 342, 344, 346, 350, - 352, 354, 356, 358, 360, 362, 364, 366, - 368, 370, 372, 374, 376, 378, 380, 382, - 384, 386, 388, 390, 392, 395, 397, 399, - 401, 403, 405, 408, 410, 412, 414, 416, - 418, 420, 422, 425, 427, 429, 431, 433, - 435, 437, 439, 441, 443, 445, 447, 449, - 451, 453, 455, 457, 459, 461, 463, 466, - 468, 470, 472, 474, 476, 478, 480, 482, - 484, 486, 488, 490, 492, 494, 496, 498, - 500, 502, 504, 507, 509, 511, 513, 515, - 517, 519, 521, 523, 525, 528, 530, 532, - 534, 536, 538, 540, 542, 544, 546, 548, - 550, 552, 554, 556, 558, 560, 562, 564, - 566, 568, 570, 572, 574, 576, 578, 580, - 582, 584, 586, 588, 590, 592, 594, 596, - 598, 600, 602, 604, 606, 608, 610, 612, - 614, 616, 618, 620, 622, 624, 626, 628, - 630, 632, 634, 636, 638, 640, 643, 645, - 647, 649, 651, 653, 655, 657, 659, 661, - 663, 665, 667, 669, 671, 673, 675, 677, - 679, 681, 683, 685, 687, 689, 691, 693, - 695, 697, 699, 701, 703, 705, 707, 709, - 711, 713, 715, 717, 719, 721, 723, 726, - 728, 730, 732, 734, 736, 738, 740, 742, - 744, 746, 748, 750, 752, 754, 756, 758, - 760, 762, 764, 767, 769, 771, 773, 775, - 777, 779, 781, 783, 785, 787, 789, 791, - 793, 795, 797, 802, 804, 806, 808, 810, - 812, 814, 816, 818, 820, 822, 824, 826, - 828, 830, 832, 834, 836, 839, 841, 843, - 845, 847, 849, 851, 853, 855, 857, 860, - 862, 864, 866, 868, 870, 872, 876, 878, - 880, 882, 884, 886, 888, 890, 892, 895, - 897, 899, 901, 903, 905, 907, 910, 912, - 914, 916, 918, 920, 922, 924, 926, 928, - 930, 932, 934, 936, 938, 940, 942, 944, - 946, 948, 950, 952, 954, 956, 958, 960, - 962, 964, 966, 968, 970, 972, 974, 976, - 978, 980, 982, 984, 986, 988, 990, 992, - 994, 996, 998, 1001, 1003, 1005, 1007, 1009, - 1011, 1013, 1018, 1020, 1022, 1024, 1026, 1028, - 1030, 1032, 1034, 1036, 1038, 1040, 1042, 1044, - 1046, 1048, 1050, 1052, 1054, 1056, 1058, 1060, - 1062, 1064, 1066, 1068, 1070, 1072, 1074, 1076, - 1078, 1080, 1082, 1084, 1086, 1088, 1090, 1093, - 1095, 1097, 1099, 1101, 1103, 1105, 1107, 1109, - 1111, 1113, 1115, 1117, 1119, 1121, 1123, 1125, - 1127, 1129, 1131, 1134, 1136, 1138, 1140, 1142, - 1144, 1146, 1148, 1150, 1152, 1154, 1156, 1158, - 1160, 1162, 1164, 1166, 1168, 1170, 1172, 1174, - 1176, 1178, 1180, 1182, 1184, 1186, 1188, 1190, - 1192, 1194, 1197, 1199, 1201, 1203, 1205, 1207, - 1209, 1211, 1213, 1215, 1217, 1219, 1221, 1225, - 1228, 1230, 1232, 1234, 1236, 1238, 1240, 1242, - 1244, 1248, 1250, 1252, 1254, 1256, 1258, 1260, - 1262, 1264, 1266, 1268, 1270, 1272, 1274, 1276, - 1279, 1281, 1283, 1286, 1288, 1290, 1292, 1294, - 1296, 1298, 1300, 1302, 1304, 1306, 1308, 1310, - 1312, 1314, 1316, 1318, 1320, 1322, 1324, 1326, - 1328, 1330, 1332, 1334, 1338, 1342, 1345, 1361, - 1365, 1369, 1372, 1387, 1393, 1399, 1404, 1409, - 1413, 1417, 1422, 1427, 1431, 1435, 1440, 1445, - 1450, 1454, 1458, 1463, 1468, 1473, 1478, 1482, - 1486, 1491, 1496, 1501, 1506, 1510, 1514, 1519, - 1524, 1529, 1534, 1538, 1542, 1547, 1552, 1557, - 1562, 1566, 1570, 1575, 1580, 1585, 1590, 1594, - 1598, 1604, 1609, 1614, 1619, 1623, 1627, 1632, - 1637, 1642, 1646, 1650, 1655, 1660, 1665, 1670, - 1674, 1678, 1683, 1688, 1693, 1698, 1702, 1706, - 1711, 1716, 1721, 1725, 1729, 1734, 1739, 1744, - 1749, 1754, 1758, 1762, 1768, 1774, 1779, 1784, - 1788, 1792, 1797, 1802, 1806, 1810, 1815, 1820, - 1825, 1829, 1833, 1838, 1843, 1848, 1853, 1857, - 1861, 1866, 1871, 1876, 1881, 1885, 1889, 1894, - 1899, 1904, 1909, 1913, 1917, 1922, 1927, 1932, - 1937, 1941, 1945, 1950, 1955, 1960, 1965, 1969, - 1973, 1979, 1984, 1989, 1994, 1998, 2002, 2007, - 2012, 2017, 2021, 2025, 2030, 2035, 2040, 2045, - 2049, 2053, 2058, 2063, 2068, 2073, 2077, 2081, - 2086, 2091, 2096, 2100, 2104, 2109, 2114, 2119, - 2124, 2129, 2133, 2137, 2141, 2145, 2148, 2150, - 2153, 2158, 2160, 2162, 2164, 2166, 2168, 2170, - 2172, 2174, 2176, 2202, 2205, 2210, 2215, 2226, - 2228, 2234, 2237, 2244, 2246, 2249, 2253, 2261, - 2264, 2267, 2270, 2274, 2318, 2320, 2322, 2325, - 2328, 2330, 2332, 2333, 2339, 2341, 2343, 2345, - 2347, 2351, 2353, 2355, 2360, 2364, 2368, 2370, - 2373, 2375, 2377, 2379, 2382, 2384, 2388, 2412, - 2416, 2422, 2432, 2435, 2437, 2440, 2446, 2450, - 2453, 2456, 2458, 2461, 2472, 2479, 2481, 2487, - 2492, 2501, 2504, 2514, 2519, 2521, 2523, 2528, - 2530, 2534, 2535, 2543, 2554, 2558, 2594, 2596, - 2598, 2600, 2602, 2604, 2605, 2607, 2609, 2614, - 2618, 2622, 2624, 2626, 2628, 2635, 2638, 2646, - 2648, 2650, 2652, 2654, 2662, 2664, 2666, 2668, - 2670, 2672 -}; - -static const short _regex_indicies[] = { - 0, 1, 3, 4, 5, 6, 7, 8, - 9, 10, 12, 13, 14, 15, 16, 17, - 18, 19, 19, 19, 19, 20, 11, 2, - 22, 23, 23, 23, 23, 21, 24, 25, - 25, 25, 25, 21, 27, 27, 27, 27, - 26, 28, 27, 27, 27, 27, 26, 29, - 26, 29, 30, 30, 30, 30, 26, 31, - 30, 32, 30, 30, 26, 29, 30, 32, - 30, 30, 26, 33, 33, 33, 33, 26, - 28, 33, 33, 33, 33, 26, 34, 35, - 36, 26, 37, 38, 26, 39, 40, 30, - 41, 30, 30, 26, 42, 42, 42, 42, - 26, 40, 42, 42, 42, 42, 26, 40, - 30, 41, 30, 30, 26, 43, 44, 21, - 45, 46, 47, 47, 47, 47, 21, 24, - 48, 48, 48, 48, 21, 49, 50, 21, - 48, 48, 48, 48, 21, 51, 51, 51, - 51, 21, 52, 51, 51, 51, 51, 21, - 23, 23, 23, 23, 21, 47, 47, 47, - 47, 21, 45, 53, 46, 54, 54, 54, - 54, 21, 57, 58, 55, 56, 57, 58, - 59, 56, 57, 58, 56, 61, 62, 55, - 60, 61, 62, 63, 60, 61, 62, 60, - 65, 66, 55, 64, 65, 66, 59, 64, - 65, 66, 64, 69, 68, 70, 67, 69, - 71, 72, 74, 73, 74, 74, 67, 75, - 67, 77, 76, 67, 77, 78, 67, 77, - 67, 74, 80, 79, 74, 74, 67, 74, - 80, 81, 74, 74, 67, 74, 80, 74, - 74, 74, 67, 74, 82, 74, 74, 74, - 67, 84, 84, 84, 84, 83, 85, 84, - 84, 84, 84, 83, 86, 86, 86, 86, - 83, 87, 86, 86, 86, 86, 83, 88, - 88, 88, 88, 83, 88, 89, 88, 88, - 88, 83, 91, 90, 92, 91, 90, 95, - 94, 94, 94, 93, 97, 99, 98, 96, - 101, 100, 96, 102, 100, 96, 104, 103, - 105, 103, 106, 103, 109, 108, 109, 110, - 108, 111, 108, 109, 112, 108, 113, 114, - 108, 115, 108, 117, 116, 118, 119, 116, - 120, 116, 121, 116, 122, 116, 123, 116, - 124, 116, 125, 116, 126, 116, 127, 116, - 128, 116, 129, 116, 130, 116, 131, 116, - 132, 116, 133, 134, 135, 116, 136, 116, - 137, 116, 138, 116, 139, 116, 140, 116, - 141, 116, 142, 116, 143, 116, 144, 116, - 145, 116, 146, 116, 147, 116, 148, 116, - 149, 116, 150, 116, 151, 116, 152, 116, - 153, 116, 154, 116, 155, 116, 156, 116, - 157, 158, 116, 159, 116, 160, 116, 161, - 116, 162, 116, 163, 116, 164, 165, 116, - 166, 116, 167, 116, 168, 116, 169, 116, - 170, 116, 171, 116, 172, 116, 174, 175, - 173, 176, 173, 177, 173, 178, 173, 179, - 173, 180, 173, 181, 173, 182, 173, 183, - 173, 184, 173, 185, 173, 186, 173, 187, - 173, 188, 173, 189, 173, 190, 173, 191, - 173, 192, 173, 193, 173, 194, 173, 195, - 196, 173, 197, 173, 198, 173, 199, 173, - 200, 173, 201, 173, 202, 173, 204, 203, - 205, 203, 206, 203, 207, 203, 208, 203, - 209, 203, 210, 173, 211, 173, 212, 173, - 213, 173, 214, 173, 215, 173, 216, 173, - 217, 218, 173, 219, 173, 220, 173, 221, - 173, 222, 173, 223, 173, 224, 173, 225, - 173, 226, 173, 227, 173, 228, 229, 116, - 230, 116, 231, 116, 232, 116, 233, 116, - 234, 116, 235, 116, 236, 116, 237, 116, - 238, 116, 239, 116, 240, 116, 241, 116, - 242, 116, 243, 116, 244, 116, 245, 116, - 246, 116, 247, 116, 248, 116, 249, 116, - 250, 116, 251, 116, 252, 116, 253, 116, - 254, 116, 255, 116, 256, 116, 257, 116, - 258, 116, 259, 116, 260, 116, 261, 116, - 262, 116, 263, 116, 264, 116, 265, 116, - 266, 116, 267, 116, 268, 116, 269, 116, - 270, 116, 271, 116, 272, 116, 273, 116, - 274, 116, 275, 116, 276, 116, 277, 116, - 278, 116, 279, 116, 280, 116, 281, 116, - 282, 116, 283, 116, 284, 116, 285, 116, - 286, 287, 116, 288, 116, 289, 116, 290, - 116, 291, 116, 292, 116, 293, 116, 294, - 116, 295, 116, 296, 116, 297, 116, 298, - 116, 300, 299, 301, 299, 302, 299, 303, - 299, 304, 299, 305, 116, 306, 116, 307, - 116, 308, 116, 309, 116, 310, 116, 311, - 116, 312, 116, 313, 116, 314, 116, 315, - 116, 316, 116, 317, 116, 318, 116, 319, - 116, 320, 116, 321, 116, 322, 116, 323, - 116, 324, 116, 325, 116, 326, 116, 327, - 116, 328, 116, 329, 330, 116, 331, 116, - 332, 116, 333, 116, 334, 116, 335, 116, - 336, 116, 337, 116, 338, 116, 339, 116, - 340, 116, 341, 116, 342, 116, 343, 116, - 344, 116, 345, 116, 346, 116, 347, 116, - 348, 116, 349, 116, 350, 351, 116, 352, - 116, 353, 116, 354, 116, 355, 116, 356, - 116, 357, 116, 358, 116, 359, 116, 360, - 116, 361, 116, 362, 116, 363, 116, 364, - 116, 365, 116, 366, 116, 367, 368, 369, - 370, 116, 371, 116, 372, 116, 373, 116, - 374, 116, 375, 116, 376, 116, 377, 116, - 378, 116, 379, 116, 380, 116, 381, 116, - 382, 116, 383, 116, 384, 116, 385, 116, - 386, 116, 387, 116, 388, 389, 116, 390, - 116, 391, 116, 392, 116, 393, 116, 394, - 116, 395, 116, 396, 116, 397, 116, 398, - 116, 400, 401, 399, 402, 399, 403, 399, - 404, 399, 405, 399, 406, 399, 407, 399, - 408, 409, 410, 399, 411, 399, 412, 399, - 413, 399, 414, 399, 415, 399, 416, 399, - 417, 399, 418, 399, 419, 420, 399, 421, - 399, 422, 399, 423, 399, 424, 399, 425, - 399, 426, 399, 428, 429, 427, 430, 427, - 431, 427, 432, 427, 433, 427, 434, 427, - 435, 427, 436, 427, 437, 427, 438, 427, - 439, 427, 441, 440, 442, 440, 443, 440, - 444, 440, 445, 440, 446, 440, 447, 440, - 448, 440, 449, 440, 450, 427, 451, 427, - 452, 427, 453, 427, 454, 427, 455, 427, - 456, 427, 457, 427, 458, 427, 459, 427, - 460, 427, 461, 427, 463, 462, 464, 462, - 465, 462, 466, 462, 467, 462, 468, 462, - 469, 462, 470, 462, 471, 462, 472, 462, - 473, 116, 474, 116, 475, 116, 476, 477, - 116, 478, 116, 479, 116, 480, 116, 481, - 116, 482, 116, 483, 116, 484, 485, 486, - 487, 116, 488, 116, 489, 116, 490, 116, - 491, 116, 492, 116, 493, 116, 494, 116, - 495, 116, 496, 116, 497, 116, 498, 116, - 499, 116, 500, 116, 501, 116, 502, 116, - 503, 116, 504, 116, 505, 116, 506, 116, - 507, 116, 508, 116, 509, 116, 510, 116, - 511, 116, 512, 116, 513, 116, 514, 116, - 515, 116, 516, 116, 517, 116, 518, 116, - 519, 116, 520, 116, 521, 116, 522, 116, - 523, 116, 525, 526, 524, 527, 524, 528, - 524, 529, 524, 530, 524, 531, 524, 532, - 524, 533, 524, 534, 524, 535, 524, 536, - 524, 537, 524, 538, 524, 539, 116, 540, - 116, 541, 116, 542, 116, 543, 116, 544, - 116, 545, 116, 547, 548, 546, 549, 546, - 550, 546, 551, 546, 552, 546, 553, 546, - 554, 546, 555, 546, 556, 546, 557, 546, - 558, 546, 559, 546, 560, 546, 561, 546, - 562, 546, 563, 546, 564, 546, 565, 546, - 566, 546, 567, 546, 568, 546, 569, 546, - 570, 546, 571, 546, 572, 546, 573, 546, - 574, 546, 575, 546, 576, 546, 577, 546, - 578, 546, 579, 580, 546, 581, 546, 582, - 546, 583, 546, 584, 546, 585, 546, 586, - 546, 587, 546, 588, 546, 589, 546, 590, - 546, 591, 546, 592, 546, 593, 594, 595, - 116, 596, 597, 116, 598, 116, 599, 116, - 600, 116, 601, 116, 602, 116, 603, 116, - 604, 116, 605, 116, 606, 607, 608, 116, - 609, 116, 610, 116, 611, 116, 612, 116, - 613, 116, 614, 116, 615, 116, 616, 116, - 617, 116, 618, 116, 619, 116, 620, 116, - 621, 116, 622, 116, 623, 624, 116, 625, - 116, 626, 116, 627, 628, 116, 629, 116, - 630, 116, 631, 116, 632, 116, 633, 116, - 634, 116, 635, 116, 636, 116, 637, 116, - 638, 116, 639, 116, 640, 116, 641, 116, - 642, 116, 643, 116, 644, 116, 645, 116, - 646, 116, 647, 116, 648, 116, 650, 649, - 652, 651, 653, 649, 649, 651, 656, 657, - 654, 655, 656, 657, 658, 655, 656, 657, - 655, 660, 661, 654, 662, 663, 664, 665, - 666, 667, 668, 669, 670, 671, 672, 673, - 659, 660, 661, 654, 659, 660, 661, 674, - 659, 660, 661, 659, 660, 661, 654, 675, - 676, 677, 678, 679, 680, 681, 682, 683, - 684, 685, 659, 660, 661, 654, 686, 687, - 659, 660, 661, 654, 688, 689, 659, 660, - 661, 654, 690, 659, 660, 661, 654, 691, - 659, 692, 661, 654, 659, 660, 661, 693, - 659, 660, 661, 654, 694, 659, 660, 661, - 654, 695, 659, 696, 661, 654, 659, 660, - 661, 697, 659, 660, 661, 654, 698, 659, - 660, 661, 654, 699, 659, 660, 661, 654, - 700, 659, 701, 661, 654, 659, 660, 661, - 702, 659, 660, 661, 654, 703, 659, 660, - 661, 654, 704, 659, 660, 661, 654, 705, - 659, 660, 661, 654, 706, 659, 707, 661, - 654, 659, 660, 661, 708, 659, 660, 661, - 654, 709, 659, 660, 661, 654, 710, 659, - 660, 661, 654, 711, 659, 660, 661, 654, - 712, 659, 713, 661, 654, 659, 660, 661, - 714, 659, 660, 661, 654, 715, 659, 660, - 661, 654, 716, 659, 660, 661, 654, 717, - 659, 660, 661, 654, 718, 659, 719, 661, - 654, 659, 660, 661, 720, 659, 660, 661, - 654, 721, 659, 660, 661, 654, 722, 659, - 660, 661, 654, 723, 659, 660, 661, 654, - 724, 659, 725, 661, 654, 659, 660, 661, - 726, 659, 660, 661, 654, 727, 659, 660, - 661, 654, 728, 659, 660, 661, 654, 729, - 659, 660, 661, 654, 730, 659, 731, 661, - 654, 659, 660, 661, 732, 659, 660, 661, - 654, 733, 734, 659, 660, 661, 654, 735, - 659, 660, 661, 654, 736, 659, 660, 661, - 654, 737, 659, 738, 661, 654, 659, 660, - 661, 739, 659, 660, 661, 654, 740, 659, - 660, 661, 654, 741, 659, 660, 661, 654, - 742, 659, 743, 661, 654, 659, 660, 661, - 744, 659, 660, 661, 654, 745, 659, 660, - 661, 654, 746, 659, 660, 661, 654, 747, - 659, 660, 661, 654, 748, 659, 749, 661, - 654, 659, 660, 661, 750, 659, 660, 661, - 654, 751, 659, 660, 661, 654, 752, 659, - 660, 661, 654, 753, 659, 660, 661, 654, - 754, 659, 755, 661, 654, 659, 660, 661, - 756, 659, 660, 661, 654, 757, 659, 660, - 661, 654, 758, 659, 660, 661, 654, 759, - 659, 760, 661, 654, 659, 660, 661, 761, - 659, 660, 661, 654, 762, 659, 660, 661, - 654, 763, 659, 660, 661, 654, 764, 659, - 660, 661, 654, 765, 659, 660, 661, 654, - 766, 659, 767, 661, 654, 659, 660, 661, - 768, 659, 660, 661, 654, 769, 770, 659, - 660, 661, 654, 771, 772, 659, 660, 661, - 654, 773, 659, 660, 661, 654, 774, 659, - 775, 661, 654, 659, 660, 661, 776, 659, - 660, 661, 654, 777, 659, 660, 661, 654, - 778, 659, 779, 661, 654, 659, 660, 661, - 780, 659, 660, 661, 654, 781, 659, 660, - 661, 654, 782, 659, 660, 661, 654, 783, - 659, 784, 661, 654, 659, 660, 661, 785, - 659, 660, 661, 654, 786, 659, 660, 661, - 654, 787, 659, 660, 661, 654, 788, 659, - 660, 661, 654, 789, 659, 790, 661, 654, - 659, 660, 661, 791, 659, 660, 661, 654, - 792, 659, 660, 661, 654, 793, 659, 660, - 661, 654, 794, 659, 660, 661, 654, 795, - 659, 796, 661, 654, 659, 660, 661, 797, - 659, 660, 661, 654, 798, 659, 660, 661, - 654, 799, 659, 660, 661, 654, 800, 659, - 660, 661, 654, 801, 659, 802, 661, 654, - 659, 660, 661, 803, 659, 660, 661, 654, - 804, 659, 660, 661, 654, 805, 659, 660, - 661, 654, 806, 659, 660, 661, 654, 807, - 659, 808, 661, 654, 659, 660, 661, 809, - 659, 660, 661, 654, 810, 659, 660, 661, - 654, 811, 659, 660, 661, 654, 812, 659, - 660, 661, 654, 813, 659, 814, 661, 654, - 659, 660, 661, 815, 659, 660, 661, 654, - 816, 817, 659, 660, 661, 654, 818, 659, - 660, 661, 654, 819, 659, 660, 661, 654, - 820, 659, 821, 661, 654, 659, 660, 661, - 822, 659, 660, 661, 654, 823, 659, 660, - 661, 654, 824, 659, 660, 661, 654, 825, - 659, 826, 661, 654, 659, 660, 661, 827, - 659, 660, 661, 654, 828, 659, 660, 661, - 654, 829, 659, 660, 661, 654, 830, 659, - 660, 661, 654, 831, 659, 832, 661, 654, - 659, 660, 661, 833, 659, 660, 661, 654, - 834, 659, 660, 661, 654, 835, 659, 660, - 661, 654, 836, 659, 660, 661, 654, 837, - 659, 838, 661, 654, 659, 660, 661, 839, - 659, 660, 661, 654, 840, 659, 660, 661, - 654, 841, 659, 660, 661, 654, 842, 659, - 843, 661, 654, 659, 660, 661, 844, 659, - 660, 661, 654, 845, 659, 660, 661, 654, - 846, 659, 660, 661, 654, 847, 659, 660, - 661, 654, 848, 659, 660, 661, 654, 849, - 659, 850, 661, 654, 659, 660, 661, 851, - 659, 853, 854, 654, 852, 853, 854, 658, - 852, 853, 854, 852, 856, 855, 857, 856, - 855, 860, 859, 859, 859, 858, 862, 861, - 863, 861, 864, 861, 866, 865, 867, 865, - 868, 865, 870, 869, 871, 869, 872, 869, - 873, 876, 877, 878, 879, 880, 881, 882, - 883, 884, 885, 886, 887, 888, 875, 893, - 875, 876, 875, 875, 889, 890, 891, 892, - 889, 874, 895, 896, 894, 23, 23, 23, - 23, 897, 25, 25, 25, 25, 897, 899, - 30, 902, 903, 904, 30, 900, 901, 30, - 30, 898, 44, 897, 47, 47, 47, 47, - 44, 897, 43, 44, 897, 905, 906, 48, - 48, 48, 48, 897, 907, 897, 49, 50, - 897, 908, 909, 910, 897, 45, 53, 46, - 54, 54, 54, 54, 897, 912, 913, 911, - 915, 916, 914, 918, 919, 917, 56, 60, - 64, 920, 923, 926, 927, 928, 929, 930, - 931, 932, 933, 934, 934, 935, 936, 937, - 938, 934, 939, 940, 941, 942, 943, 944, - 945, 946, 947, 948, 949, 950, 951, 934, - 952, 953, 954, 955, 956, 957, 934, 958, - 959, 960, 961, 924, 925, 922, 963, 962, - 964, 962, 966, 967, 965, 969, 967, 968, - 967, 970, 973, 972, 975, 68, 977, 71, - 979, 978, 976, 981, 980, 982, 980, 984, - 983, 985, 983, 987, 988, 989, 986, 991, - 990, 994, 993, 999, 996, 997, 998, 995, - 1000, 1001, 1002, 995, 94, 94, 94, 1003, - 98, 1004, 1006, 1007, 1005, 1009, 1008, 1010, - 1008, 1011, 1008, 1013, 1014, 1012, 109, 108, - 109, 1016, 1017, 108, 1019, 1020, 1021, 1022, - 1023, 1024, 1025, 1026, 1027, 1028, 1029, 1030, - 1031, 1032, 1033, 1034, 1035, 1036, 1037, 1038, - 1039, 1040, 1041, 1018, 1043, 1044, 1045, 1042, - 1046, 1047, 1048, 1049, 1050, 1042, 1052, 1053, - 1054, 1055, 1056, 1057, 1058, 1059, 1060, 1051, - 1062, 1063, 1061, 1064, 1042, 1065, 1066, 1042, - 1067, 1068, 1069, 1070, 1071, 1042, 1072, 1073, - 1074, 1042, 1076, 1077, 1075, 1078, 1079, 1042, - 1080, 1042, 1081, 1082, 1042, 1084, 1085, 1086, - 1087, 1088, 1089, 1090, 1091, 1092, 1093, 1083, - 1095, 1096, 1097, 1098, 1099, 1100, 1094, 1102, - 1101, 1104, 1105, 1106, 1107, 1108, 1103, 1109, - 1110, 1111, 1112, 1042, 1114, 1115, 1116, 1117, - 1118, 1119, 1120, 1121, 1113, 1122, 1123, 1042, - 1125, 1126, 1127, 1128, 1129, 1130, 1131, 1132, - 1133, 1124, 1134, 1135, 1136, 1137, 1042, 1138, - 1042, 1139, 1042, 1140, 1141, 1142, 1143, 1042, - 1144, 1042, 1146, 1147, 1148, 1145, 649, 1150, - 1151, 1152, 1153, 1154, 1155, 1156, 1149, 1158, - 1159, 1160, 1161, 1157, 1162, 1163, 1164, 1165, - 1162, 874, 655, 1167, 852, 1166, 1169, 1173, - 1174, 1175, 1176, 1176, 1177, 1178, 1179, 1176, - 1180, 1181, 1182, 1183, 1184, 1185, 1186, 1187, - 1188, 1189, 1176, 1190, 1191, 1192, 1193, 1194, - 1195, 1176, 1196, 1197, 1198, 1170, 1171, 1172, - 1172, 1168, 1200, 1199, 1201, 1199, 1203, 1202, - 1204, 1202, 1207, 1206, 1209, 1211, 1210, 1214, - 1213, 1219, 1216, 1217, 1218, 1215, 1220, 1221, - 1222, 1215, 859, 859, 859, 1223, 1225, 1224, - 1226, 1224, 1227, 1224, 1229, 1230, 1231, 1228, - 1228, 1228, 874, 1233, 1234, 1232, 1236, 1235, - 1237, 1238, 1239, 1240, 1237, 874, 1242, 1241, - 1244, 1243, 1245, 1243, 1246, 1243, 1248, 1247, - 1249, 1250, 1251, 1252, 1249, 874, 1254, 1253, - 1256, 1255, 1257, 1255, 1258, 1255, 1260, 1259, - 1262, 1261, 0 -}; - -static const short _regex_trans_targs[] = { - 746, 746, 746, 746, 746, 748, 749, 750, - 746, 751, 752, 753, 746, 754, 746, 746, - 755, 756, 757, 758, 746, 746, 746, 3, - 746, 4, 746, 6, 7, 746, 8, 746, - 9, 12, 746, 14, 746, 746, 746, 16, - 746, 18, 17, 746, 19, 746, 746, 20, - 21, 746, 22, 25, 746, 27, 28, 746, - 29, 30, 31, 746, 32, 33, 34, 746, - 35, 36, 37, 746, 38, 746, 772, 40, - 42, 46, 49, 43, 44, 746, 45, 47, - 746, 48, 746, 746, 51, 746, 53, 746, - 55, 746, 746, 57, 746, 746, 58, 746, - 746, 60, 59, 783, 61, 783, 783, 746, - 746, 64, 746, 787, 65, 787, 67, 787, - 69, 787, 70, 787, 790, 790, 73, 76, - 74, 75, 790, 77, 78, 79, 80, 790, - 82, 83, 84, 85, 790, 87, 92, 94, - 88, 89, 90, 91, 790, 93, 790, 95, - 790, 97, 98, 99, 100, 790, 102, 103, - 104, 105, 106, 790, 108, 109, 111, 110, - 790, 112, 113, 790, 115, 120, 116, 117, - 118, 119, 790, 121, 790, 790, 123, 139, - 124, 125, 126, 127, 128, 129, 130, 131, - 132, 133, 134, 135, 136, 137, 138, 790, - 140, 141, 790, 143, 144, 790, 145, 146, - 147, 148, 790, 790, 150, 151, 790, 153, - 154, 790, 156, 157, 158, 159, 160, 161, - 790, 163, 167, 164, 165, 166, 790, 168, - 169, 170, 171, 790, 173, 177, 174, 175, - 176, 790, 178, 179, 180, 181, 182, 183, - 790, 185, 186, 187, 188, 189, 190, 191, - 192, 193, 194, 195, 196, 197, 198, 199, - 200, 201, 790, 203, 204, 205, 206, 207, - 790, 209, 210, 211, 212, 213, 790, 215, - 216, 217, 218, 219, 220, 221, 790, 223, - 224, 225, 790, 227, 228, 790, 230, 235, - 231, 232, 233, 234, 790, 236, 237, 238, - 239, 790, 799, 790, 242, 790, 244, 245, - 790, 247, 248, 249, 790, 251, 252, 253, - 254, 255, 790, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, - 790, 271, 277, 272, 273, 274, 275, 276, - 790, 278, 279, 280, 281, 282, 283, 284, - 285, 286, 287, 288, 289, 290, 291, 295, - 292, 293, 294, 790, 296, 297, 298, 299, - 790, 301, 302, 303, 304, 305, 790, 307, - 310, 314, 319, 308, 309, 790, 311, 312, - 313, 790, 315, 316, 317, 318, 790, 320, - 321, 322, 323, 790, 325, 332, 326, 327, - 328, 329, 330, 331, 790, 333, 790, 790, - 790, 335, 336, 790, 338, 339, 340, 790, - 342, 344, 349, 343, 790, 345, 346, 347, - 348, 790, 790, 351, 354, 352, 353, 790, - 355, 356, 790, 790, 358, 364, 359, 360, - 361, 362, 363, 790, 365, 366, 367, 790, - 790, 369, 370, 371, 372, 373, 374, 375, - 376, 790, 378, 379, 380, 381, 382, 383, - 790, 385, 386, 387, 388, 790, 790, 390, - 391, 392, 393, 394, 395, 396, 397, 790, - 790, 400, 401, 790, 403, 408, 404, 405, - 406, 407, 790, 409, 410, 415, 421, 433, - 411, 412, 413, 414, 790, 416, 417, 418, - 419, 420, 790, 422, 423, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 790, 434, - 435, 436, 437, 790, 439, 440, 790, 442, - 443, 444, 445, 790, 790, 447, 452, 448, - 449, 450, 451, 790, 453, 454, 455, 456, - 457, 458, 790, 460, 461, 462, 790, 464, - 465, 790, 790, 467, 473, 468, 469, 470, - 471, 472, 790, 474, 475, 476, 477, 478, - 479, 790, 481, 482, 483, 484, 790, 486, - 487, 488, 489, 790, 491, 492, 493, 494, - 495, 496, 790, 498, 507, 499, 500, 501, - 502, 503, 504, 505, 506, 790, 508, 509, - 790, 511, 519, 528, 512, 515, 513, 514, - 790, 516, 517, 518, 790, 520, 521, 522, - 525, 790, 523, 524, 790, 526, 527, 790, - 529, 790, 531, 532, 533, 790, 535, 536, - 790, 537, 790, 539, 543, 540, 541, 542, - 790, 544, 545, 546, 547, 790, 549, 550, - 551, 552, 553, 790, 790, 790, 790, 790, - 790, 0, 560, 561, 562, 817, 819, 563, - 564, 565, 819, 567, 568, 569, 570, 651, - 666, 672, 678, 684, 690, 696, 707, 713, - 719, 724, 819, 571, 586, 592, 598, 604, - 610, 616, 627, 633, 639, 644, 572, 581, - 573, 577, 574, 575, 576, 819, 578, 579, - 580, 819, 582, 583, 584, 585, 819, 587, - 588, 589, 590, 591, 819, 593, 594, 595, - 596, 597, 819, 599, 600, 601, 602, 603, - 819, 605, 606, 607, 608, 609, 819, 611, - 612, 613, 614, 615, 819, 617, 622, 618, - 619, 620, 621, 819, 623, 624, 625, 626, - 819, 628, 629, 630, 631, 632, 819, 634, - 635, 636, 637, 638, 819, 640, 641, 642, - 643, 819, 645, 646, 647, 648, 649, 650, - 819, 652, 661, 653, 657, 654, 655, 656, - 819, 658, 659, 660, 819, 662, 663, 664, - 665, 819, 667, 668, 669, 670, 671, 819, - 673, 674, 675, 676, 677, 819, 679, 680, - 681, 682, 683, 819, 685, 686, 687, 688, - 689, 819, 691, 692, 693, 694, 695, 819, - 697, 702, 698, 699, 700, 701, 819, 703, - 704, 705, 706, 819, 708, 709, 710, 711, - 712, 819, 714, 715, 716, 717, 718, 819, - 720, 721, 722, 723, 819, 725, 726, 727, - 728, 729, 730, 819, 731, 732, 733, 819, - 735, 819, 819, 736, 819, 819, 819, 739, - 819, 838, 838, 742, 838, 843, 843, 745, - 843, 746, 0, 746, 746, 746, 747, 746, - 759, 760, 746, 761, 762, 763, 746, 782, - 746, 746, 784, 785, 786, 746, 746, 1, - 2, 746, 746, 5, 9, 10, 11, 13, - 15, 746, 746, 746, 23, 24, 26, 746, - 746, 746, 746, 746, 746, 746, 746, 746, - 746, 746, 746, 764, 766, 768, 746, 746, - 746, 746, 746, 746, 746, 746, 746, 769, - 746, 746, 746, 746, 746, 746, 746, 746, - 746, 770, 746, 746, 746, 771, 746, 776, - 746, 777, 778, 746, 746, 746, 746, 746, - 779, 746, 746, 765, 746, 746, 767, 768, - 746, 768, 746, 746, 746, 746, 746, 746, - 746, 39, 774, 41, 746, 773, 746, 746, - 775, 746, 746, 50, 52, 54, 746, 56, - 746, 746, 746, 746, 780, 780, 780, 781, - 746, 746, 746, 746, 746, 746, 746, 746, - 746, 746, 62, 63, 788, 787, 789, 787, - 66, 68, 790, 791, 792, 793, 795, 796, - 797, 798, 800, 801, 802, 803, 804, 806, - 807, 808, 809, 810, 811, 812, 813, 814, - 815, 816, 790, 71, 72, 81, 86, 96, - 101, 107, 114, 790, 122, 790, 790, 142, - 790, 794, 790, 155, 162, 790, 149, 152, - 172, 184, 202, 208, 214, 222, 226, 229, - 240, 246, 250, 790, 241, 243, 256, 270, - 300, 306, 324, 790, 790, 334, 337, 341, - 790, 790, 790, 790, 790, 350, 790, 357, - 790, 805, 790, 377, 384, 790, 368, 790, - 790, 389, 398, 790, 790, 399, 402, 438, - 441, 790, 790, 790, 790, 790, 446, 790, - 790, 790, 459, 463, 790, 466, 790, 480, - 485, 790, 790, 790, 490, 497, 510, 530, - 534, 538, 548, 554, 555, 556, 557, 558, - 790, 790, 790, 790, 790, 818, 818, 818, - 818, 818, 818, 818, 818, 819, 819, 820, - 821, 819, 819, 833, 834, 835, 819, 566, - 819, 822, 824, 819, 819, 819, 819, 819, - 819, 826, 819, 819, 819, 819, 819, 819, - 827, 819, 819, 819, 819, 819, 819, 828, - 829, 819, 819, 819, 819, 819, 830, 819, - 823, 819, 819, 825, 819, 819, 819, 819, - 819, 819, 819, 734, 819, 819, 819, 819, - 831, 831, 831, 832, 819, 819, 819, 819, - 819, 819, 737, 738, 836, 837, 836, 836, - 836, 836, 836, 838, 839, 838, 840, 841, - 842, 838, 838, 838, 838, 740, 741, 843, - 844, 843, 845, 846, 847, 843, 843, 843, - 843, 743, 744, 848, 848, 849, 849 -}; - -static const short _regex_trans_actions[] = { - 827, 631, 765, 731, 723, 45, 903, 903, - 897, 45, 912, 45, 900, 903, 729, 741, - 0, 45, 45, 923, 737, 841, 747, 0, - 743, 3, 839, 3, 0, 761, 3, 759, - 870, 3, 751, 0, 749, 755, 753, 0, - 757, 3, 0, 745, 0, 725, 727, 27, - 3, 763, 0, 3, 673, 0, 25, 829, - 0, 0, 0, 603, 0, 0, 0, 601, - 0, 0, 0, 831, 0, 675, 17, 0, - 7, 870, 3, 17, 17, 663, 17, 870, - 661, 870, 665, 837, 3, 671, 3, 669, - 3, 667, 833, 0, 677, 835, 0, 679, - 845, 0, 11, 29, 13, 31, 0, 843, - 769, 0, 771, 59, 0, 53, 0, 51, - 0, 49, 0, 47, 359, 315, 0, 0, - 0, 0, 127, 0, 0, 0, 0, 129, - 0, 0, 0, 0, 131, 0, 0, 0, - 0, 0, 0, 0, 133, 0, 135, 0, - 137, 0, 0, 0, 0, 139, 0, 0, - 0, 0, 0, 141, 0, 0, 0, 0, - 143, 0, 0, 145, 0, 0, 0, 0, - 0, 0, 147, 0, 149, 341, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 151, - 0, 0, 153, 0, 0, 155, 0, 0, - 0, 0, 157, 343, 0, 0, 159, 0, - 0, 161, 0, 0, 0, 0, 0, 0, - 163, 0, 0, 0, 0, 0, 165, 0, - 0, 0, 0, 167, 0, 0, 0, 0, - 0, 169, 0, 0, 0, 0, 0, 0, - 171, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 173, 0, 0, 0, 0, 0, - 175, 0, 0, 0, 0, 0, 177, 0, - 0, 0, 0, 0, 0, 0, 179, 0, - 0, 0, 181, 0, 0, 183, 0, 0, - 0, 0, 0, 0, 185, 0, 0, 0, - 0, 187, 45, 357, 0, 189, 0, 0, - 191, 0, 0, 0, 193, 0, 0, 0, - 0, 0, 195, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 197, 0, 0, 0, 0, 0, 0, 0, - 199, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 201, 0, 0, 0, 0, - 203, 0, 0, 0, 0, 0, 205, 0, - 0, 0, 0, 0, 0, 207, 0, 0, - 0, 209, 0, 0, 0, 0, 211, 0, - 0, 0, 0, 213, 0, 0, 0, 0, - 0, 0, 0, 0, 215, 0, 217, 345, - 219, 0, 0, 221, 0, 0, 0, 223, - 0, 0, 0, 0, 225, 0, 0, 0, - 0, 227, 229, 0, 0, 0, 0, 231, - 0, 0, 233, 347, 0, 0, 0, 0, - 0, 0, 0, 235, 0, 0, 0, 237, - 349, 0, 0, 0, 0, 0, 0, 0, - 0, 239, 0, 0, 0, 0, 0, 0, - 241, 0, 0, 0, 0, 243, 351, 0, - 0, 0, 0, 0, 0, 0, 0, 245, - 247, 0, 0, 249, 0, 0, 0, 0, - 0, 0, 251, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 253, 0, 0, 0, - 0, 0, 255, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 257, 0, - 0, 0, 0, 259, 0, 0, 261, 0, - 0, 0, 0, 263, 353, 0, 0, 0, - 0, 0, 0, 265, 0, 0, 0, 0, - 0, 0, 267, 0, 0, 0, 269, 0, - 0, 271, 355, 0, 0, 0, 0, 0, - 0, 0, 273, 0, 0, 0, 0, 0, - 0, 275, 0, 0, 0, 0, 277, 0, - 0, 0, 0, 279, 0, 0, 0, 0, - 0, 0, 281, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 283, 0, 0, - 285, 0, 0, 0, 0, 0, 0, 0, - 287, 0, 0, 0, 289, 0, 0, 0, - 0, 291, 0, 0, 293, 0, 0, 295, - 0, 297, 0, 0, 0, 299, 0, 0, - 303, 0, 301, 0, 0, 0, 0, 0, - 305, 0, 0, 0, 0, 307, 0, 0, - 0, 0, 0, 309, 311, 119, 121, 123, - 125, 39, 0, 35, 33, 37, 539, 0, - 0, 0, 377, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 435, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 381, 0, 0, - 0, 385, 0, 0, 0, 0, 389, 0, - 0, 0, 0, 0, 393, 0, 0, 0, - 0, 0, 397, 0, 0, 0, 0, 0, - 401, 0, 0, 0, 0, 0, 405, 0, - 0, 0, 0, 0, 409, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, - 417, 0, 0, 0, 0, 0, 421, 0, - 0, 0, 0, 0, 425, 0, 0, 0, - 0, 429, 0, 0, 0, 0, 0, 0, - 433, 0, 0, 0, 0, 0, 0, 0, - 379, 0, 0, 0, 383, 0, 0, 0, - 0, 387, 0, 0, 0, 0, 0, 391, - 0, 0, 0, 0, 0, 395, 0, 0, - 0, 0, 0, 399, 0, 0, 0, 0, - 0, 403, 0, 0, 0, 0, 0, 407, - 0, 0, 0, 0, 0, 0, 411, 0, - 0, 0, 0, 415, 0, 0, 0, 0, - 0, 419, 0, 0, 0, 0, 0, 423, - 0, 0, 0, 0, 427, 0, 0, 0, - 0, 0, 0, 431, 0, 0, 0, 533, - 0, 471, 535, 0, 475, 537, 503, 0, - 505, 569, 557, 0, 559, 587, 575, 0, - 577, 633, 0, 777, 775, 637, 45, 597, - 0, 0, 609, 0, 45, 0, 635, 909, - 599, 773, 0, 45, 45, 629, 779, 0, - 0, 821, 819, 1, 855, 855, 1, 0, - 3, 735, 733, 739, 1, 1, 0, 783, - 615, 613, 785, 619, 617, 787, 623, 621, - 781, 817, 721, 5, 852, 915, 639, 647, - 611, 695, 607, 717, 699, 715, 683, 0, - 605, 713, 691, 703, 687, 719, 641, 657, - 645, 0, 693, 659, 655, 906, 697, 45, - 651, 45, 0, 653, 689, 649, 701, 685, - 7, 643, 791, 15, 867, 795, 858, 919, - 793, 927, 847, 811, 711, 709, 809, 681, - 801, 7, 17, 849, 799, 17, 876, 797, - 17, 873, 815, 1, 1, 1, 803, 0, - 813, 707, 705, 805, 19, 23, 21, 45, - 882, 894, 888, 807, 825, 789, 627, 625, - 823, 767, 0, 0, 45, 55, 45, 57, - 0, 0, 317, 45, 45, 45, 45, 45, - 45, 45, 45, 45, 45, 45, 45, 45, - 45, 45, 45, 45, 45, 45, 45, 45, - 0, 0, 339, 0, 0, 0, 0, 0, - 0, 0, 0, 319, 0, 61, 63, 0, - 65, 45, 67, 0, 0, 321, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 337, 0, 0, 0, 0, - 0, 0, 0, 323, 79, 0, 0, 0, - 69, 71, 73, 75, 77, 0, 325, 0, - 81, 45, 83, 0, 0, 327, 0, 329, - 85, 0, 0, 87, 89, 0, 0, 0, - 0, 331, 91, 93, 95, 97, 0, 99, - 101, 103, 0, 0, 333, 0, 105, 0, - 0, 107, 109, 111, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 313, 335, 113, 115, 117, 375, 361, 363, - 365, 367, 369, 371, 373, 509, 491, 45, - 0, 511, 507, 0, 45, 45, 531, 0, - 499, 5, 9, 473, 497, 489, 439, 457, - 493, 0, 437, 485, 461, 481, 451, 441, - 0, 487, 453, 449, 495, 455, 445, 45, - 0, 447, 483, 443, 459, 479, 7, 517, - 15, 861, 519, 15, 864, 513, 469, 467, - 527, 477, 521, 0, 515, 465, 463, 523, - 19, 23, 21, 45, 879, 891, 885, 525, - 529, 501, 0, 0, 549, 0, 543, 541, - 551, 547, 545, 563, 0, 561, 0, 45, - 45, 567, 553, 565, 555, 0, 0, 581, - 0, 579, 0, 45, 45, 585, 571, 583, - 573, 0, 0, 591, 589, 595, 593 -}; - -static const short _regex_to_state_actions[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 41, - 0, 41, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 41, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 41, 0, 0, 41, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 41, 41, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 41, 0, 41, 0, - 0, 0, 0, 41, 0, 0, 0, 0, - 41, 41 -}; - -static const short _regex_from_state_actions[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 43, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 43, 0, 0, 43, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 43, 43, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 43, 0, 43, 0, - 0, 0, 0, 43, 0, 0, 0, 0, - 43, 43 -}; - -static const short _regex_eof_actions[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 39, - 39, 39, 39, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0 -}; - -static const short _regex_eof_trans[] = { - 0, 1, 1, 22, 22, 27, 27, 27, - 27, 27, 27, 27, 27, 27, 27, 27, - 27, 27, 27, 22, 22, 22, 22, 22, - 22, 22, 22, 22, 22, 56, 56, 56, - 56, 56, 56, 56, 56, 56, 68, 68, - 68, 68, 68, 68, 68, 68, 68, 68, - 68, 68, 84, 84, 84, 84, 84, 84, - 91, 91, 94, 97, 97, 97, 104, 104, - 104, 108, 108, 108, 108, 108, 108, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 174, 174, 174, 174, 174, 174, - 174, 174, 174, 174, 174, 174, 174, 174, - 174, 174, 174, 174, 174, 174, 174, 174, - 174, 174, 174, 174, 174, 204, 204, 204, - 204, 204, 204, 174, 174, 174, 174, 174, - 174, 174, 174, 174, 174, 174, 174, 174, - 174, 174, 174, 174, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 300, 300, 300, 300, 300, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 400, 400, - 400, 400, 400, 400, 400, 400, 400, 400, - 400, 400, 400, 400, 400, 400, 400, 400, - 400, 400, 400, 400, 400, 428, 428, 428, - 428, 428, 428, 428, 428, 428, 428, 428, - 441, 441, 441, 441, 441, 441, 441, 441, - 441, 428, 428, 428, 428, 428, 428, 428, - 428, 428, 428, 428, 428, 463, 463, 463, - 463, 463, 463, 463, 463, 463, 463, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 525, 525, - 525, 525, 525, 525, 525, 525, 525, 525, - 525, 525, 525, 117, 117, 117, 117, 117, - 117, 117, 547, 547, 547, 547, 547, 547, - 547, 547, 547, 547, 547, 547, 547, 547, - 547, 547, 547, 547, 547, 547, 547, 547, - 547, 547, 547, 547, 547, 547, 547, 547, - 547, 547, 547, 547, 547, 547, 547, 547, - 547, 547, 547, 547, 547, 547, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 117, - 117, 117, 117, 117, 117, 117, 117, 0, - 0, 0, 0, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 655, 655, - 655, 655, 655, 655, 655, 655, 856, 856, - 859, 862, 862, 862, 866, 866, 866, 870, - 870, 870, 0, 895, 898, 898, 899, 898, - 898, 898, 898, 898, 898, 898, 898, 912, - 915, 918, 921, 922, 963, 963, 966, 969, - 971, 972, 975, 977, 981, 981, 984, 984, - 987, 991, 993, 996, 996, 1004, 1005, 1006, - 1009, 1009, 1009, 0, 1016, 1016, 0, 1043, - 1043, 1052, 1062, 1043, 1043, 1043, 1043, 1076, - 1043, 1043, 1043, 1084, 1095, 1102, 1104, 1043, - 1114, 1043, 1125, 1043, 1043, 1043, 1043, 1043, - 1146, 0, 0, 0, 1167, 1167, 1200, 1200, - 1203, 1203, 1206, 1209, 1211, 1213, 1216, 1216, - 1224, 1225, 1225, 1225, 0, 1233, 0, 1242, - 1244, 1244, 1244, 0, 1254, 1256, 1256, 1256, - 0, 0 -}; - -static const int regex_start = 746; -static const int regex_error = 0; - -static const int regex_en_readVerb = 787; -static const int regex_en_readUCP = 790; -static const int regex_en_readBracedUCP = 559; -static const int regex_en_readUCPSingle = 818; -static const int regex_en_charClassGuts = 819; -static const int regex_en_readClass = 836; -static const int regex_en_readQuotedLiteral = 838; -static const int regex_en_readQuotedClass = 843; -static const int regex_en_readComment = 848; -static const int regex_en_readNewlineTerminatedComment = 849; -static const int regex_en_main = 746; - - -#line 1912 "Parser.rl" - -/** \brief Main parser call, returns root Component or nullptr. */ -unique_ptr parse(const char *ptr, ParseMode &globalMode) { - assert(ptr); - - const char *p = ptr; - const char *pe = ptr + strlen(ptr); - - // First, read the control verbs, set any global mode flags and move the - // ptr forward. - p = read_control_verbs(p, pe, 0, globalMode); - - const char *eof = pe; - int cs; - UNUSED int act; - int top; - vector stack; - const char *ts, *te; - unichar accumulator = 0; - unichar octAccumulator = 0; /* required as we are also accumulating for - * back ref when looking for octals */ - unsigned repeatN = 0; - unsigned repeatM = 0; - string label; - - ParseMode mode = globalMode; - ParseMode newMode; - - bool negated = false; - bool inComment = false; - - // Stack of sequences and flags used to store state when we enter - // sub-sequences. - vector sequences; - - // Index of the next capturing group. Note that zero is reserved for the - // root sequence. - unsigned groupIndex = 1; - - // Set storing group names that are currently in use. - flat_set groupNames; - - // Root sequence. - unique_ptr rootSeq = std::make_unique(); - rootSeq->setCaptureIndex(0); - - // Current sequence being appended to - ComponentSequence *currentSeq = rootSeq.get(); - - // The current character class being appended to. This is used as the - // accumulator for both character class and UCP properties. - unique_ptr currentCls; - - // True if the machine is currently inside a character class, i.e. square - // brackets [..]. - bool inCharClass = false; - - // True if the machine is inside a character class but it has not processed - // any "real" elements yet, i.e. it's still processing meta-characters like - // '^'. - bool inCharClassEarly = false; - - // Location at which the current character class began. - const char *currentClsBegin = p; - - // We throw exceptions on various parsing failures beyond this point: we - // use a try/catch block here to clean up our allocated memory before we - // re-throw the exception to the caller. - try { - // Embed the Ragel machine here - -#line 2533 "Parser.cpp" +#line 4187 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" { - cs = regex_start; - top = 0; - ts = 0; - te = 0; - act = 0; - } + cs = (int)regex_start; + top = 0; + ts = 0; + te = 0; + act = 0; + } + +#line 1982 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" -#line 1983 "Parser.rl" - -#line 2544 "Parser.cpp" + +#line 4196 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" { - int _klen; - unsigned int _trans; - short _widec; - const short *_acts; - unsigned int _nacts; - const short *_keys; - - if ( p == pe ) - goto _test_eof; - if ( cs == 0 ) - goto _out; -_resume: - _acts = _regex_actions + _regex_from_state_actions[cs]; - _nacts = (unsigned int) *_acts++; - while ( _nacts-- > 0 ) { - switch ( *_acts++ ) { - case 24: + int _cpc; + int _klen; + const signed char * _ckeys; + unsigned int _trans = 0; + unsigned int _cond = 0; + const char * _keys; + const short * _acts; + unsigned int _nacts; + _resume: {} + if ( p == pe && p != eof ) + goto _out; + _acts = ( _regex_actions + (_regex_from_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 24: { + { #line 1 "NONE" - {ts = p;} - break; -#line 2566 "Parser.cpp" - } - } + {ts = p;}} + +#line 4218 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" - _widec = (*p); - _klen = _regex_cond_lengths[cs]; - _keys = _regex_cond_keys + (_regex_cond_offsets[cs]*2); - if ( _klen > 0 ) { - const short *_lower = _keys; - const short *_mid; - const short *_upper = _keys + (_klen<<1) - 2; - while (1) { - if ( _upper < _lower ) - break; - - _mid = _lower + (((_upper-_lower) >> 1) & ~1); - if ( _widec < _mid[0] ) - _upper = _mid - 2; - else if ( _widec > _mid[1] ) - _lower = _mid + 2; - else { - switch ( _regex_cond_spaces[_regex_cond_offsets[cs] + ((_mid - _keys)>>1)] ) { - case 0: { - _widec = (short)(128 + ((*p) - -128)); - if ( -#line 475 "Parser.rl" - mode.utf8 ) _widec += 256; - break; - } - case 1: { - _widec = (short)(1152 + ((*p) - -128)); - if ( -#line 476 "Parser.rl" - mode.ignore_space ) _widec += 256; - break; - } - case 2: { - _widec = (short)(640 + ((*p) - -128)); - if ( -#line 477 "Parser.rl" - inCharClassEarly ) _widec += 256; - break; - } + break; + } + } + _nacts -= 1; + _acts += 1; } - break; + + if ( p == eof ) { + if ( _regex_eof_trans[cs] > 0 ) { + _trans = (unsigned int)_regex_eof_trans[cs] - 1; + } + } + else { + _keys = ( _regex_trans_keys + (_regex_key_offsets[cs])); + _trans = (unsigned int)_regex_index_offsets[cs]; + + _klen = (int)_regex_single_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + _klen - 1; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _keys += _klen; + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + ((_upper-_lower) >> 1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 1; + else if ( ( (*( p))) > (*( _mid)) ) + _lower = _mid + 1; + else { + _trans += (unsigned int)(_mid - _keys); + goto _match; + } + } + } + + _klen = (int)_regex_range_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + (_klen<<1) - 2; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + (((_upper-_lower) >> 1) & ~1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 2; + else if ( ( (*( p))) > (*( _mid + 1)) ) + _lower = _mid + 2; + else { + _trans += (unsigned int)((_mid - _keys)>>1); + break; + } + } + } + + _match: {} + } + _ckeys = ( _regex_cond_keys + (_regex_trans_offsets[_trans])); + _klen = (int)_regex_trans_lengths[_trans]; + _cond = (unsigned int)_regex_trans_offsets[_trans]; + + _cpc = 0; + switch ( _regex_trans_cond_spaces[_trans] ) { + + case 0: { + if ( ( mode.utf8 ) +#line 4293 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + ) _cpc += 1; + break; + } + case 1: { + if ( ( mode.ignore_space ) +#line 4298 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + ) _cpc += 1; + break; + } + case 2: { + if ( ( inCharClassEarly ) +#line 4303 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + ) _cpc += 1; + break; + } + } + { + const signed char *_lower = _ckeys; + const signed char *_upper = _ckeys + _klen - 1; + const signed char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _cond = 3561; + + break; + } + + _mid = _lower + ((_upper-_lower) >> 1); + if ( _cpc < (int)(*( _mid)) ) + _upper = _mid - 1; + else if ( _cpc > (int)(*( _mid)) ) + _lower = _mid + 1; + else { + _cond += (unsigned int)(_mid - _ckeys); + + break; + } + } + } + cs = (int)_regex_cond_targs[_cond]; + + if ( _regex_cond_actions[_cond] != 0 ) { + + _acts = ( _regex_actions + (_regex_cond_actions[_cond])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) + { + case 0: { + { +#line 285 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + label.clear();} + +#line 4345 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 1: { + { +#line 286 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + label.push_back((( (*( p)))));} + +#line 4353 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 2: { + { +#line 287 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + octAccumulator = 0;} + +#line 4361 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 3: { + { +#line 288 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + accumulator = 0;} + +#line 4369 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 4: { + { +#line 289 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + octAccumulator = 0; + pushOct(&octAccumulator, (( (*( p))))); + } + +#line 4380 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 5: { + { +#line 293 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + accumulator = 0; + pushDec(&accumulator, (( (*( p))))); + } + +#line 4391 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 6: { + { +#line 297 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + repeatN = 0; repeatM = 0; } + +#line 4399 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 7: { + { +#line 298 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + pushDec(&repeatN, (( (*( p))))); } + +#line 4407 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 8: { + { +#line 299 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + pushDec(&repeatM, (( (*( p))))); } + +#line 4415 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 9: { + { +#line 300 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + pushOct(&octAccumulator, (( (*( p))))); } + +#line 4423 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 10: { + { +#line 301 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + pushDec(&accumulator, (( (*( p))))); } + +#line 4431 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 11: { + { +#line 302 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + accumulator *= 16; + accumulator += (( (*( p)))) - '0'; + } + +#line 4442 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 12: { + { +#line 306 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + accumulator *= 16; + accumulator += 10 + (( (*( p)))) - 'a'; + } + +#line 4453 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 13: { + { +#line 310 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + accumulator *= 16; + accumulator += 10 + (( (*( p)))) - 'A'; + } + +#line 4464 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 14: { + { +#line 430 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + newMode = mode; + } + +#line 4474 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 15: { + { +#line 437 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + switch ((( (*( p))))) { + case 'i': + newMode.caseless = true; + break; + case 'm': + newMode.multiline = true; + break; + case 's': + newMode.dotall = true; + break; + case 'x': + newMode.ignore_space = true; + break; + default: + assert(0); // this action only called for [imsx] + break; + } + } + +#line 4500 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 16: { + { +#line 456 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + switch ((( (*( p))))) { + case 'i': + newMode.caseless = false; + break; + case 'm': + newMode.multiline = false; + break; + case 's': + newMode.dotall = false; + break; + case 'x': + newMode.ignore_space = false; + break; + default: + assert(0); // this action only called for [imsx] + break; + } + } + +#line 4526 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 17: { + { +#line 510 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + repeatM = repeatN;} + +#line 4534 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 18: { + { +#line 510 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + repeatM = ComponentRepeat::NoLimit;} + +#line 4542 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 19: { + { +#line 722 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + negated = !negated; } + +#line 4550 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 20: { + { +#line 723 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = p - 1; } {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 790;goto _again;}} + +#line 4566 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 21: { + { +#line 724 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + if (!inCharClass) { // not inside [..] + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + +#line 4579 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 22: { + { +#line 730 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Malformed property"); } + +#line 4587 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 25: { + { +#line 1 "NONE" + {te = p+1;}} + +#line 4595 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 26: { + { +#line 550 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 550 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("(*UTF8) must be at start of " + "expression, encountered"); + } + }} + +#line 4609 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 27: { + { +#line 554 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 554 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("(*UTF) must be at start of " + "expression, encountered"); + } + }} + +#line 4623 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 28: { + { +#line 558 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 558 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("(*UCP) must be at start of " + "expression, encountered"); + } + }} + +#line 4637 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 29: { + { +#line 564 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 564 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ParseMode temp_mode; + assert(ts - 2 >= ptr); // parser needs the '(*' at the start too. + read_control_verbs(ts - 2, te, (ts - 2 - ptr), temp_mode); + assert(0); // Should have thrown a parse error. + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4654 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 30: { + { +#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4667 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 31: { + { +#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4680 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 32: { + { +#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4694 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 33: { + { +#line 581 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 581 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_CC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4705 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 34: { + { +#line 582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_CF, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4716 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 35: { + { +#line 583 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 583 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_CN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4727 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 36: { + { +#line 585 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 585 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_CS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4738 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 37: { + { +#line 587 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 587 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_LL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4749 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 38: { + { +#line 588 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 588 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_LM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4760 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 39: { + { +#line 589 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 589 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_LO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4771 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 40: { + { +#line 590 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 590 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_LT, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4782 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 41: { + { +#line 591 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 591 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_LU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4793 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 42: { + { +#line 592 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 592 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_L_AND, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4804 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 43: { + { +#line 594 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 594 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_MC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4815 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 44: { + { +#line 596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_MN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4826 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 45: { + { +#line 598 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 598 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ND, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4837 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 46: { + { +#line 599 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 599 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_NL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4848 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 47: { + { +#line 600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_NO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4859 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 48: { + { +#line 602 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 602 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4870 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 49: { + { +#line 603 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 603 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PD, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4881 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 50: { + { +#line 604 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 604 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4892 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 51: { + { +#line 605 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 605 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PF, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4903 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 52: { + { +#line 606 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 606 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4914 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 53: { + { +#line 607 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 607 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4925 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 54: { + { +#line 608 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 608 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_PS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4936 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 55: { + { +#line 610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_SC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4947 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 56: { + { +#line 611 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 611 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_SK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4958 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 57: { + { +#line 612 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 612 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_SM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4969 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 58: { + { +#line 613 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 613 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_SO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4980 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 59: { + { +#line 615 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 615 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ZL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4991 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 60: { + { +#line 616 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 616 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ZP, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5002 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 61: { + { +#line 617 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 617 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ZS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5013 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 62: { + { +#line 618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_XAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5024 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 63: { + { +#line 619 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 619 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_XPS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5035 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 64: { + { +#line 620 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 620 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_XSP, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5046 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 65: { + { +#line 621 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 621 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_XWD, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5057 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 66: { + { +#line 622 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 622 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_ARABIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5068 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 67: { + { +#line 623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_ARMENIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5079 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 68: { + { +#line 624 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 624 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_AVESTAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5090 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 69: { + { +#line 625 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 625 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BALINESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5101 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 70: { + { +#line 626 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 626 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BAMUM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5112 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 71: { + { +#line 627 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 627 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BATAK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5123 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 72: { + { +#line 628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BENGALI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5134 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 73: { + { +#line 629 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 629 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BOPOMOFO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5145 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 74: { + { +#line 630 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 630 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BRAHMI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5156 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 75: { + { +#line 631 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 631 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BRAILLE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5167 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 76: { + { +#line 632 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 632 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BUGINESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5178 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 77: { + { +#line 633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_BUHID, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5189 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 78: { + { +#line 634 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 634 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CANADIAN_ABORIGINAL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5200 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 79: { + { +#line 635 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 635 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CARIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5211 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 80: { + { +#line 636 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 636 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CHAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5222 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 81: { + { +#line 637 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 637 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CHEROKEE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5233 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 82: { + { +#line 638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_COMMON, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5244 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 83: { + { +#line 639 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 639 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_COPTIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5255 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 84: { + { +#line 640 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 640 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CUNEIFORM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5266 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 85: { + { +#line 641 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 641 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CYPRIOT, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5277 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 86: { + { +#line 642 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 642 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_CYRILLIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5288 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 87: { + { +#line 643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_DESERET, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5299 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 88: { + { +#line 644 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 644 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_DEVANAGARI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5310 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 89: { + { +#line 645 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 645 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_EGYPTIAN_HIEROGLYPHS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5321 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 90: { + { +#line 646 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 646 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_ETHIOPIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5332 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 91: { + { +#line 647 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 647 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_GEORGIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5343 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 92: { + { +#line 648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_GLAGOLITIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5354 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 93: { + { +#line 649 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 649 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_GOTHIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5365 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 94: { + { +#line 650 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 650 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_GREEK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5376 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 95: { + { +#line 651 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 651 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_GUJARATI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5387 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 96: { + { +#line 652 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 652 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_GURMUKHI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5398 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 97: { + { +#line 654 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 654 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_HANGUL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5409 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 98: { + { +#line 655 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 655 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_HANUNOO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5420 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 99: { + { +#line 656 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 656 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_HEBREW, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5431 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 100: { + { +#line 657 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 657 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_HIRAGANA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5442 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 101: { + { +#line 658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_IMPERIAL_ARAMAIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5453 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 102: { + { +#line 659 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 659 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_INHERITED, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5464 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 103: { + { +#line 660 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 660 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PAHLAVI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5475 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 104: { + { +#line 661 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 661 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PARTHIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5486 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 105: { + { +#line 662 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 662 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_JAVANESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5497 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 106: { + { +#line 663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_KAITHI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5508 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 107: { + { +#line 664 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 664 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_KANNADA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5519 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 108: { + { +#line 665 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 665 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_KATAKANA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5530 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 109: { + { +#line 666 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 666 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_KAYAH_LI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5541 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 110: { + { +#line 667 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 667 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_KHAROSHTHI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5552 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 111: { + { +#line 668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_KHMER, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5563 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 112: { + { +#line 669 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 669 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LAO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5574 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 113: { + { +#line 670 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 670 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LATIN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5585 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 114: { + { +#line 671 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 671 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LEPCHA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5596 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 115: { + { +#line 672 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 672 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LIMBU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5607 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 116: { + { +#line 673 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 673 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LINEAR_B, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5618 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 117: { + { +#line 674 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 674 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LISU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5629 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 118: { + { +#line 675 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 675 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LYCIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5640 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 119: { + { +#line 676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_LYDIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5651 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 120: { + { +#line 677 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 677 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_MALAYALAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5662 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 121: { + { +#line 678 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 678 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_MANDAIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5673 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 122: { + { +#line 679 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 679 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_MEETEI_MAYEK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5684 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 123: { + { +#line 680 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 680 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_MONGOLIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5695 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 124: { + { +#line 681 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 681 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_MYANMAR, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5706 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 125: { + { +#line 682 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 682 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_NEW_TAI_LUE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5717 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 126: { + { +#line 683 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 683 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_NKO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5728 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 127: { + { +#line 684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OGHAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5739 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 128: { + { +#line 685 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 685 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OL_CHIKI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5750 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 129: { + { +#line 686 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 686 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_ITALIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5761 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 130: { + { +#line 687 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 687 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_PERSIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5772 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 131: { + { +#line 688 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 688 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_SOUTH_ARABIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5783 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 132: { + { +#line 689 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 689 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_TURKIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5794 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 133: { + { +#line 690 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 690 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_ORIYA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5805 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 134: { + { +#line 691 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 691 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_OSMANYA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5816 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 135: { + { +#line 692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_PHAGS_PA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5827 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 136: { + { +#line 693 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 693 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_PHOENICIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5838 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 137: { + { +#line 694 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 694 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_REJANG, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5849 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 138: { + { +#line 695 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 695 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_RUNIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5860 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 139: { + { +#line 696 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 696 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SAMARITAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5871 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 140: { + { +#line 697 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 697 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SAURASHTRA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5882 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 141: { + { +#line 698 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 698 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SHAVIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5893 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 142: { + { +#line 699 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 699 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SINHALA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5904 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 143: { + { +#line 700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SUNDANESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5915 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 144: { + { +#line 701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SYLOTI_NAGRI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5926 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 145: { + { +#line 702 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 702 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_SYRIAC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5937 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 146: { + { +#line 703 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 703 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TAGALOG, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5948 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 147: { + { +#line 704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TAGBANWA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5959 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 148: { + { +#line 705 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 705 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TAI_LE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5970 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 149: { + { +#line 706 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 706 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TAI_THAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5981 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 150: { + { +#line 707 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 707 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TAI_VIET, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5992 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 151: { + { +#line 708 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 708 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TAMIL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6003 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 152: { + { +#line 709 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 709 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TELUGU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6014 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 153: { + { +#line 710 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 710 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_THAANA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6025 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 154: { + { +#line 711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_THAI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6036 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 155: { + { +#line 712 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 712 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TIBETAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6047 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 156: { + { +#line 713 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 713 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_TIFINAGH, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6058 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 157: { + { +#line 714 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 714 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_UGARITIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6069 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 158: { + { +#line 715 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 715 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_VAI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6080 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 159: { + { +#line 716 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 716 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_YI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6091 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 160: { + { +#line 717 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 717 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ANY, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6102 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 161: { + { +#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6113 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 162: { + { +#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_C, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6124 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 163: { + { +#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_CO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6135 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 164: { + { +#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_L, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6146 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 165: { + { +#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_M, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6157 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 166: { + { +#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ME, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6168 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 167: { + { +#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_N, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6179 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 168: { + { +#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_P, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6190 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 169: { + { +#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_S, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6201 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 170: { + { +#line 614 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 614 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_Z, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6212 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 171: { + { +#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_HAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6223 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 172: { + { +#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6234 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 173: { + { +#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_C, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6246 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 174: { + { +#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_CO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6258 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 175: { + { +#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_L, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6270 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 176: { + { +#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_M, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6282 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 177: { + { +#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_ME, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6294 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 178: { + { +#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_N, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6306 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 179: { + { +#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_P, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6318 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 180: { + { +#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_UCP_S, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6330 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 181: { + { +#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + currentCls->add(CLASS_SCRIPT_HAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6342 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 182: { + { +#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6354 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 183: { + { +#line 733 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 733 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_C, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6372 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 184: { + { +#line 741 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 741 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_L, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6390 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 185: { + { +#line 749 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 749 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_M, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6408 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 186: { + { +#line 757 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 757 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_N, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6426 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 187: { + { +#line 765 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 765 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_P, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6444 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 188: { + { +#line 773 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 773 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_S, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6462 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 189: { + { +#line 781 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 781 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UCP_Z, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6480 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 190: { + { +#line 790 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 790 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6491 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 191: { + { +#line 796 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 796 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unsupported POSIX collating " + "element"); + } + }} + +#line 6505 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 192: { + { +#line 803 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 803 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_ALNUM, false); + } + }} + +#line 6518 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 193: { + { +#line 806 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 806 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_ALNUM, true); + } + }} + +#line 6531 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 194: { + { +#line 809 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 809 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_ALPHA, false); + } + }} + +#line 6544 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 195: { + { +#line 812 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 812 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_ALPHA, true); + } + }} + +#line 6557 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 196: { + { +#line 815 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 815 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_ASCII, false); + } + }} + +#line 6570 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 197: { + { +#line 818 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 818 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_ASCII, true); + } + }} + +#line 6583 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 198: { + { +#line 821 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 821 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_BLANK, false); + } + }} + +#line 6596 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 199: { + { +#line 824 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 824 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_BLANK, true); + } + }} + +#line 6609 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 200: { + { +#line 827 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 827 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_CNTRL, false); + } + }} + +#line 6622 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 201: { + { +#line 830 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 830 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_CNTRL, true); + } + }} + +#line 6635 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 202: { + { +#line 833 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 833 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_DIGIT, false); + } + }} + +#line 6648 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 203: { + { +#line 836 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 836 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_DIGIT, true); + } + }} + +#line 6661 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 204: { + { +#line 839 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 839 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_GRAPH, false); + } + }} + +#line 6674 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 205: { + { +#line 842 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 842 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_GRAPH, true); + } + }} + +#line 6687 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 206: { + { +#line 845 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 845 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_LOWER, false); + } + }} + +#line 6700 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 207: { + { +#line 848 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 848 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_LOWER, true); + } + }} + +#line 6713 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 208: { + { +#line 851 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 851 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_PRINT, false); + } + }} + +#line 6726 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 209: { + { +#line 854 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 854 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_PRINT, true); + } + }} + +#line 6739 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 210: { + { +#line 857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_PUNCT, false); + } + }} + +#line 6752 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 211: { + { +#line 860 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 860 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_PUNCT, true); + } + }} + +#line 6765 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 212: { + { +#line 864 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 864 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_SPACE, false); + } + }} + +#line 6778 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 213: { + { +#line 867 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 867 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_SPACE, true); + } + }} + +#line 6791 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 214: { + { +#line 870 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 870 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UPPER, false); + } + }} + +#line 6804 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 215: { + { +#line 873 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 873 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_UPPER, true); + } + }} + +#line 6817 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 216: { + { +#line 876 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 876 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_WORD, false); + } + }} + +#line 6830 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 217: { + { +#line 879 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 879 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_WORD, true); + } + }} + +#line 6843 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 218: { + { +#line 882 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 882 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_XDIGIT, false); + } + }} + +#line 6856 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 219: { + { +#line 885 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 885 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_XDIGIT, true); + } + }} + +#line 6869 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 220: { + { +#line 890 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 890 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Invalid POSIX named class"); + } + }} + +#line 6882 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 221: { + { +#line 893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 843;goto _again;}} + }} + +#line 6902 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 222: { + { +#line 896 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 896 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + /*noop*/} + }} + +#line 6913 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 223: { + { +#line 898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x08'); + } + }} + +#line 6926 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 224: { + { +#line 902 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 902 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x09'); + } + }} + +#line 6939 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 225: { + { +#line 906 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 906 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x0a'); + } + }} + +#line 6952 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 226: { + { +#line 910 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 910 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x0d'); + } + }} + +#line 6965 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 227: { + { +#line 914 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 914 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x0c'); + } + }} + +#line 6978 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 228: { + { +#line 918 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 918 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x07'); + } + }} + +#line 6991 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 229: { + { +#line 922 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 922 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add('\x1b'); + } + }} + +#line 7004 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 230: { + { +#line 926 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 926 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_HORZ, false); + } + }} + +#line 7017 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 231: { + { +#line 930 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 930 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_HORZ, true); + } + }} + +#line 7030 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 232: { + { +#line 934 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 934 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_VERT, false); + } + }} + +#line 7043 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 233: { + { +#line 938 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 938 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_VERT, true); + } + }} + +#line 7056 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 234: { + { +#line 942 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 942 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + negated = false; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 7078 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 235: { + { +#line 948 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 948 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + negated = false; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 7100 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 236: { + { +#line 954 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 954 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + negated = true; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 7122 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 237: { + { +#line 960 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 960 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + negated = true; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 7144 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 238: { + { +#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7157 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 239: { + { +#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7170 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 240: { + { +#line 977 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 977 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + string oct(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(oct, nullptr, 8); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if ((!mode.utf8 && val > 255) || val > MAX_UNICODE) { + throw LocatedParseError("Value in \\o{...} sequence is too large"); + } + currentCls->add((unichar)val); + } + }} + +#line 7193 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 241: { + { +#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(accumulator); + } + }} + +#line 7206 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 242: { + { +#line 1001 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1001 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // whatever we found here + currentCls->add(*(ts + 1)); + + } + }} + +#line 7221 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 243: { + { +#line 1007 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1007 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + string hex(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(hex, nullptr, 16); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if (val > MAX_UNICODE) { + throw LocatedParseError("Value in \\x{...} sequence is too large"); + } + currentCls->add((unichar)val); + } + }} + +#line 7244 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 244: { + { +#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + currentCls->add(decodeCtrl(ts[2])); + } + } + }} + +#line 7263 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 245: { + { +#line 1035 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1035 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_WORD, false); + } + }} + +#line 7276 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 246: { + { +#line 1039 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1039 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_WORD, true); + } + }} + +#line 7289 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 247: { + { +#line 1043 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1043 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_SPACE, false); + } + }} + +#line 7302 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 248: { + { +#line 1047 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1047 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_SPACE, true); + } + }} + +#line 7315 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 249: { + { +#line 1051 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1051 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_DIGIT, false); + } + }} + +#line 7328 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 250: { + { +#line 1055 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1055 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(CLASS_DIGIT, true); + } + }} + +#line 7341 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 251: { + { +#line 1058 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1058 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->addDash(); + } + }} + +#line 7354 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 252: { + { +#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported in a character class."; + throw ParseError(str.str()); + } + }} + +#line 7370 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 253: { + { +#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported in a character class."; + throw ParseError(str.str()); + } + }} + +#line 7386 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 254: { + { +#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported in a character class."; + throw ParseError(str.str()); + } + }} + +#line 7402 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 255: { + { +#line 1075 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1075 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // add the literal char + currentCls->add(*(ts + 1)); + } + }} + +#line 7416 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 256: { + { +#line 1081 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1081 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint2c(ts)); + } + }} + +#line 7430 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 257: { + { +#line 1086 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1086 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint3c(ts)); + } + }} + +#line 7444 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 258: { + { +#line 1091 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1091 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint4c(ts)); + } + }} + +#line 7458 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 259: { + { +#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7472 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 260: { + { +#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add((u8)*ts); + } + }} + +#line 7485 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 261: { + { +#line 1106 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1106 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + inCharClass = false; + {cs = 746;goto _again;}} + }} + +#line 7500 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 262: { + { +#line 966 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 966 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 7511 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 263: { + { +#line 967 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 967 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 7522 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 264: { + { +#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7535 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 265: { + { +#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7548 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 266: { + { +#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 7561 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 267: { + { +#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(accumulator); + } + }} + +#line 7574 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 268: { + { +#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 7587 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 269: { + { +#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + currentCls->add(decodeCtrl(ts[2])); + } + } + }} + +#line 7606 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 270: { + { +#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7620 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 271: { + { +#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add((u8)*ts); + } + }} + +#line 7633 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 272: { + { +#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 7647 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 273: { + { +#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 7661 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 274: { + { +#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7676 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 275: { + { +#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add((u8)*ts); + } + }} + +#line 7690 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 276: { + { +#line 1120 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1120 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (currentCls->isNegated()) { + // Already seen a caret; the second one is not a meta-character. + inCharClassEarly = false; + {p = p - 1; } {cs = 819;goto _again;}} else { + currentCls->negate(); + // Note: we cannot switch off inCharClassEarly here, as /[^]]/ + // needs to use the right square bracket path below. + } + } + }} + +#line 7710 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 277: { + { +#line 1133 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1133 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(']'); + inCharClassEarly = false; + } + }} + +#line 7724 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 278: { + { +#line 1138 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1138 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 843;goto _again;}} + }} + +#line 7743 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 279: { + { +#line 1139 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1139 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + /*noop*/} + }} + +#line 7754 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 280: { + { +#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + inCharClassEarly = false; + {p = p - 1; } + {cs = 819;goto _again;}} + }} + +#line 7768 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 281: { + { +#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + inCharClassEarly = false; + {p = p - 1; } + {cs = 819;goto _again;}} + }} + +#line 7782 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 282: { + { +#line 1154 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1154 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + {cs = 746;goto _again;}} + }} + +#line 7794 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 283: { + { +#line 1159 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1159 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint2c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 7812 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 284: { + { +#line 1168 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1168 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint3c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 7830 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 285: { + { +#line 1177 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1177 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint4c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 7848 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 286: { + { +#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7862 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 287: { + { +#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 7875 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 288: { + { +#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7889 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 289: { + { +#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 7902 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 290: { + { +#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7917 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 291: { + { +#line 1202 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1202 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 7930 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 292: { + { +#line 1207 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1207 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint2c(ts)); + inCharClassEarly = false; + } + }} + +#line 7945 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 293: { + { +#line 1213 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1213 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint3c(ts)); + inCharClassEarly = false; + } + }} + +#line 7960 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 294: { + { +#line 1219 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1219 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint4c(ts)); + inCharClassEarly = false; + } + }} + +#line 7975 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 295: { + { +#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7989 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 296: { + { +#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(*ts); + inCharClassEarly = false; + } + }} + +#line 8003 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 297: { + { +#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 8017 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 298: { + { +#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentCls->add(*ts); + inCharClassEarly = false; + } + }} + +#line 8031 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 299: { + { +#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 8046 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 300: { + { +#line 1243 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1243 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + inComment = false; {cs = 746;goto _again;}} + }} + +#line 8057 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 301: { + { +#line 1 "-" + {te = p+1;}} + +#line 8065 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 302: { + { +#line 1255 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1255 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + inComment = false; {cs = 746;goto _again;}} + }} + +#line 8076 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 303: { + { +#line 1 "-" + {te = p+1;}} + +#line 8084 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 304: { + { +#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {act = 288;}} + +#line 8092 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 305: { + { +#line 1508 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {act = 290;}} + +#line 8100 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 306: { + { +#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {act = 330;}} + +#line 8108 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 307: { + { +#line 362 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 362 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (sequences.empty()) { + throw LocatedParseError("Unmatched parentheses"); + } + currentSeq->finalize(); + POP_SEQUENCE; + } + }} + +#line 8125 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 308: { + { +#line 1274 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1274 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addAlternation(); + } + }} + +#line 8138 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 309: { + { +#line 1279 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1279 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("POSIX named classes are only " + "supported inside a class"); + } + }} + +#line 8152 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 310: { + { +#line 1286 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1286 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unsupported POSIX collating " + "element"); + } + }} + +#line 8166 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 311: { + { +#line 1293 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1293 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + {cs = 838;goto _again;}} + }} + +#line 8178 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 312: { + { +#line 1297 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1297 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + /* noop */ } + }} + +#line 8189 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 313: { + { +#line 1299 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1299 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(generateComponent(CLASS_ANY, false, mode)); + } + }} + +#line 8202 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 314: { + { +#line 1303 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1303 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (mode.utf8) { + throw LocatedParseError("\\C is unsupported in UTF8"); + } + currentSeq->addComponent(std::make_unique()); + } + }} + +#line 8218 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 315: { + { +#line 1317 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1317 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8234 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 316: { + { +#line 1324 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1324 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8250 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 317: { + { +#line 1338 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1338 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8266 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 318: { + { +#line 1345 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1345 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8282 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 319: { + { +#line 1359 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1359 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat( + 0, 1, ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8298 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 320: { + { +#line 1366 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1366 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat( + 0, 1, ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8314 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 321: { + { +#line 1383 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1383 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (repeatN > repeatM || repeatM == 0) { + throwInvalidRepeat(); + } else if (!currentSeq->addRepeat( + repeatN, repeatM, + ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8333 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 322: { + { +#line 1393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (repeatN > repeatM || repeatM == 0) { + throwInvalidRepeat(); + } else if (!currentSeq->addRepeat( + repeatN, repeatM, + ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8352 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 323: { + { +#line 322 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 322 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + inComment = true; + {cs = 849;goto _again;}} + }} + +#line 8365 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 324: { + { +#line 1410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = p - 1; } {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 787;goto _again;}} + }} + +#line 8384 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 325: { + { +#line 1414 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1414 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + assert(0); {p += 1; goto _out; } } + }} + +#line 8395 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 326: { + { +#line 1421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto bound = mode.multiline ? ComponentBoundary::BEGIN_LINE + : ComponentBoundary::BEGIN_STRING; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8410 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 327: { + { +#line 1428 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1428 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto bound = mode.multiline ? ComponentBoundary::END_LINE + : ComponentBoundary::END_STRING_OPTIONAL_LF; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8425 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 328: { + { +#line 1434 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1434 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto bound = ComponentBoundary::BEGIN_STRING; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8439 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 329: { + { +#line 1439 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1439 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto bound = ComponentBoundary::END_STRING_OPTIONAL_LF; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8453 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 330: { + { +#line 1444 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1444 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto bound = ComponentBoundary::END_STRING; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8467 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 331: { + { +#line 1449 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1449 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent( + std::make_unique(ts - ptr, false, mode)); + } + }} + +#line 8481 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 332: { + { +#line 1454 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1454 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent( + std::make_unique(ts - ptr, true, mode)); + } + }} + +#line 8495 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 333: { + { +#line 1464 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1464 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, '\x09', mode); + } + }} + +#line 8508 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 334: { + { +#line 1468 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1468 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, '\x0a', mode); + } + }} + +#line 8521 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 335: { + { +#line 1472 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1472 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, '\x0d', mode); + } + }} + +#line 8534 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 336: { + { +#line 1476 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1476 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, '\x0c', mode); + } + }} + +#line 8547 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 337: { + { +#line 1480 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1480 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, '\x07', mode); + } + }} + +#line 8560 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 338: { + { +#line 1484 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1484 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, '\x1b', mode); + } + }} + +#line 8573 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 339: { + { +#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, octAccumulator, mode); + } + }} + +#line 8586 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 340: { + { +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 8602 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 341: { + { +#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // Accumulator is a negative offset. + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + if (accumulator >= groupIndex) { + throw LocatedParseError("Invalid reference"); + } + unsigned idx = groupIndex - accumulator; + currentSeq->addComponent(std::make_unique(idx)); + } + }} + +#line 8623 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 342: { + { +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 8639 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 343: { + { +#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // Accumulator is a negative offset. + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + if (accumulator >= groupIndex) { + throw LocatedParseError("Invalid reference"); + } + unsigned idx = groupIndex - accumulator; + currentSeq->addComponent(std::make_unique(idx)); + } + }} + +#line 8660 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 344: { + { +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8673 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 345: { + { +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8686 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 346: { + { +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8699 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 347: { + { +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8712 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 348: { + { +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8725 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 349: { + { +#line 1549 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1549 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "Onigiruma subroutine call at index " << ts - ptr << + " not supported."; + throw ParseError(str.str()); + } + }} + +#line 8741 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 350: { + { +#line 1560 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1560 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + string oct(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(oct, nullptr, 8); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if ((!mode.utf8 && val > 255) || val > MAX_UNICODE) { + throw LocatedParseError("Value in \\o{...} sequence is too large"); + } + addEscapedOctal(currentSeq, (unichar)val, mode); + } + }} + +#line 8764 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 351: { + { +#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addEscapedHex(currentSeq, accumulator, mode); + } + }} + +#line 8777 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 352: { + { +#line 1582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + string hex(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(hex, nullptr, 16); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if (val > MAX_UNICODE) { + throw LocatedParseError("Value in \\x{...} sequence is too large"); + } + addEscapedHex(currentSeq, (unichar)val, mode); + } + }} + +#line 8800 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 353: { + { +#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + addLiteral(currentSeq, decodeCtrl(ts[2]), mode); + } + } + }} + +#line 8819 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 354: { + { +#line 1610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 8835 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 355: { + { +#line 1618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_WORD, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8849 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 356: { + { +#line 1623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_WORD, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8863 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 357: { + { +#line 1628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_SPACE, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8877 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 358: { + { +#line 1633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_SPACE, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8891 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 359: { + { +#line 1638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_DIGIT, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8905 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 360: { + { +#line 1643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_DIGIT, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8919 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 361: { + { +#line 1648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_HORZ, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8933 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 362: { + { +#line 1653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_HORZ, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8947 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 363: { + { +#line 1658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_VERT, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8961 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 364: { + { +#line 1663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto cc = generateComponent(CLASS_VERT, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8975 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 365: { + { +#line 1668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = false; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 8999 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 366: { + { +#line 1676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = false; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 9023 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 367: { + { +#line 1684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = true; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 9047 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 368: { + { +#line 1692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = true; + {p = p - 1; } + {{ +#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 9071 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 369: { + { +#line 1704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "\\R at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9086 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 370: { + { +#line 1711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "\\K at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9101 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 371: { + { +#line 1726 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1726 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "\\G at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9116 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 372: { + { +#line 1732 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1732 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + currentSeq->addComponent(std::make_unique(ts - ptr, mode)); + } + }} + +#line 9129 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 373: { + { +#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *(ts + 1), mode); + } + }} + +#line 9142 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 374: { + { +#line 316 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 316 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + inComment = true; + {cs = 848;goto _again;}} + }} + +#line 9155 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 375: { + { +#line 433 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 433 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + mode = newMode; + currentSeq->addComponent(std::make_unique()); + } + }} + +#line 9169 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 376: { + { +#line 355 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 355 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + mode = newMode; + currentSeq = + enterSequence(currentSeq, std::make_unique()); + } + }} + +#line 9185 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 377: { + { +#line 369 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 369 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKAHEAD, + ComponentAssertion::POS)); + } + }} + +#line 9201 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 378: { + { +#line 375 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 375 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKAHEAD, + ComponentAssertion::NEG)); + } + }} + +#line 9217 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 379: { + { +#line 381 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 381 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKBEHIND, + ComponentAssertion::POS)); + } + }} + +#line 9233 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 380: { + { +#line 387 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 387 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKBEHIND, + ComponentAssertion::NEG)); + } + }} + +#line 9249 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 381: { + { +#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Embedded code is not supported"); + } + }} + +#line 9262 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 382: { + { +#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Embedded code is not supported"); + } + }} + +#line 9275 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 383: { + { +#line 416 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 416 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique()); + } + }} + +#line 9290 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 384: { + { +#line 336 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 336 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!label.empty()); // should be guaranteed by machine + char c = *label.begin(); + if (c >= '0' && c <= '9') { + throw LocatedParseError("Group name cannot begin with a digit"); + } + if (!groupNames.insert(label).second) { + throw LocatedParseError("Two named subpatterns use the name '" + label + "'"); + } + PUSH_SEQUENCE; + auto seq = std::make_unique(); + seq->setCaptureIndex(groupIndex++); + seq->setCaptureName(label); + currentSeq = enterSequence(currentSeq, move(seq)); + } + }} + +#line 9315 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 385: { + { +#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Subpattern reference unsupported"); + } + }} + +#line 9328 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 386: { + { +#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Subpattern reference unsupported"); + } + }} + +#line 9341 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 387: { + { +#line 1783 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1783 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKAHEAD, ComponentAssertion::POS); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9361 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 388: { + { +#line 1794 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1794 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKAHEAD, ComponentAssertion::NEG); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9381 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 389: { + { +#line 1805 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1805 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKBEHIND, ComponentAssertion::POS); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9401 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 390: { + { +#line 1816 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1816 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKBEHIND, ComponentAssertion::NEG); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9421 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 391: { + { +#line 1828 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1828 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Pattern recursion not supported"); + } + }} + +#line 9434 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 392: { + { +#line 402 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 402 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(accumulator)); + } + }} + +#line 9452 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 393: { + { +#line 410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + assert(!label.empty()); + currentSeq = enterSequence(currentSeq, + std::make_unique(label)); + } + }} + +#line 9468 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 394: { + { +#line 1844 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1844 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "Callout at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9483 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 395: { + { +#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unrecognised character after (?"); + } + }} + +#line 9496 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 396: { + { +#line 1857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint2c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 9514 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 397: { + { +#line 1866 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1866 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint3c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 9532 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 398: { + { +#line 1875 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1875 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint4c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 9550 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 399: { + { +#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 9564 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 400: { + { +#line 1893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (mode.ignore_space == false) { + addLiteral(currentSeq, *ts, mode); + } + } + }} + +#line 9579 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 401: { + { +#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p+1;{ +#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 9592 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 402: { + { +#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + auto seq = std::make_unique(); + seq->setCaptureIndex(groupIndex++); + currentSeq = enterSequence(currentSeq, move(seq)); + } + }} + +#line 9608 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 403: { + { +#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!currentCls); + assert(!inCharClass); // not reentrant + currentCls = getComponentClass(mode); + inCharClass = true; + inCharClassEarly = true; + currentClsBegin = ts; + {cs = 836;goto _again;}} + }} + +#line 9626 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 404: { + { +#line 1310 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1310 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9642 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 405: { + { +#line 1331 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1331 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9658 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 406: { + { +#line 1352 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1352 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (!currentSeq->addRepeat( + 0, 1, ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9674 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 407: { + { +#line 1373 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1373 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (repeatN > repeatM || repeatM == 0) { + throwInvalidRepeat(); + } else if (!currentSeq->addRepeat( + repeatN, repeatM, + ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9693 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 408: { + { +#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, octAccumulator, mode); + } + }} + +#line 9706 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 409: { + { +#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // If there are enough capturing sub expressions, this may be + // a back reference + accumulator = parseAsDecimal(octAccumulator); + if (accumulator < groupIndex) { + currentSeq->addComponent(std::make_unique(accumulator)); + } else { + addEscapedOctal(currentSeq, octAccumulator, mode); + } + } + }} + +#line 9726 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 410: { + { +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 9742 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 411: { + { +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 9758 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 412: { + { +#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // Accumulator is a negative offset. + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + if (accumulator >= groupIndex) { + throw LocatedParseError("Invalid reference"); + } + unsigned idx = groupIndex - accumulator; + currentSeq->addComponent(std::make_unique(idx)); + } + }} + +#line 9779 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 413: { + { +#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Invalid reference after \\g"); + } + }} + +#line 9792 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 414: { + { +#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 9805 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 415: { + { +#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addEscapedHex(currentSeq, accumulator, mode); + } + }} + +#line 9818 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 416: { + { +#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 9831 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 417: { + { +#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + addLiteral(currentSeq, decodeCtrl(ts[2]), mode); + } + } + }} + +#line 9850 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 418: { + { +#line 1700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 9861 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 419: { + { +#line 1701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 9872 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 420: { + { +#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "\\k at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9887 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 421: { + { +#line 1742 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1742 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(ts + 1 == pe); + ostringstream str; + str << "Unescaped \\ at end of input, index " << ts - ptr << "."; + throw ParseError(str.str()); + } + }} + +#line 9903 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 422: { + { +#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Conditional subpattern unsupported"); + } + }} + +#line 9916 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 423: { + { +#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unrecognised character after (?"); + } + }} + +#line 9929 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 424: { + { +#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 9943 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 425: { + { +#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {te = p;p = p - 1;{ +#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 9956 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 426: { + { +#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + PUSH_SEQUENCE; + auto seq = std::make_unique(); + seq->setCaptureIndex(groupIndex++); + currentSeq = enterSequence(currentSeq, move(seq)); + } + }} + +#line 9973 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 427: { + { +#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(!currentCls); + assert(!inCharClass); // not reentrant + currentCls = getComponentClass(mode); + inCharClass = true; + inCharClassEarly = true; + currentClsBegin = ts; + {cs = 836;goto _again;}} + }} + +#line 9992 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 428: { + { +#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Invalid reference after \\g"); + } + }} + +#line 10006 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 429: { + { +#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 10020 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 430: { + { +#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 10034 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 431: { + { +#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + ostringstream str; + str << "\\k at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 10050 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 432: { + { +#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Conditional subpattern unsupported"); + } + }} + +#line 10064 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 433: { + { +#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + throw LocatedParseError("Unrecognised character after (?"); + } + }} + +#line 10078 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 434: { + { +#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 10093 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 435: { + { +#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + {p = ((te))-1; + { +#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 10107 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } + case 436: { + { +#line 1 "NONE" + {switch( act ) { + case 288: { + p = ((te))-1; + { +#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // If there are enough capturing sub expressions, this may be + // a back reference + accumulator = parseAsDecimal(octAccumulator); + if (accumulator < groupIndex) { + currentSeq->addComponent(std::make_unique(accumulator)); + } else { + addEscapedOctal(currentSeq, octAccumulator, mode); + } + } + break; + } + case 290: { + p = ((te))-1; + { +#line 1508 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + // if there are enough left parens to this point, back ref + if (accumulator < groupIndex) { + currentSeq->addComponent(std::make_unique(accumulator)); + } else { + // Otherwise, we interpret the first three digits as an + // octal escape, and the remaining characters stand for + // themselves as literals. + const char *s = ts; + unsigned int accum = 0; + unsigned int oct_digits = 0; + assert(*s == '\\'); // token starts at backslash + for (++s; s < te && oct_digits < 3; ++oct_digits, ++s) { + u8 digit = *s - '0'; + if (digit < 8) { + accum = digit + accum * 8; + } else { + break; + } + } + + if (oct_digits > 0) { + addEscapedOctal(currentSeq, accum, mode); + } + + // And then the rest of the digits, if any, are literal. + for (; s < te; ++s) { + addLiteral(currentSeq, *s, mode); + } + } + } + break; + } + case 330: { + p = ((te))-1; + { +#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" + + addLiteral(currentSeq, *(ts + 1), mode); + } + break; + } + }} + } + +#line 10179 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; } } + _nacts -= 1; + _acts += 1; } + +} - _keys = _regex_trans_keys + _regex_key_offsets[cs]; - _trans = _regex_index_offsets[cs]; - - _klen = _regex_single_lengths[cs]; - if ( _klen > 0 ) { - const short *_lower = _keys; - const short *_mid; - const short *_upper = _keys + _klen - 1; - while (1) { - if ( _upper < _lower ) - break; - - _mid = _lower + ((_upper-_lower) >> 1); - if ( _widec < *_mid ) - _upper = _mid - 1; - else if ( _widec > *_mid ) - _lower = _mid + 1; - else { - _trans += (unsigned int)(_mid - _keys); - goto _match; - } - } - _keys += _klen; - _trans += _klen; - } - - _klen = _regex_range_lengths[cs]; - if ( _klen > 0 ) { - const short *_lower = _keys; - const short *_mid; - const short *_upper = _keys + (_klen<<1) - 2; - while (1) { - if ( _upper < _lower ) - break; - - _mid = _lower + (((_upper-_lower) >> 1) & ~1); - if ( _widec < _mid[0] ) - _upper = _mid - 2; - else if ( _widec > _mid[1] ) - _lower = _mid + 2; - else { - _trans += (unsigned int)((_mid - _keys)>>1); - goto _match; - } - } - _trans += _klen; - } - -_match: - _trans = _regex_indicies[_trans]; -_eof_trans: - cs = _regex_trans_targs[_trans]; - - if ( _regex_trans_actions[_trans] == 0 ) - goto _again; - - _acts = _regex_actions + _regex_trans_actions[_trans]; - _nacts = (unsigned int) *_acts++; - while ( _nacts-- > 0 ) - { - switch ( *_acts++ ) - { - case 0: -#line 285 "Parser.rl" - { label.clear();} - break; - case 1: -#line 286 "Parser.rl" - { label.push_back((*p));} - break; - case 2: -#line 287 "Parser.rl" - { octAccumulator = 0;} - break; - case 3: -#line 288 "Parser.rl" - { accumulator = 0;} - break; - case 4: -#line 289 "Parser.rl" - { - octAccumulator = 0; - pushOct(&octAccumulator, (*p)); - } - break; - case 5: -#line 293 "Parser.rl" - { - accumulator = 0; - pushDec(&accumulator, (*p)); - } - break; - case 6: -#line 297 "Parser.rl" - { repeatN = 0; repeatM = 0; } - break; - case 7: -#line 298 "Parser.rl" - { pushDec(&repeatN, (*p)); } - break; - case 8: -#line 299 "Parser.rl" - { pushDec(&repeatM, (*p)); } - break; - case 9: -#line 300 "Parser.rl" - { pushOct(&octAccumulator, (*p)); } - break; - case 10: -#line 301 "Parser.rl" - { pushDec(&accumulator, (*p)); } - break; - case 11: -#line 302 "Parser.rl" - { - accumulator *= 16; - accumulator += (*p) - '0'; - } - break; - case 12: -#line 306 "Parser.rl" - { - accumulator *= 16; - accumulator += 10 + (*p) - 'a'; - } - break; - case 13: -#line 310 "Parser.rl" - { - accumulator *= 16; - accumulator += 10 + (*p) - 'A'; - } - break; - case 14: -#line 430 "Parser.rl" - { - newMode = mode; - } - break; - case 15: -#line 437 "Parser.rl" - { - switch ((*p)) { - case 'i': - newMode.caseless = true; - break; - case 'm': - newMode.multiline = true; - break; - case 's': - newMode.dotall = true; - break; - case 'x': - newMode.ignore_space = true; - break; - default: - assert(0); // this action only called for [imsx] - break; - } - } - break; - case 16: -#line 456 "Parser.rl" - { - switch ((*p)) { - case 'i': - newMode.caseless = false; - break; - case 'm': - newMode.multiline = false; - break; - case 's': - newMode.dotall = false; - break; - case 'x': - newMode.ignore_space = false; - break; - default: - assert(0); // this action only called for [imsx] - break; - } - } - break; - case 17: -#line 510 "Parser.rl" - {repeatM = repeatN;} - break; - case 18: -#line 510 "Parser.rl" - {repeatM = ComponentRepeat::NoLimit;} - break; - case 19: -#line 722 "Parser.rl" - { negated = !negated; } - break; - case 20: -#line 723 "Parser.rl" - { p--; { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 790;goto _again;}} } - break; - case 21: -#line 724 "Parser.rl" - { if (!inCharClass) { // not inside [..] - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - } - break; - case 22: -#line 730 "Parser.rl" - { throw LocatedParseError("Malformed property"); } - break; - case 25: -#line 1 "NONE" - {te = p+1;} - break; - case 26: -#line 550 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("(*UTF8) must be at start of " - "expression, encountered"); - }} - break; - case 27: -#line 554 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("(*UTF) must be at start of " - "expression, encountered"); - }} - break; - case 28: -#line 558 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("(*UCP) must be at start of " - "expression, encountered"); - }} - break; - case 29: -#line 564 "Parser.rl" - {te = p+1;{ - ParseMode temp_mode; - assert(ts - 2 >= ptr); // parser needs the '(*' at the start too. - read_control_verbs(ts - 2, te, (ts - 2 - ptr), temp_mode); - assert(0); // Should have thrown a parse error. - throw LocatedParseError("Unknown control verb"); - }} - break; - case 30: -#line 571 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Unknown control verb"); - }} - break; - case 31: -#line 571 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Unknown control verb"); - }} - break; - case 32: -#line 571 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Unknown control verb"); - }} - break; - case 33: -#line 581 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_CC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 34: -#line 582 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_CF, negated); {cs = stack[--top]; goto _again;} }} - break; - case 35: -#line 583 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_CN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 36: -#line 585 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_CS, negated); {cs = stack[--top]; goto _again;} }} - break; - case 37: -#line 587 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_LL, negated); {cs = stack[--top]; goto _again;} }} - break; - case 38: -#line 588 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_LM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 39: -#line 589 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_LO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 40: -#line 590 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_LT, negated); {cs = stack[--top]; goto _again;} }} - break; - case 41: -#line 591 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_LU, negated); {cs = stack[--top]; goto _again;} }} - break; - case 42: -#line 592 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_L_AND, negated); {cs = stack[--top]; goto _again;} }} - break; - case 43: -#line 594 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_MC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 44: -#line 596 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_MN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 45: -#line 598 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_ND, negated); {cs = stack[--top]; goto _again;} }} - break; - case 46: -#line 599 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_NL, negated); {cs = stack[--top]; goto _again;} }} - break; - case 47: -#line 600 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_NO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 48: -#line 602 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 49: -#line 603 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PD, negated); {cs = stack[--top]; goto _again;} }} - break; - case 50: -#line 604 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 51: -#line 605 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PF, negated); {cs = stack[--top]; goto _again;} }} - break; - case 52: -#line 606 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 53: -#line 607 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 54: -#line 608 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_PS, negated); {cs = stack[--top]; goto _again;} }} - break; - case 55: -#line 610 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_SC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 56: -#line 611 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_SK, negated); {cs = stack[--top]; goto _again;} }} - break; - case 57: -#line 612 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_SM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 58: -#line 613 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_SO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 59: -#line 615 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_ZL, negated); {cs = stack[--top]; goto _again;} }} - break; - case 60: -#line 616 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_ZP, negated); {cs = stack[--top]; goto _again;} }} - break; - case 61: -#line 617 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_ZS, negated); {cs = stack[--top]; goto _again;} }} - break; - case 62: -#line 618 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_XAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 63: -#line 619 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_XPS, negated); {cs = stack[--top]; goto _again;} }} - break; - case 64: -#line 620 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_XSP, negated); {cs = stack[--top]; goto _again;} }} - break; - case 65: -#line 621 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_XWD, negated); {cs = stack[--top]; goto _again;} }} - break; - case 66: -#line 622 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_ARABIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 67: -#line 623 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_ARMENIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 68: -#line 624 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_AVESTAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 69: -#line 625 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BALINESE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 70: -#line 626 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BAMUM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 71: -#line 627 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BATAK, negated); {cs = stack[--top]; goto _again;} }} - break; - case 72: -#line 628 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BENGALI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 73: -#line 629 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BOPOMOFO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 74: -#line 630 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BRAHMI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 75: -#line 631 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BRAILLE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 76: -#line 632 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BUGINESE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 77: -#line 633 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_BUHID, negated); {cs = stack[--top]; goto _again;} }} - break; - case 78: -#line 634 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CANADIAN_ABORIGINAL, negated); {cs = stack[--top]; goto _again;} }} - break; - case 79: -#line 635 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CARIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 80: -#line 636 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CHAM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 81: -#line 637 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CHEROKEE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 82: -#line 638 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_COMMON, negated); {cs = stack[--top]; goto _again;} }} - break; - case 83: -#line 639 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_COPTIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 84: -#line 640 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CUNEIFORM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 85: -#line 641 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CYPRIOT, negated); {cs = stack[--top]; goto _again;} }} - break; - case 86: -#line 642 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_CYRILLIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 87: -#line 643 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_DESERET, negated); {cs = stack[--top]; goto _again;} }} - break; - case 88: -#line 644 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_DEVANAGARI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 89: -#line 645 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_EGYPTIAN_HIEROGLYPHS, negated); {cs = stack[--top]; goto _again;} }} - break; - case 90: -#line 646 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_ETHIOPIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 91: -#line 647 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_GEORGIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 92: -#line 648 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_GLAGOLITIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 93: -#line 649 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_GOTHIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 94: -#line 650 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_GREEK, negated); {cs = stack[--top]; goto _again;} }} - break; - case 95: -#line 651 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_GUJARATI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 96: -#line 652 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_GURMUKHI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 97: -#line 654 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_HANGUL, negated); {cs = stack[--top]; goto _again;} }} - break; - case 98: -#line 655 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_HANUNOO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 99: -#line 656 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_HEBREW, negated); {cs = stack[--top]; goto _again;} }} - break; - case 100: -#line 657 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_HIRAGANA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 101: -#line 658 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_IMPERIAL_ARAMAIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 102: -#line 659 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_INHERITED, negated); {cs = stack[--top]; goto _again;} }} - break; - case 103: -#line 660 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PAHLAVI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 104: -#line 661 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PARTHIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 105: -#line 662 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_JAVANESE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 106: -#line 663 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_KAITHI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 107: -#line 664 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_KANNADA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 108: -#line 665 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_KATAKANA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 109: -#line 666 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_KAYAH_LI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 110: -#line 667 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_KHAROSHTHI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 111: -#line 668 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_KHMER, negated); {cs = stack[--top]; goto _again;} }} - break; - case 112: -#line 669 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LAO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 113: -#line 670 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LATIN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 114: -#line 671 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LEPCHA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 115: -#line 672 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LIMBU, negated); {cs = stack[--top]; goto _again;} }} - break; - case 116: -#line 673 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LINEAR_B, negated); {cs = stack[--top]; goto _again;} }} - break; - case 117: -#line 674 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LISU, negated); {cs = stack[--top]; goto _again;} }} - break; - case 118: -#line 675 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LYCIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 119: -#line 676 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_LYDIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 120: -#line 677 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_MALAYALAM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 121: -#line 678 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_MANDAIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 122: -#line 679 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_MEETEI_MAYEK, negated); {cs = stack[--top]; goto _again;} }} - break; - case 123: -#line 680 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_MONGOLIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 124: -#line 681 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_MYANMAR, negated); {cs = stack[--top]; goto _again;} }} - break; - case 125: -#line 682 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_NEW_TAI_LUE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 126: -#line 683 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_NKO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 127: -#line 684 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OGHAM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 128: -#line 685 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OL_CHIKI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 129: -#line 686 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OLD_ITALIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 130: -#line 687 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OLD_PERSIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 131: -#line 688 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OLD_SOUTH_ARABIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 132: -#line 689 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OLD_TURKIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 133: -#line 690 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_ORIYA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 134: -#line 691 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_OSMANYA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 135: -#line 692 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_PHAGS_PA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 136: -#line 693 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_PHOENICIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 137: -#line 694 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_REJANG, negated); {cs = stack[--top]; goto _again;} }} - break; - case 138: -#line 695 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_RUNIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 139: -#line 696 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SAMARITAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 140: -#line 697 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SAURASHTRA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 141: -#line 698 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SHAVIAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 142: -#line 699 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SINHALA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 143: -#line 700 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SUNDANESE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 144: -#line 701 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SYLOTI_NAGRI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 145: -#line 702 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_SYRIAC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 146: -#line 703 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TAGALOG, negated); {cs = stack[--top]; goto _again;} }} - break; - case 147: -#line 704 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TAGBANWA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 148: -#line 705 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TAI_LE, negated); {cs = stack[--top]; goto _again;} }} - break; - case 149: -#line 706 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TAI_THAM, negated); {cs = stack[--top]; goto _again;} }} - break; - case 150: -#line 707 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TAI_VIET, negated); {cs = stack[--top]; goto _again;} }} - break; - case 151: -#line 708 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TAMIL, negated); {cs = stack[--top]; goto _again;} }} - break; - case 152: -#line 709 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TELUGU, negated); {cs = stack[--top]; goto _again;} }} - break; - case 153: -#line 710 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_THAANA, negated); {cs = stack[--top]; goto _again;} }} - break; - case 154: -#line 711 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_THAI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 155: -#line 712 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TIBETAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 156: -#line 713 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_TIFINAGH, negated); {cs = stack[--top]; goto _again;} }} - break; - case 157: -#line 714 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_UGARITIC, negated); {cs = stack[--top]; goto _again;} }} - break; - case 158: -#line 715 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_VAI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 159: -#line 716 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_SCRIPT_YI, negated); {cs = stack[--top]; goto _again;} }} - break; - case 160: -#line 717 "Parser.rl" - {te = p+1;{ currentCls->add(CLASS_UCP_ANY, negated); {cs = stack[--top]; goto _again;} }} - break; - case 161: -#line 718 "Parser.rl" - {te = p+1;{ throw LocatedParseError("Unknown property"); }} - break; - case 162: -#line 580 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_C, negated); {cs = stack[--top]; goto _again;} }} - break; - case 163: -#line 584 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_CO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 164: -#line 586 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_L, negated); {cs = stack[--top]; goto _again;} }} - break; - case 165: -#line 593 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_M, negated); {cs = stack[--top]; goto _again;} }} - break; - case 166: -#line 595 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_ME, negated); {cs = stack[--top]; goto _again;} }} - break; - case 167: -#line 597 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_N, negated); {cs = stack[--top]; goto _again;} }} - break; - case 168: -#line 601 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_P, negated); {cs = stack[--top]; goto _again;} }} - break; - case 169: -#line 609 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_S, negated); {cs = stack[--top]; goto _again;} }} - break; - case 170: -#line 614 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_UCP_Z, negated); {cs = stack[--top]; goto _again;} }} - break; - case 171: -#line 653 "Parser.rl" - {te = p;p--;{ currentCls->add(CLASS_SCRIPT_HAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 172: -#line 718 "Parser.rl" - {te = p;p--;{ throw LocatedParseError("Unknown property"); }} - break; - case 173: -#line 580 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_C, negated); {cs = stack[--top]; goto _again;} }} - break; - case 174: -#line 584 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_CO, negated); {cs = stack[--top]; goto _again;} }} - break; - case 175: -#line 586 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_L, negated); {cs = stack[--top]; goto _again;} }} - break; - case 176: -#line 593 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_M, negated); {cs = stack[--top]; goto _again;} }} - break; - case 177: -#line 595 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_ME, negated); {cs = stack[--top]; goto _again;} }} - break; - case 178: -#line 597 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_N, negated); {cs = stack[--top]; goto _again;} }} - break; - case 179: -#line 601 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_P, negated); {cs = stack[--top]; goto _again;} }} - break; - case 180: -#line 609 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_UCP_S, negated); {cs = stack[--top]; goto _again;} }} - break; - case 181: -#line 653 "Parser.rl" - {{p = ((te))-1;}{ currentCls->add(CLASS_SCRIPT_HAN, negated); {cs = stack[--top]; goto _again;} }} - break; - case 182: -#line 718 "Parser.rl" - {{p = ((te))-1;}{ throw LocatedParseError("Unknown property"); }} - break; - case 183: -#line 733 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_C, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 184: -#line 741 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_L, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 185: -#line 749 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_M, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 186: -#line 757 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_N, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 187: -#line 765 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_P, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 188: -#line 773 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_S, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 189: -#line 781 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UCP_Z, negated); - if (!inCharClass) { - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - } - {cs = stack[--top]; goto _again;} - }} - break; - case 190: -#line 790 "Parser.rl" - {te = p+1;{ throw LocatedParseError("Unknown property"); }} - break; - case 191: -#line 796 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Unsupported POSIX collating " - "element"); - }} - break; - case 192: -#line 803 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_ALNUM, false); - }} - break; - case 193: -#line 806 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_ALNUM, true); - }} - break; - case 194: -#line 809 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_ALPHA, false); - }} - break; - case 195: -#line 812 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_ALPHA, true); - }} - break; - case 196: -#line 815 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_ASCII, false); - }} - break; - case 197: -#line 818 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_ASCII, true); - }} - break; - case 198: -#line 821 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_BLANK, false); - }} - break; - case 199: -#line 824 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_BLANK, true); - }} - break; - case 200: -#line 827 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_CNTRL, false); - }} - break; - case 201: -#line 830 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_CNTRL, true); - }} - break; - case 202: -#line 833 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_DIGIT, false); - }} - break; - case 203: -#line 836 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_DIGIT, true); - }} - break; - case 204: -#line 839 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_GRAPH, false); - }} - break; - case 205: -#line 842 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_GRAPH, true); - }} - break; - case 206: -#line 845 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_LOWER, false); - }} - break; - case 207: -#line 848 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_LOWER, true); - }} - break; - case 208: -#line 851 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_PRINT, false); - }} - break; - case 209: -#line 854 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_PRINT, true); - }} - break; - case 210: -#line 857 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_PUNCT, false); - }} - break; - case 211: -#line 860 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_PUNCT, true); - }} - break; - case 212: -#line 864 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_SPACE, false); - }} - break; - case 213: -#line 867 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_SPACE, true); - }} - break; - case 214: -#line 870 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UPPER, false); - }} - break; - case 215: -#line 873 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_UPPER, true); - }} - break; - case 216: -#line 876 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_WORD, false); - }} - break; - case 217: -#line 879 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_WORD, true); - }} - break; - case 218: -#line 882 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_XDIGIT, false); - }} - break; - case 219: -#line 885 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_XDIGIT, true); - }} - break; - case 220: -#line 890 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Invalid POSIX named class"); - }} - break; - case 221: -#line 893 "Parser.rl" - {te = p+1;{ - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 843;goto _again;}} - }} - break; - case 222: -#line 896 "Parser.rl" - {te = p+1;{ /*noop*/}} - break; - case 223: -#line 898 "Parser.rl" - {te = p+1;{ - currentCls->add('\x08'); - }} - break; - case 224: -#line 902 "Parser.rl" - {te = p+1;{ - currentCls->add('\x09'); - }} - break; - case 225: -#line 906 "Parser.rl" - {te = p+1;{ - currentCls->add('\x0a'); - }} - break; - case 226: -#line 910 "Parser.rl" - {te = p+1;{ - currentCls->add('\x0d'); - }} - break; - case 227: -#line 914 "Parser.rl" - {te = p+1;{ - currentCls->add('\x0c'); - }} - break; - case 228: -#line 918 "Parser.rl" - {te = p+1;{ - currentCls->add('\x07'); - }} - break; - case 229: -#line 922 "Parser.rl" - {te = p+1;{ - currentCls->add('\x1b'); - }} - break; - case 230: -#line 926 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_HORZ, false); - }} - break; - case 231: -#line 930 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_HORZ, true); - }} - break; - case 232: -#line 934 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_VERT, false); - }} - break; - case 233: -#line 938 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_VERT, true); - }} - break; - case 234: -#line 942 "Parser.rl" - {te = p+1;{ - negated = false; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 559;goto _again;}} - }} - break; - case 235: -#line 948 "Parser.rl" - {te = p+1;{ - negated = false; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 818;goto _again;}} - }} - break; - case 236: -#line 954 "Parser.rl" - {te = p+1;{ - negated = true; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 559;goto _again;}} - }} - break; - case 237: -#line 960 "Parser.rl" - {te = p+1;{ - negated = true; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 818;goto _again;}} - }} - break; - case 238: -#line 970 "Parser.rl" - {te = p+1;{ - currentCls->add(octAccumulator); - }} - break; - case 239: -#line 973 "Parser.rl" - {te = p+1;{ - currentCls->add(octAccumulator); - }} - break; - case 240: -#line 977 "Parser.rl" - {te = p+1;{ - string oct(ts + 3, te - ts - 4); - unsigned long val; - try { - val = stoul(oct, nullptr, 8); - } catch (const std::out_of_range &) { - val = MAX_UNICODE + 1; - } - if ((!mode.utf8 && val > 255) || val > MAX_UNICODE) { - throw LocatedParseError("Value in \\o{...} sequence is too large"); - } - currentCls->add((unichar)val); - }} - break; - case 241: -#line 997 "Parser.rl" - {te = p+1;{ - currentCls->add(accumulator); - }} - break; - case 242: -#line 1001 "Parser.rl" - {te = p+1;{ - // whatever we found here - currentCls->add(*(ts + 1)); - - }} - break; - case 243: -#line 1007 "Parser.rl" - {te = p+1;{ - string hex(ts + 3, te - ts - 4); - unsigned long val; - try { - val = stoul(hex, nullptr, 16); - } catch (const std::out_of_range &) { - val = MAX_UNICODE + 1; - } - if (val > MAX_UNICODE) { - throw LocatedParseError("Value in \\x{...} sequence is too large"); - } - currentCls->add((unichar)val); - }} - break; - case 244: -#line 1025 "Parser.rl" - {te = p+1;{ - if (te - ts < 3) { - assert(te - ts == 2); - throw LocatedParseError(SLASH_C_ERROR); - } else { - assert(te - ts == 3); - currentCls->add(decodeCtrl(ts[2])); - } - }} - break; - case 245: -#line 1035 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_WORD, false); - }} - break; - case 246: -#line 1039 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_WORD, true); - }} - break; - case 247: -#line 1043 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_SPACE, false); - }} - break; - case 248: -#line 1047 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_SPACE, true); - }} - break; - case 249: -#line 1051 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_DIGIT, false); - }} - break; - case 250: -#line 1055 "Parser.rl" - {te = p+1;{ - currentCls->add(CLASS_DIGIT, true); - }} - break; - case 251: -#line 1058 "Parser.rl" - {te = p+1;{ - currentCls->addDash(); - }} - break; - case 252: -#line 276 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "'\\" << *(ts + 1) << "' at index " << ts - ptr - << " not supported in a character class."; - throw ParseError(str.str()); - }} - break; - case 253: -#line 276 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "'\\" << *(ts + 1) << "' at index " << ts - ptr - << " not supported in a character class."; - throw ParseError(str.str()); - }} - break; - case 254: -#line 276 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "'\\" << *(ts + 1) << "' at index " << ts - ptr - << " not supported in a character class."; - throw ParseError(str.str()); - }} - break; - case 255: -#line 1075 "Parser.rl" - {te = p+1;{ - // add the literal char - currentCls->add(*(ts + 1)); - }} - break; - case 256: -#line 1081 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - currentCls->add(readUtf8CodePoint2c(ts)); - }} - break; - case 257: -#line 1086 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - currentCls->add(readUtf8CodePoint3c(ts)); - }} - break; - case 258: -#line 1091 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - currentCls->add(readUtf8CodePoint4c(ts)); - }} - break; - case 259: -#line 1096 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 260: -#line 1102 "Parser.rl" - {te = p+1;{ - currentCls->add((u8)*ts); - }} - break; - case 261: -#line 1106 "Parser.rl" - {te = p+1;{ - currentCls->finalize(); - currentSeq->addComponent(move(currentCls)); - inCharClass = false; - {cs = 746;goto _again;} - }} - break; - case 262: -#line 966 "Parser.rl" - {te = p;p--;{ throw LocatedParseError("Malformed property"); }} - break; - case 263: -#line 967 "Parser.rl" - {te = p;p--;{ throw LocatedParseError("Malformed property"); }} - break; - case 264: -#line 970 "Parser.rl" - {te = p;p--;{ - currentCls->add(octAccumulator); - }} - break; - case 265: -#line 973 "Parser.rl" - {te = p;p--;{ - currentCls->add(octAccumulator); - }} - break; - case 266: -#line 992 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); - }} - break; - case 267: -#line 997 "Parser.rl" - {te = p;p--;{ - currentCls->add(accumulator); - }} - break; - case 268: -#line 1021 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); - }} - break; - case 269: -#line 1025 "Parser.rl" - {te = p;p--;{ - if (te - ts < 3) { - assert(te - ts == 2); - throw LocatedParseError(SLASH_C_ERROR); - } else { - assert(te - ts == 3); - currentCls->add(decodeCtrl(ts[2])); - } - }} - break; - case 270: -#line 1096 "Parser.rl" - {te = p;p--;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 271: -#line 1102 "Parser.rl" - {te = p;p--;{ - currentCls->add((u8)*ts); - }} - break; - case 272: -#line 992 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); - }} - break; - case 273: -#line 1021 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); - }} - break; - case 274: -#line 1096 "Parser.rl" - {{p = ((te))-1;}{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 275: -#line 1102 "Parser.rl" - {{p = ((te))-1;}{ - currentCls->add((u8)*ts); - }} - break; - case 276: -#line 1120 "Parser.rl" - {te = p+1;{ - if (currentCls->isNegated()) { - // Already seen a caret; the second one is not a meta-character. - inCharClassEarly = false; - p--; {cs = 819;goto _again;} - } else { - currentCls->negate(); - // Note: we cannot switch off inCharClassEarly here, as /[^]]/ - // needs to use the right square bracket path below. - } - }} - break; - case 277: -#line 1133 "Parser.rl" - {te = p+1;{ - currentCls->add(']'); - inCharClassEarly = false; - }} - break; - case 278: -#line 1138 "Parser.rl" - {te = p+1;{ { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 843;goto _again;}} }} - break; - case 279: -#line 1139 "Parser.rl" - {te = p+1;{ /*noop*/}} - break; - case 280: -#line 1142 "Parser.rl" - {te = p+1;{ - inCharClassEarly = false; - p--; - {cs = 819;goto _again;} - }} - break; - case 281: -#line 1142 "Parser.rl" - {te = p;p--;{ - inCharClassEarly = false; - p--; - {cs = 819;goto _again;} - }} - break; - case 282: -#line 1154 "Parser.rl" - {te = p+1;{ - {cs = 746;goto _again;} - }} - break; - case 283: -#line 1159 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - cc->add(readUtf8CodePoint2c(ts)); - cc->finalize(); - currentSeq->addComponent(move(cc)); - }} - break; - case 284: -#line 1168 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - cc->add(readUtf8CodePoint3c(ts)); - cc->finalize(); - currentSeq->addComponent(move(cc)); - }} - break; - case 285: -#line 1177 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - cc->add(readUtf8CodePoint4c(ts)); - cc->finalize(); - currentSeq->addComponent(move(cc)); - }} - break; - case 286: -#line 1186 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 287: -#line 1192 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, *ts, mode); - }} - break; - case 288: -#line 1186 "Parser.rl" - {te = p;p--;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 289: -#line 1192 "Parser.rl" - {te = p;p--;{ - addLiteral(currentSeq, *ts, mode); - }} - break; - case 290: -#line 1186 "Parser.rl" - {{p = ((te))-1;}{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 291: -#line 1202 "Parser.rl" - {te = p+1;{ - {cs = stack[--top]; goto _again;} - }} - break; - case 292: -#line 1207 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - currentCls->add(readUtf8CodePoint2c(ts)); - inCharClassEarly = false; - }} - break; - case 293: -#line 1213 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - currentCls->add(readUtf8CodePoint3c(ts)); - inCharClassEarly = false; - }} - break; - case 294: -#line 1219 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - currentCls->add(readUtf8CodePoint4c(ts)); - inCharClassEarly = false; - }} - break; - case 295: -#line 1225 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 296: -#line 1231 "Parser.rl" - {te = p+1;{ - currentCls->add(*ts); - inCharClassEarly = false; - }} - break; - case 297: -#line 1225 "Parser.rl" - {te = p;p--;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 298: -#line 1231 "Parser.rl" - {te = p;p--;{ - currentCls->add(*ts); - inCharClassEarly = false; - }} - break; - case 299: -#line 1225 "Parser.rl" - {{p = ((te))-1;}{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 300: -#line 1243 "Parser.rl" - {te = p+1;{ inComment = false; {cs = 746;goto _again;} }} - break; - case 301: -#line 1247 "Parser.rl" - {te = p+1;} - break; - case 302: -#line 1255 "Parser.rl" - {te = p+1;{ inComment = false; {cs = 746;goto _again;} }} - break; - case 303: -#line 1259 "Parser.rl" - {te = p+1;} - break; - case 304: -#line 1491 "Parser.rl" - {act = 288;} - break; - case 305: -#line 1508 "Parser.rl" - {act = 290;} - break; - case 306: -#line 1737 "Parser.rl" - {act = 330;} - break; - case 307: -#line 362 "Parser.rl" - {te = p+1;{ - if (sequences.empty()) { - throw LocatedParseError("Unmatched parentheses"); - } - currentSeq->finalize(); - POP_SEQUENCE; - }} - break; - case 308: -#line 1274 "Parser.rl" - {te = p+1;{ - currentSeq->addAlternation(); - }} - break; - case 309: -#line 1279 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("POSIX named classes are only " - "supported inside a class"); - }} - break; - case 310: -#line 1286 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Unsupported POSIX collating " - "element"); - }} - break; - case 311: -#line 1293 "Parser.rl" - {te = p+1;{ - {cs = 838;goto _again;} - }} - break; - case 312: -#line 1297 "Parser.rl" - {te = p+1;{ /* noop */ }} - break; - case 313: -#line 1299 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(generateComponent(CLASS_ANY, false, mode)); - }} - break; - case 314: -#line 1303 "Parser.rl" - {te = p+1;{ - if (mode.utf8) { - throw LocatedParseError("\\C is unsupported in UTF8"); - } - currentSeq->addComponent(std::make_unique()); - }} - break; - case 315: -#line 1317 "Parser.rl" - {te = p+1;{ - if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, - ComponentRepeat::REPEAT_NONGREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 316: -#line 1324 "Parser.rl" - {te = p+1;{ - if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, - ComponentRepeat::REPEAT_POSSESSIVE)) { - throwInvalidRepeat(); - } - }} - break; - case 317: -#line 1338 "Parser.rl" - {te = p+1;{ - if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, - ComponentRepeat::REPEAT_NONGREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 318: -#line 1345 "Parser.rl" - {te = p+1;{ - if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, - ComponentRepeat::REPEAT_POSSESSIVE)) { - throwInvalidRepeat(); - } - }} - break; - case 319: -#line 1359 "Parser.rl" - {te = p+1;{ - if (!currentSeq->addRepeat( - 0, 1, ComponentRepeat::REPEAT_NONGREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 320: -#line 1366 "Parser.rl" - {te = p+1;{ - if (!currentSeq->addRepeat( - 0, 1, ComponentRepeat::REPEAT_POSSESSIVE)) { - throwInvalidRepeat(); - } - }} - break; - case 321: -#line 1383 "Parser.rl" - {te = p+1;{ - if (repeatN > repeatM || repeatM == 0) { - throwInvalidRepeat(); - } else if (!currentSeq->addRepeat( - repeatN, repeatM, - ComponentRepeat::REPEAT_NONGREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 322: -#line 1393 "Parser.rl" - {te = p+1;{ - if (repeatN > repeatM || repeatM == 0) { - throwInvalidRepeat(); - } else if (!currentSeq->addRepeat( - repeatN, repeatM, - ComponentRepeat::REPEAT_POSSESSIVE)) { - throwInvalidRepeat(); - } - }} - break; - case 323: -#line 322 "Parser.rl" - {te = p+1;{ - inComment = true; - {cs = 849;goto _again;} - }} - break; - case 324: -#line 1410 "Parser.rl" - {te = p+1;{ p--; { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 787;goto _again;}} }} - break; - case 325: -#line 1414 "Parser.rl" - {te = p+1;{ assert(0); {p++; goto _out; } }} - break; - case 326: -#line 1421 "Parser.rl" - {te = p+1;{ - auto bound = mode.multiline ? ComponentBoundary::BEGIN_LINE - : ComponentBoundary::BEGIN_STRING; - currentSeq->addComponent(std::make_unique(bound)); - }} - break; - case 327: -#line 1428 "Parser.rl" - {te = p+1;{ - auto bound = mode.multiline ? ComponentBoundary::END_LINE - : ComponentBoundary::END_STRING_OPTIONAL_LF; - currentSeq->addComponent(std::make_unique(bound)); - }} - break; - case 328: -#line 1434 "Parser.rl" - {te = p+1;{ - auto bound = ComponentBoundary::BEGIN_STRING; - currentSeq->addComponent(std::make_unique(bound)); - }} - break; - case 329: -#line 1439 "Parser.rl" - {te = p+1;{ - auto bound = ComponentBoundary::END_STRING_OPTIONAL_LF; - currentSeq->addComponent(std::make_unique(bound)); - }} - break; - case 330: -#line 1444 "Parser.rl" - {te = p+1;{ - auto bound = ComponentBoundary::END_STRING; - currentSeq->addComponent(std::make_unique(bound)); - }} - break; - case 331: -#line 1449 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent( - std::make_unique(ts - ptr, false, mode)); - }} - break; - case 332: -#line 1454 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent( - std::make_unique(ts - ptr, true, mode)); - }} - break; - case 333: -#line 1464 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, '\x09', mode); - }} - break; - case 334: -#line 1468 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, '\x0a', mode); - }} - break; - case 335: -#line 1472 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, '\x0d', mode); - }} - break; - case 336: -#line 1476 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, '\x0c', mode); - }} - break; - case 337: -#line 1480 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, '\x07', mode); - }} - break; - case 338: -#line 1484 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, '\x1b', mode); - }} - break; - case 339: -#line 1488 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, octAccumulator, mode); - }} - break; - case 340: -#line 479 "Parser.rl" - {te = p+1;{ - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - currentSeq->addComponent(std::make_unique(accumulator)); - }} - break; - case 341: -#line 486 "Parser.rl" - {te = p+1;{ - // Accumulator is a negative offset. - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - if (accumulator >= groupIndex) { - throw LocatedParseError("Invalid reference"); - } - unsigned idx = groupIndex - accumulator; - currentSeq->addComponent(std::make_unique(idx)); - }} - break; - case 342: -#line 479 "Parser.rl" - {te = p+1;{ - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - currentSeq->addComponent(std::make_unique(accumulator)); - }} - break; - case 343: -#line 486 "Parser.rl" - {te = p+1;{ - // Accumulator is a negative offset. - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - if (accumulator >= groupIndex) { - throw LocatedParseError("Invalid reference"); - } - unsigned idx = groupIndex - accumulator; - currentSeq->addComponent(std::make_unique(idx)); - }} - break; - case 344: -#line 498 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(std::make_unique(label)); - }} - break; - case 345: -#line 498 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(std::make_unique(label)); - }} - break; - case 346: -#line 498 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(std::make_unique(label)); - }} - break; - case 347: -#line 498 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(std::make_unique(label)); - }} - break; - case 348: -#line 498 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(std::make_unique(label)); - }} - break; - case 349: -#line 1549 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "Onigiruma subroutine call at index " << ts - ptr << - " not supported."; - throw ParseError(str.str()); - }} - break; - case 350: -#line 1560 "Parser.rl" - {te = p+1;{ - string oct(ts + 3, te - ts - 4); - unsigned long val; - try { - val = stoul(oct, nullptr, 8); - } catch (const std::out_of_range &) { - val = MAX_UNICODE + 1; - } - if ((!mode.utf8 && val > 255) || val > MAX_UNICODE) { - throw LocatedParseError("Value in \\o{...} sequence is too large"); - } - addEscapedOctal(currentSeq, (unichar)val, mode); - }} - break; - case 351: -#line 1578 "Parser.rl" - {te = p+1;{ - addEscapedHex(currentSeq, accumulator, mode); - }} - break; - case 352: -#line 1582 "Parser.rl" - {te = p+1;{ - string hex(ts + 3, te - ts - 4); - unsigned long val; - try { - val = stoul(hex, nullptr, 16); - } catch (const std::out_of_range &) { - val = MAX_UNICODE + 1; - } - if (val > MAX_UNICODE) { - throw LocatedParseError("Value in \\x{...} sequence is too large"); - } - addEscapedHex(currentSeq, (unichar)val, mode); - }} - break; - case 353: -#line 1600 "Parser.rl" - {te = p+1;{ - if (te - ts < 3) { - assert(te - ts == 2); - throw LocatedParseError(SLASH_C_ERROR); - } else { - assert(te - ts == 3); - addLiteral(currentSeq, decodeCtrl(ts[2]), mode); - } - }} - break; - case 354: -#line 1610 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "'\\" << *(ts + 1) << "' at index " << ts - ptr - << " not supported."; - throw ParseError(str.str()); - }} - break; - case 355: -#line 1618 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_WORD, false, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 356: -#line 1623 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_WORD, true, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 357: -#line 1628 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_SPACE, false, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 358: -#line 1633 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_SPACE, true, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 359: -#line 1638 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_DIGIT, false, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 360: -#line 1643 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_DIGIT, true, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 361: -#line 1648 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_HORZ, false, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 362: -#line 1653 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_HORZ, true, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 363: -#line 1658 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_VERT, false, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 364: -#line 1663 "Parser.rl" - {te = p+1;{ - auto cc = generateComponent(CLASS_VERT, true, mode); - currentSeq->addComponent(move(cc)); - }} - break; - case 365: -#line 1668 "Parser.rl" - {te = p+1;{ - assert(!currentCls && !inCharClass); - currentCls = getComponentClass(mode); - negated = false; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 559;goto _again;}} - }} - break; - case 366: -#line 1676 "Parser.rl" - {te = p+1;{ - assert(!currentCls && !inCharClass); - currentCls = getComponentClass(mode); - negated = false; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 818;goto _again;}} - }} - break; - case 367: -#line 1684 "Parser.rl" - {te = p+1;{ - assert(!currentCls && !inCharClass); - currentCls = getComponentClass(mode); - negated = true; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 559;goto _again;}} - }} - break; - case 368: -#line 1692 "Parser.rl" - {te = p+1;{ - assert(!currentCls && !inCharClass); - currentCls = getComponentClass(mode); - negated = true; - p--; - { - DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); - if ((int)stack.size() == top) { - stack.resize(2 * (top + 1)); - } - {stack[top++] = cs; cs = 818;goto _again;}} - }} - break; - case 369: -#line 1704 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "\\R at index " << ts - ptr << " not supported."; - throw ParseError(str.str()); - }} - break; - case 370: -#line 1711 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "\\K at index " << ts - ptr << " not supported."; - throw ParseError(str.str()); - }} - break; - case 371: -#line 1726 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "\\G at index " << ts - ptr << " not supported."; - throw ParseError(str.str()); - }} - break; - case 372: -#line 1732 "Parser.rl" - {te = p+1;{ - currentSeq->addComponent(std::make_unique(ts - ptr, mode)); - }} - break; - case 373: -#line 1737 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, *(ts + 1), mode); - }} - break; - case 374: -#line 316 "Parser.rl" - {te = p+1;{ - inComment = true; - {cs = 848;goto _again;} - }} - break; - case 375: -#line 433 "Parser.rl" - {te = p+1;{ - mode = newMode; - currentSeq->addComponent(std::make_unique()); - }} - break; - case 376: -#line 355 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - mode = newMode; - currentSeq = - enterSequence(currentSeq, std::make_unique()); - }} - break; - case 377: -#line 369 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(ComponentAssertion::LOOKAHEAD, - ComponentAssertion::POS)); - }} - break; - case 378: -#line 375 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(ComponentAssertion::LOOKAHEAD, - ComponentAssertion::NEG)); - }} - break; - case 379: -#line 381 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(ComponentAssertion::LOOKBEHIND, - ComponentAssertion::POS)); - }} - break; - case 380: -#line 387 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(ComponentAssertion::LOOKBEHIND, - ComponentAssertion::NEG)); - }} - break; - case 381: -#line 393 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Embedded code is not supported"); - }} - break; - case 382: -#line 393 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Embedded code is not supported"); - }} - break; - case 383: -#line 416 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique()); - }} - break; - case 384: -#line 336 "Parser.rl" - {te = p+1;{ - assert(!label.empty()); // should be guaranteed by machine - char c = *label.begin(); - if (c >= '0' && c <= '9') { - throw LocatedParseError("Group name cannot begin with a digit"); - } - if (!groupNames.insert(label).second) { - throw LocatedParseError("Two named subpatterns use the name '" + label + "'"); - } - PUSH_SEQUENCE; - auto seq = std::make_unique(); - seq->setCaptureIndex(groupIndex++); - seq->setCaptureName(label); - currentSeq = enterSequence(currentSeq, move(seq)); - }} - break; - case 385: -#line 399 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Subpattern reference unsupported"); - }} - break; - case 386: -#line 399 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Subpattern reference unsupported"); - }} - break; - case 387: -#line 1783 "Parser.rl" - {te = p+1;{ - auto a = std::make_unique( - ComponentAssertion::LOOKAHEAD, ComponentAssertion::POS); - ComponentAssertion *a_seq = a.get(); - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(move(a))); - PUSH_SEQUENCE; - currentSeq = a_seq; - }} - break; - case 388: -#line 1794 "Parser.rl" - {te = p+1;{ - auto a = std::make_unique( - ComponentAssertion::LOOKAHEAD, ComponentAssertion::NEG); - ComponentAssertion *a_seq = a.get(); - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(move(a))); - PUSH_SEQUENCE; - currentSeq = a_seq; - }} - break; - case 389: -#line 1805 "Parser.rl" - {te = p+1;{ - auto a = std::make_unique( - ComponentAssertion::LOOKBEHIND, ComponentAssertion::POS); - ComponentAssertion *a_seq = a.get(); - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(move(a))); - PUSH_SEQUENCE; - currentSeq = a_seq; - }} - break; - case 390: -#line 1816 "Parser.rl" - {te = p+1;{ - auto a = std::make_unique( - ComponentAssertion::LOOKBEHIND, ComponentAssertion::NEG); - ComponentAssertion *a_seq = a.get(); - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(move(a))); - PUSH_SEQUENCE; - currentSeq = a_seq; - }} - break; - case 391: -#line 1828 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Pattern recursion not supported"); - }} - break; - case 392: -#line 402 "Parser.rl" - {te = p+1;{ - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - PUSH_SEQUENCE; - currentSeq = enterSequence(currentSeq, - std::make_unique(accumulator)); - }} - break; - case 393: -#line 410 "Parser.rl" - {te = p+1;{ - PUSH_SEQUENCE; - assert(!label.empty()); - currentSeq = enterSequence(currentSeq, - std::make_unique(label)); - }} - break; - case 394: -#line 1844 "Parser.rl" - {te = p+1;{ - ostringstream str; - str << "Callout at index " << ts - ptr << " not supported."; - throw ParseError(str.str()); - }} - break; - case 395: -#line 1852 "Parser.rl" - {te = p+1;{ - throw LocatedParseError("Unrecognised character after (?"); - }} - break; - case 396: -#line 1857 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - cc->add(readUtf8CodePoint2c(ts)); - cc->finalize(); - currentSeq->addComponent(move(cc)); - }} - break; - case 397: -#line 1866 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - cc->add(readUtf8CodePoint3c(ts)); - cc->finalize(); - currentSeq->addComponent(move(cc)); - }} - break; - case 398: -#line 1875 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - /* leverage ComponentClass to generate the vertices */ - auto cc = getComponentClass(mode); - cc->add(readUtf8CodePoint4c(ts)); - cc->finalize(); - currentSeq->addComponent(move(cc)); - }} - break; - case 399: -#line 1884 "Parser.rl" - {te = p+1;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 400: -#line 1893 "Parser.rl" - {te = p+1;{ - if (mode.ignore_space == false) { - addLiteral(currentSeq, *ts, mode); - } - }} - break; - case 401: -#line 1898 "Parser.rl" - {te = p+1;{ - addLiteral(currentSeq, *ts, mode); - }} - break; - case 402: -#line 328 "Parser.rl" - {te = p;p--;{ - PUSH_SEQUENCE; - auto seq = std::make_unique(); - seq->setCaptureIndex(groupIndex++); - currentSeq = enterSequence(currentSeq, move(seq)); - }} - break; - case 403: -#line 421 "Parser.rl" - {te = p;p--;{ - assert(!currentCls); - assert(!inCharClass); // not reentrant - currentCls = getComponentClass(mode); - inCharClass = true; - inCharClassEarly = true; - currentClsBegin = ts; - {cs = 836;goto _again;} - }} - break; - case 404: -#line 1310 "Parser.rl" - {te = p;p--;{ - if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, - ComponentRepeat::REPEAT_GREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 405: -#line 1331 "Parser.rl" - {te = p;p--;{ - if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, - ComponentRepeat::REPEAT_GREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 406: -#line 1352 "Parser.rl" - {te = p;p--;{ - if (!currentSeq->addRepeat( - 0, 1, ComponentRepeat::REPEAT_GREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 407: -#line 1373 "Parser.rl" - {te = p;p--;{ - if (repeatN > repeatM || repeatM == 0) { - throwInvalidRepeat(); - } else if (!currentSeq->addRepeat( - repeatN, repeatM, - ComponentRepeat::REPEAT_GREEDY)) { - throwInvalidRepeat(); - } - }} - break; - case 408: -#line 1488 "Parser.rl" - {te = p;p--;{ - addLiteral(currentSeq, octAccumulator, mode); - }} - break; - case 409: -#line 1491 "Parser.rl" - {te = p;p--;{ - // If there are enough capturing sub expressions, this may be - // a back reference - accumulator = parseAsDecimal(octAccumulator); - if (accumulator < groupIndex) { - currentSeq->addComponent(std::make_unique(accumulator)); - } else { - addEscapedOctal(currentSeq, octAccumulator, mode); - } - }} - break; - case 410: -#line 479 "Parser.rl" - {te = p;p--;{ - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - currentSeq->addComponent(std::make_unique(accumulator)); - }} - break; - case 411: -#line 479 "Parser.rl" - {te = p;p--;{ - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - currentSeq->addComponent(std::make_unique(accumulator)); - }} - break; - case 412: -#line 486 "Parser.rl" - {te = p;p--;{ - // Accumulator is a negative offset. - if (accumulator == 0) { - throw LocatedParseError("Numbered reference cannot be zero"); - } - if (accumulator >= groupIndex) { - throw LocatedParseError("Invalid reference"); - } - unsigned idx = groupIndex - accumulator; - currentSeq->addComponent(std::make_unique(idx)); - }} - break; - case 413: -#line 1557 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Invalid reference after \\g"); - }} - break; - case 414: -#line 1574 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); - }} - break; - case 415: -#line 1578 "Parser.rl" - {te = p;p--;{ - addEscapedHex(currentSeq, accumulator, mode); - }} - break; - case 416: -#line 1596 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); - }} - break; - case 417: -#line 1600 "Parser.rl" - {te = p;p--;{ - if (te - ts < 3) { - assert(te - ts == 2); - throw LocatedParseError(SLASH_C_ERROR); - } else { - assert(te - ts == 3); - addLiteral(currentSeq, decodeCtrl(ts[2]), mode); - } - }} - break; - case 418: -#line 1700 "Parser.rl" - {te = p;p--;{ throw LocatedParseError("Malformed property"); }} - break; - case 419: -#line 1701 "Parser.rl" - {te = p;p--;{ throw LocatedParseError("Malformed property"); }} - break; - case 420: -#line 1719 "Parser.rl" - {te = p;p--;{ - ostringstream str; - str << "\\k at index " << ts - ptr << " not supported."; - throw ParseError(str.str()); - }} - break; - case 421: -#line 1742 "Parser.rl" - {te = p;p--;{ - assert(ts + 1 == pe); - ostringstream str; - str << "Unescaped \\ at end of input, index " << ts - ptr << "."; - throw ParseError(str.str()); - }} - break; - case 422: -#line 396 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Conditional subpattern unsupported"); - }} - break; - case 423: -#line 1852 "Parser.rl" - {te = p;p--;{ - throw LocatedParseError("Unrecognised character after (?"); - }} - break; - case 424: -#line 1884 "Parser.rl" - {te = p;p--;{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 425: -#line 1898 "Parser.rl" - {te = p;p--;{ - addLiteral(currentSeq, *ts, mode); - }} - break; - case 426: -#line 328 "Parser.rl" - {{p = ((te))-1;}{ - PUSH_SEQUENCE; - auto seq = std::make_unique(); - seq->setCaptureIndex(groupIndex++); - currentSeq = enterSequence(currentSeq, move(seq)); - }} - break; - case 427: -#line 421 "Parser.rl" - {{p = ((te))-1;}{ - assert(!currentCls); - assert(!inCharClass); // not reentrant - currentCls = getComponentClass(mode); - inCharClass = true; - inCharClassEarly = true; - currentClsBegin = ts; - {cs = 836;goto _again;} - }} - break; - case 428: -#line 1557 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Invalid reference after \\g"); - }} - break; - case 429: -#line 1574 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); - }} - break; - case 430: -#line 1596 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); - }} - break; - case 431: -#line 1719 "Parser.rl" - {{p = ((te))-1;}{ - ostringstream str; - str << "\\k at index " << ts - ptr << " not supported."; - throw ParseError(str.str()); - }} - break; - case 432: -#line 396 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Conditional subpattern unsupported"); - }} - break; - case 433: -#line 1852 "Parser.rl" - {{p = ((te))-1;}{ - throw LocatedParseError("Unrecognised character after (?"); - }} - break; - case 434: -#line 1884 "Parser.rl" - {{p = ((te))-1;}{ - assert(mode.utf8); - throwInvalidUtf8(); - }} - break; - case 435: -#line 1898 "Parser.rl" - {{p = ((te))-1;}{ - addLiteral(currentSeq, *ts, mode); - }} - break; - case 436: -#line 1 "NONE" - { switch( act ) { - case 288: - {{p = ((te))-1;} - // If there are enough capturing sub expressions, this may be - // a back reference - accumulator = parseAsDecimal(octAccumulator); - if (accumulator < groupIndex) { - currentSeq->addComponent(std::make_unique(accumulator)); - } else { - addEscapedOctal(currentSeq, octAccumulator, mode); - } - } - break; - case 290: - {{p = ((te))-1;} - // if there are enough left parens to this point, back ref - if (accumulator < groupIndex) { - currentSeq->addComponent(std::make_unique(accumulator)); - } else { - // Otherwise, we interpret the first three digits as an - // octal escape, and the remaining characters stand for - // themselves as literals. - const char *s = ts; - unsigned int accum = 0; - unsigned int oct_digits = 0; - assert(*s == '\\'); // token starts at backslash - for (++s; s < te && oct_digits < 3; ++oct_digits, ++s) { - u8 digit = *s - '0'; - if (digit < 8) { - accum = digit + accum * 8; - } else { - break; - } - } - - if (oct_digits > 0) { - addEscapedOctal(currentSeq, accum, mode); - } - - // And then the rest of the digits, if any, are literal. - for (; s < te; ++s) { - addLiteral(currentSeq, *s, mode); - } - } - } - break; - case 330: - {{p = ((te))-1;} - addLiteral(currentSeq, *(ts + 1), mode); - } - break; - } - } - break; -#line 5508 "Parser.cpp" - } - } - -_again: - _acts = _regex_actions + _regex_to_state_actions[cs]; - _nacts = (unsigned int) *_acts++; - while ( _nacts-- > 0 ) { - switch ( *_acts++ ) { - case 23: -#line 1 "NONE" - {ts = 0;} - break; -#line 5521 "Parser.cpp" - } - } - - if ( cs == 0 ) +_again: {} +if ( p == eof ) { + if ( cs >= 746 ) goto _out; - if ( ++p != pe ) - goto _resume; - _test_eof: {} - if ( p == eof ) - { - if ( _regex_eof_trans[cs] > 0 ) { - _trans = _regex_eof_trans[cs] - 1; - goto _eof_trans; - } - const short *__acts = _regex_actions + _regex_eof_actions[cs]; - unsigned int __nacts = (unsigned int) *__acts++; - while ( __nacts-- > 0 ) { - switch ( *__acts++ ) { - case 22: -#line 730 "Parser.rl" - { throw LocatedParseError("Malformed property"); } - break; -#line 5544 "Parser.cpp" +} +else { + _acts = ( _regex_actions + (_regex_to_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 23: { + { +#line 1 "NONE" + {ts = 0;}} + +#line 10205 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" + + break; + } } + _nacts -= 1; + _acts += 1; } + + if ( cs != 0 ) { + p += 1; + goto _resume; } +} +_out: {} +} - _out: {} - } +#line 1983 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" -#line 1984 "Parser.rl" - if (p != pe && *p != '\0') { - // didn't make it to the end of our input, but we didn't throw a ParseError? - assert(0); - ostringstream str; - str << "Parse error at index " << (p - ptr) << "."; - throw ParseError(str.str()); - } +if (p != pe && *p != '\0') { +// didn't make it to the end of our input, but we didn't throw a ParseError? +assert(0); +ostringstream str; +str << "Parse error at index " << (p - ptr) << "."; +throw ParseError(str.str()); +} - if (currentCls) { - assert(inCharClass); - assert(currentClsBegin); - ostringstream oss; - oss << "Unterminated character class starting at index " - << currentClsBegin - ptr << "."; - throw ParseError(oss.str()); - } +if (currentCls) { +assert(inCharClass); +assert(currentClsBegin); +ostringstream oss; +oss << "Unterminated character class starting at index " +<< currentClsBegin - ptr << "."; +throw ParseError(oss.str()); +} - if (inComment) { - throw ParseError("Unterminated comment."); - } +if (inComment) { +throw ParseError("Unterminated comment."); +} - if (!sequences.empty()) { - ostringstream str; - str << "Missing close parenthesis for group started at index " - << sequences.back().seqOffset << "."; - throw ParseError(str.str()); - } +if (!sequences.empty()) { +ostringstream str; +str << "Missing close parenthesis for group started at index " +<< sequences.back().seqOffset << "."; +throw ParseError(str.str()); +} - // Unlikely, but possible - if (groupIndex > 65535) { - throw ParseError("The maximum number of capturing subexpressions is 65535."); - } +// Unlikely, but possible +if (groupIndex > 65535) { +throw ParseError("The maximum number of capturing subexpressions is 65535."); +} - // Finalize the top-level sequence, which will take care of any - // top-level alternation. - currentSeq->finalize(); - assert(currentSeq == rootSeq.get()); +// Finalize the top-level sequence, which will take care of any +// top-level alternation. +currentSeq->finalize(); +assert(currentSeq == rootSeq.get()); - // Ensure that all references are valid. - checkReferences(*rootSeq, groupIndex, groupNames); +// Ensure that all references are valid. +checkReferences(*rootSeq, groupIndex, groupNames); - return move(rootSeq); - } catch (LocatedParseError &error) { - if (ts >= ptr && ts <= pe) { - error.locate(ts - ptr); - } else { - error.locate(0); - } - throw; - } +return move(rootSeq); +} catch (LocatedParseError &error) { +if (ts >= ptr && ts <= pe) { +error.locate(ts - ptr); +} else { +error.locate(0); +} +throw; +} } } // namespace ue2 diff --git a/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp b/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp index 19b5c6955e1..edcfaec0c97 100644 --- a/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp +++ b/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp @@ -1,37 +1,36 @@ - -#line 1 "control_verbs.rl" +#line 1 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" /* - * Copyright (c) 2017, Intel Corporation - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Intel Corporation nor the names of its contributors - * may be used to endorse or promote products derived from this software - * without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ +* Copyright (c) 2017, Intel Corporation +* +* Redistribution and use in source and binary forms, with or without +* modification, are permitted provided that the following conditions are met: +* +* * Redistributions of source code must retain the above copyright notice, +* this list of conditions and the following disclaimer. +* * Redistributions in binary form must reproduce the above copyright +* notice, this list of conditions and the following disclaimer in the +* documentation and/or other materials provided with the distribution. +* * Neither the name of Intel Corporation nor the names of its contributors +* may be used to endorse or promote products derived from this software +* without specific prior written permission. +* +* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +* POSSIBILITY OF SUCH DAMAGE. +*/ /** - * \file - * \brief Parser for control verbs that can occur at the beginning of a pattern. - */ +* \file +* \brief Parser for control verbs that can occur at the beginning of a pattern. +*/ #include "parser/control_verbs.h" @@ -44,400 +43,505 @@ using namespace std; namespace ue2 { + + const char *read_control_verbs(const char *ptr, const char *end, size_t start, + ParseMode &mode) { + const char *p = ptr; + const char *pe = end; + const char *eof = pe; + const char *ts, *te; + int cs; + UNUSED int act; + -const char *read_control_verbs(const char *ptr, const char *end, size_t start, - ParseMode &mode) { - const char *p = ptr; - const char *pe = end; - const char *eof = pe; - const char *ts, *te; - int cs; - UNUSED int act; +#line 56 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +static const signed char _ControlVerbs_actions[] = { + 0, 1, 0, 1, 1, 1, 2, 1, + 3, 1, 4, 1, 5, 1, 6, 1, + 7, 1, 8, 1, 9, 0 + }; + + static const short _ControlVerbs_key_offsets[] = { + 0, 7, 8, 10, 12, 14, 16, 18, + 20, 21, 23, 25, 27, 30, 32, 34, + 36, 38, 40, 42, 44, 46, 48, 50, + 52, 55, 57, 59, 61, 63, 66, 68, + 70, 72, 74, 76, 79, 82, 84, 86, + 88, 90, 92, 94, 96, 98, 100, 102, + 105, 107, 109, 111, 113, 115, 117, 119, + 121, 123, 125, 127, 129, 131, 133, 135, + 137, 139, 141, 143, 146, 148, 149, 151, + 155, 157, 159, 160, 161, 0 + }; + + static const char _ControlVerbs_trans_keys[] = { + 41u, 65u, 66u, 67u, 76u, 78u, 85u, 41u, + 41u, 78u, 41u, 89u, 41u, 67u, 41u, 82u, + 41u, 76u, 41u, 70u, 41u, 41u, 83u, 41u, + 82u, 41u, 95u, 41u, 65u, 85u, 41u, 78u, + 41u, 89u, 41u, 67u, 41u, 78u, 41u, 73u, + 41u, 67u, 41u, 79u, 41u, 68u, 41u, 69u, + 41u, 82u, 41u, 76u, 41u, 70u, 73u, 41u, + 77u, 41u, 73u, 41u, 84u, 41u, 95u, 41u, + 77u, 82u, 41u, 65u, 41u, 84u, 41u, 67u, + 41u, 72u, 41u, 61u, 41u, 48u, 57u, 41u, + 48u, 57u, 41u, 69u, 41u, 67u, 41u, 85u, + 41u, 82u, 41u, 83u, 41u, 73u, 41u, 79u, + 41u, 78u, 41u, 79u, 41u, 95u, 41u, 65u, + 83u, 41u, 85u, 41u, 84u, 41u, 79u, 41u, + 95u, 41u, 80u, 41u, 79u, 41u, 83u, 41u, + 83u, 41u, 69u, 41u, 83u, 41u, 83u, 41u, + 84u, 41u, 65u, 41u, 82u, 41u, 84u, 41u, + 95u, 41u, 79u, 41u, 80u, 41u, 84u, 41u, + 67u, 84u, 41u, 80u, 41u, 41u, 70u, 41u, + 49u, 51u, 56u, 41u, 54u, 41u, 50u, 41u, + 40u, 42u, 0u + }; + + static const signed char _ControlVerbs_single_lengths[] = { + 7, 1, 2, 2, 2, 2, 2, 2, + 1, 2, 2, 2, 3, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, + 3, 2, 2, 2, 2, 3, 2, 2, + 2, 2, 2, 1, 1, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 3, + 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 3, 2, 1, 2, 4, + 2, 2, 1, 1, 1, 0 + }; + + static const signed char _ControlVerbs_range_lengths[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 1, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0 + }; + + static const short _ControlVerbs_index_offsets[] = { + 0, 8, 10, 13, 16, 19, 22, 25, + 28, 30, 33, 36, 39, 43, 46, 49, + 52, 55, 58, 61, 64, 67, 70, 73, + 76, 80, 83, 86, 89, 92, 96, 99, + 102, 105, 108, 111, 114, 117, 120, 123, + 126, 129, 132, 135, 138, 141, 144, 147, + 151, 154, 157, 160, 163, 166, 169, 172, + 175, 178, 181, 184, 187, 190, 193, 196, + 199, 202, 205, 208, 212, 215, 217, 220, + 225, 228, 231, 233, 235, 0 + }; + + static const signed char _ControlVerbs_cond_targs[] = { + 75, 2, 9, 22, 24, 45, 67, 1, + 75, 1, 75, 3, 1, 75, 4, 1, + 75, 5, 1, 75, 6, 1, 75, 7, + 1, 75, 8, 1, 75, 1, 75, 10, + 1, 75, 11, 1, 75, 12, 1, 75, + 13, 16, 1, 75, 14, 1, 75, 15, + 1, 75, 5, 1, 75, 17, 1, 75, + 18, 1, 75, 19, 1, 75, 20, 1, + 75, 21, 1, 75, 8, 1, 75, 23, + 1, 75, 7, 1, 75, 8, 25, 1, + 75, 26, 1, 75, 27, 1, 75, 28, + 1, 75, 29, 1, 75, 30, 37, 1, + 75, 31, 1, 75, 32, 1, 75, 33, + 1, 75, 34, 1, 75, 35, 1, 75, + 36, 1, 75, 36, 1, 75, 38, 1, + 75, 39, 1, 75, 40, 1, 75, 41, + 1, 75, 42, 1, 75, 43, 1, 75, + 44, 1, 75, 34, 1, 75, 46, 1, + 75, 47, 1, 75, 48, 59, 1, 75, + 49, 1, 75, 50, 1, 75, 51, 1, + 75, 52, 1, 75, 53, 1, 75, 54, + 1, 75, 55, 1, 75, 56, 1, 75, + 57, 1, 75, 58, 1, 75, 8, 1, + 75, 60, 1, 75, 61, 1, 75, 62, + 1, 75, 63, 1, 75, 64, 1, 75, + 65, 1, 75, 66, 1, 75, 8, 1, + 75, 68, 70, 1, 75, 69, 1, 75, + 1, 75, 71, 1, 75, 72, 73, 74, + 1, 75, 8, 1, 75, 8, 1, 75, + 1, 76, 75, 0, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 0 + }; + + static const signed char _ControlVerbs_cond_actions[] = { + 19, 0, 0, 0, 0, 0, 0, 0, + 13, 0, 13, 0, 0, 13, 0, 0, + 11, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 11, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 11, 0, 0, 13, 0, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 11, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 0, 13, 0, 0, 9, + 0, 13, 0, 0, 7, 0, 0, 0, + 0, 13, 0, 0, 13, 0, 0, 7, + 0, 5, 15, 0, 17, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 0, 17, 0 + }; + + static const signed char _ControlVerbs_to_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 0, 0 + }; + + static const signed char _ControlVerbs_from_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 3, 0, 0 + }; + + static const short _ControlVerbs_eof_trans[] = { + 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 277, + 278, 279, 280, 281, 282, 283, 284, 285, + 286, 287, 288, 289, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, + 310, 311, 312, 313, 314, 0 + }; + + static const int ControlVerbs_start = 75; + static const int ControlVerbs_first_final = 75; + static const int ControlVerbs_error = -1; + + static const int ControlVerbs_en_main = 75; + - -#line 59 "control_verbs.cpp" -static const char _ControlVerbs_actions[] = { - 0, 1, 0, 1, 1, 1, 2, 1, - 3, 1, 4, 1, 5, 1, 6, 1, - 7, 1, 8, 1, 9 -}; - -static const unsigned char _ControlVerbs_key_offsets[] = { - 0, 7, 8, 10, 12, 14, 16, 18, - 20, 21, 23, 25, 27, 30, 32, 34, - 36, 38, 40, 42, 44, 46, 48, 50, - 52, 55, 57, 59, 61, 63, 66, 68, - 70, 72, 74, 76, 79, 82, 84, 86, - 88, 90, 92, 94, 96, 98, 100, 102, - 105, 107, 109, 111, 113, 115, 117, 119, - 121, 123, 125, 127, 129, 131, 133, 135, - 137, 139, 141, 143, 146, 148, 149, 151, - 155, 157, 159, 160, 161 -}; - -static const char _ControlVerbs_trans_keys[] = { - 41, 65, 66, 67, 76, 78, 85, 41, - 41, 78, 41, 89, 41, 67, 41, 82, - 41, 76, 41, 70, 41, 41, 83, 41, - 82, 41, 95, 41, 65, 85, 41, 78, - 41, 89, 41, 67, 41, 78, 41, 73, - 41, 67, 41, 79, 41, 68, 41, 69, - 41, 82, 41, 76, 41, 70, 73, 41, - 77, 41, 73, 41, 84, 41, 95, 41, - 77, 82, 41, 65, 41, 84, 41, 67, - 41, 72, 41, 61, 41, 48, 57, 41, - 48, 57, 41, 69, 41, 67, 41, 85, - 41, 82, 41, 83, 41, 73, 41, 79, - 41, 78, 41, 79, 41, 95, 41, 65, - 83, 41, 85, 41, 84, 41, 79, 41, - 95, 41, 80, 41, 79, 41, 83, 41, - 83, 41, 69, 41, 83, 41, 83, 41, - 84, 41, 65, 41, 82, 41, 84, 41, - 95, 41, 79, 41, 80, 41, 84, 41, - 67, 84, 41, 80, 41, 41, 70, 41, - 49, 51, 56, 41, 54, 41, 50, 41, - 40, 42, 0 -}; - -static const char _ControlVerbs_single_lengths[] = { - 7, 1, 2, 2, 2, 2, 2, 2, - 1, 2, 2, 2, 3, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, - 3, 2, 2, 2, 2, 3, 2, 2, - 2, 2, 2, 1, 1, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 3, - 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 3, 2, 1, 2, 4, - 2, 2, 1, 1, 1 -}; - -static const char _ControlVerbs_range_lengths[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1, 1, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0 -}; - -static const short _ControlVerbs_index_offsets[] = { - 0, 8, 10, 13, 16, 19, 22, 25, - 28, 30, 33, 36, 39, 43, 46, 49, - 52, 55, 58, 61, 64, 67, 70, 73, - 76, 80, 83, 86, 89, 92, 96, 99, - 102, 105, 108, 111, 114, 117, 120, 123, - 126, 129, 132, 135, 138, 141, 144, 147, - 151, 154, 157, 160, 163, 166, 169, 172, - 175, 178, 181, 184, 187, 190, 193, 196, - 199, 202, 205, 208, 212, 215, 217, 220, - 225, 228, 231, 233, 235 -}; - -static const char _ControlVerbs_indicies[] = { - 0, 2, 3, 4, 5, 6, 7, 1, - 8, 1, 8, 9, 1, 8, 10, 1, - 11, 12, 1, 8, 13, 1, 8, 14, - 1, 8, 15, 1, 11, 1, 8, 16, - 1, 8, 17, 1, 8, 18, 1, 8, - 19, 20, 1, 8, 21, 1, 8, 22, - 1, 8, 12, 1, 8, 23, 1, 8, - 24, 1, 8, 25, 1, 8, 26, 1, - 8, 27, 1, 8, 15, 1, 8, 28, - 1, 11, 14, 1, 8, 15, 29, 1, - 8, 30, 1, 8, 31, 1, 8, 32, - 1, 8, 33, 1, 8, 34, 35, 1, - 8, 36, 1, 8, 37, 1, 8, 38, - 1, 8, 39, 1, 8, 40, 1, 8, - 41, 1, 11, 41, 1, 8, 42, 1, - 8, 43, 1, 8, 44, 1, 8, 45, - 1, 8, 46, 1, 8, 47, 1, 8, - 48, 1, 8, 39, 1, 8, 49, 1, - 8, 50, 1, 8, 51, 52, 1, 8, - 53, 1, 8, 54, 1, 8, 55, 1, - 8, 56, 1, 8, 57, 1, 8, 58, - 1, 8, 59, 1, 8, 60, 1, 8, - 61, 1, 8, 62, 1, 8, 15, 1, - 8, 63, 1, 8, 64, 1, 8, 65, - 1, 8, 66, 1, 8, 67, 1, 8, - 68, 1, 8, 69, 1, 8, 15, 1, - 8, 70, 71, 1, 8, 72, 1, 73, - 1, 8, 74, 1, 75, 76, 77, 78, - 1, 8, 15, 1, 8, 15, 1, 75, - 1, 80, 79, 82, 81, 0 -}; - -static const char _ControlVerbs_trans_targs[] = { - 75, 1, 2, 9, 22, 24, 45, 67, - 75, 3, 4, 75, 5, 6, 7, 8, - 10, 11, 12, 13, 16, 14, 15, 17, - 18, 19, 20, 21, 23, 25, 26, 27, - 28, 29, 30, 37, 31, 32, 33, 34, - 35, 36, 38, 39, 40, 41, 42, 43, - 44, 46, 47, 48, 59, 49, 50, 51, - 52, 53, 54, 55, 56, 57, 58, 60, - 61, 62, 63, 64, 65, 66, 68, 70, - 69, 75, 71, 75, 72, 73, 74, 75, - 76, 75, 0 -}; - -static const char _ControlVerbs_trans_actions[] = { - 19, 0, 0, 0, 0, 0, 0, 0, - 13, 0, 0, 11, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 9, 0, 7, 0, 0, 0, 15, - 5, 17, 0 -}; - -static const char _ControlVerbs_to_state_actions[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1, 0 -}; - -static const char _ControlVerbs_from_state_actions[] = { - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3, 0 -}; - -static const short _ControlVerbs_eof_trans[] = { - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 0, 82 -}; - -static const int ControlVerbs_start = 75; -static const int ControlVerbs_first_final = 75; -static const int ControlVerbs_error = -1; - -static const int ControlVerbs_en_main = 75; - - -#line 249 "control_verbs.cpp" +#line 269 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" { - cs = ControlVerbs_start; - ts = 0; - te = 0; - act = 0; - } - -#line 105 "control_verbs.rl" - - - try { - -#line 262 "control_verbs.cpp" - { - int _klen; - unsigned int _trans; - const char *_acts; - unsigned int _nacts; - const char *_keys; - - if ( p == pe ) - goto _test_eof; -_resume: - _acts = _ControlVerbs_actions + _ControlVerbs_from_state_actions[cs]; - _nacts = (unsigned int) *_acts++; - while ( _nacts-- > 0 ) { - switch ( *_acts++ ) { - case 1: -#line 1 "NONE" - {ts = p;} - break; -#line 281 "control_verbs.cpp" + cs = (int)ControlVerbs_start; + ts = 0; + te = 0; } - } + +#line 105 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" - _keys = _ControlVerbs_trans_keys + _ControlVerbs_key_offsets[cs]; - _trans = _ControlVerbs_index_offsets[cs]; + + try { - _klen = _ControlVerbs_single_lengths[cs]; - if ( _klen > 0 ) { - const char *_lower = _keys; - const char *_mid; - const char *_upper = _keys + _klen - 1; - while (1) { - if ( _upper < _lower ) - break; +#line 278 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + { + int _klen; + unsigned int _trans = 0; + const char * _keys; + const signed char * _acts; + unsigned int _nacts; + _resume: {} + if ( p == pe && p != eof ) + goto _out; + _acts = ( _ControlVerbs_actions + (_ControlVerbs_from_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 1: { + { +#line 1 "NONE" + {ts = p;}} + +#line 297 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" - _mid = _lower + ((_upper-_lower) >> 1); - if ( (*p) < *_mid ) - _upper = _mid - 1; - else if ( (*p) > *_mid ) - _lower = _mid + 1; - else { - _trans += (unsigned int)(_mid - _keys); - goto _match; + break; + } + } + _nacts -= 1; + _acts += 1; + } + + if ( p == eof ) { + if ( _ControlVerbs_eof_trans[cs] > 0 ) { + _trans = (unsigned int)_ControlVerbs_eof_trans[cs] - 1; + } + } + else { + _keys = ( _ControlVerbs_trans_keys + (_ControlVerbs_key_offsets[cs])); + _trans = (unsigned int)_ControlVerbs_index_offsets[cs]; + + _klen = (int)_ControlVerbs_single_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + _klen - 1; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _keys += _klen; + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + ((_upper-_lower) >> 1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 1; + else if ( ( (*( p))) > (*( _mid)) ) + _lower = _mid + 1; + else { + _trans += (unsigned int)(_mid - _keys); + goto _match; + } + } + } + + _klen = (int)_ControlVerbs_range_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + (_klen<<1) - 2; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + (((_upper-_lower) >> 1) & ~1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 2; + else if ( ( (*( p))) > (*( _mid + 1)) ) + _lower = _mid + 2; + else { + _trans += (unsigned int)((_mid - _keys)>>1); + break; + } + } + } + + _match: {} + } + cs = (int)_ControlVerbs_cond_targs[_trans]; + + if ( _ControlVerbs_cond_actions[_trans] != 0 ) { + + _acts = ( _ControlVerbs_actions + (_ControlVerbs_cond_actions[_trans])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) + { + case 2: { + { +#line 1 "NONE" + {te = p+1;}} + +#line 378 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 3: { + { +#line 76 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {te = p+1;{ +#line 76 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + mode.utf8 = true; + } + }} + +#line 391 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 4: { + { +#line 80 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {te = p+1;{ +#line 80 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + mode.ucp = true; + } + }} + +#line 404 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 5: { + { +#line 84 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {te = p+1;{ +#line 84 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + ostringstream str; + str << "Unsupported control verb " << string(ts, te - ts); + throw LocatedParseError(str.str()); + } + }} + +#line 419 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 6: { + { +#line 90 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {te = p+1;{ +#line 90 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + ostringstream str; + str << "Unknown control verb " << string(ts, te - ts); + throw LocatedParseError(str.str()); + } + }} + +#line 434 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 7: { + { +#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {te = p+1;{ +#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + {p = p - 1; } + {p += 1; goto _out; } + } + }} + +#line 448 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 8: { + { +#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {te = p;p = p - 1;{ +#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + {p = p - 1; } + {p += 1; goto _out; } + } + }} + +#line 462 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + case 9: { + { +#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + {p = ((te))-1; + { +#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" + + {p = p - 1; } + {p += 1; goto _out; } + } + }} + +#line 477 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + } + + if ( p == eof ) { + if ( cs >= 75 ) + goto _out; + } + else { + _acts = ( _ControlVerbs_actions + (_ControlVerbs_to_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 0: { + { +#line 1 "NONE" + {ts = 0;}} + +#line 502 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + p += 1; + goto _resume; + } + _out: {} } - } - _keys += _klen; - _trans += _klen; - } + +#line 108 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" - _klen = _ControlVerbs_range_lengths[cs]; - if ( _klen > 0 ) { - const char *_lower = _keys; - const char *_mid; - const char *_upper = _keys + (_klen<<1) - 2; - while (1) { - if ( _upper < _lower ) - break; - - _mid = _lower + (((_upper-_lower) >> 1) & ~1); - if ( (*p) < _mid[0] ) - _upper = _mid - 2; - else if ( (*p) > _mid[1] ) - _lower = _mid + 2; - else { - _trans += (unsigned int)((_mid - _keys)>>1); - goto _match; + } catch (LocatedParseError &error) { + if (ts >= ptr && ts <= pe) { + error.locate(ts - ptr + start); + } else { + error.locate(0); } + throw; } - _trans += _klen; + + return p; } - -_match: - _trans = _ControlVerbs_indicies[_trans]; -_eof_trans: - cs = _ControlVerbs_trans_targs[_trans]; - - if ( _ControlVerbs_trans_actions[_trans] == 0 ) - goto _again; - - _acts = _ControlVerbs_actions + _ControlVerbs_trans_actions[_trans]; - _nacts = (unsigned int) *_acts++; - while ( _nacts-- > 0 ) - { - switch ( *_acts++ ) - { - case 2: -#line 1 "NONE" - {te = p+1;} - break; - case 3: -#line 76 "control_verbs.rl" - {te = p+1;{ - mode.utf8 = true; - }} - break; - case 4: -#line 80 "control_verbs.rl" - {te = p+1;{ - mode.ucp = true; - }} - break; - case 5: -#line 84 "control_verbs.rl" - {te = p+1;{ - ostringstream str; - str << "Unsupported control verb " << string(ts, te - ts); - throw LocatedParseError(str.str()); - }} - break; - case 6: -#line 90 "control_verbs.rl" - {te = p+1;{ - ostringstream str; - str << "Unknown control verb " << string(ts, te - ts); - throw LocatedParseError(str.str()); - }} - break; - case 7: -#line 97 "control_verbs.rl" - {te = p+1;{ - p--; - {p++; goto _out; } - }} - break; - case 8: -#line 97 "control_verbs.rl" - {te = p;p--;{ - p--; - {p++; goto _out; } - }} - break; - case 9: -#line 97 "control_verbs.rl" - {{p = ((te))-1;}{ - p--; - {p++; goto _out; } - }} - break; -#line 400 "control_verbs.cpp" - } - } - -_again: - _acts = _ControlVerbs_actions + _ControlVerbs_to_state_actions[cs]; - _nacts = (unsigned int) *_acts++; - while ( _nacts-- > 0 ) { - switch ( *_acts++ ) { - case 0: -#line 1 "NONE" - {ts = 0;} - break; -#line 413 "control_verbs.cpp" - } - } - - if ( ++p != pe ) - goto _resume; - _test_eof: {} - if ( p == eof ) - { - if ( _ControlVerbs_eof_trans[cs] > 0 ) { - _trans = _ControlVerbs_eof_trans[cs] - 1; - goto _eof_trans; - } - } - - _out: {} - } - -#line 109 "control_verbs.rl" - } catch (LocatedParseError &error) { - if (ts >= ptr && ts <= pe) { - error.locate(ts - ptr + start); - } else { - error.locate(0); - } - throw; - } - - return p; -} - + } // namespace ue2 From 6ded03c000c24e3c1837fe9fe96ac32ed6956911 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Sep 2022 00:00:10 +0200 Subject: [PATCH 420/582] Disable fetch shortcut for zero copy replication --- src/Storages/StorageReplicatedMergeTree.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index aafa2be5c55..00b99d3e6df 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3831,9 +3831,10 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora LOG_DEBUG(log, "Fetching part {} from {}", part_name, source_replica_path); + auto settings_ptr = getSettings(); TableLockHolder table_lock_holder; if (!to_detached) - table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, settings_ptr->lock_acquire_timeout_for_background_operations); /// Logging Stopwatch stopwatch; @@ -3857,7 +3858,8 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora covered_part_info.mutation = 0; auto source_part = getActiveContainingPart(covered_part_info); - if (source_part) + /// Fetch for zero-copy replication is cheap and straightforward, so we don't use local clone here + if (source_part && (!settings_ptr->allow_remote_fs_zero_copy_replication || !source_part->data_part_storage->supportZeroCopyReplication())) { auto source_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( source_part->getColumns(), source_part->checksums); @@ -3897,7 +3899,6 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora part_to_clone = source_part; } } - } ReplicatedMergeTreeAddress address; From c4297c6c997056531198dc0b4b6ebba10ed9d65f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Sep 2022 21:57:59 +0000 Subject: [PATCH 421/582] fix: disable narrowing warnings --- contrib/vectorscan-cmake/CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index b19215ba6f3..bf6f82eadc0 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -242,6 +242,12 @@ list (APPEND SRCS "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/control_verbs.cpp" ) +set_source_files_properties( + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/Parser.cpp" + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/control_verbs.cpp" + COMPILE_FLAGS -Wno-c++11-narrowing +) + # Platform-dependent files if (ARCH_AMD64) list(APPEND SRCS From 3073da9ba5c789890cd769fcbfbfdaa8606d7155 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 6 Sep 2022 18:39:06 -0400 Subject: [PATCH 422/582] move doc --- .../example-datasets/nypd_complaint_data.md | 654 ++++++++++++++++++ 1 file changed, 654 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/nypd_complaint_data.md diff --git a/docs/en/getting-started/example-datasets/nypd_complaint_data.md b/docs/en/getting-started/example-datasets/nypd_complaint_data.md new file mode 100644 index 00000000000..8b02ac23cf9 --- /dev/null +++ b/docs/en/getting-started/example-datasets/nypd_complaint_data.md @@ -0,0 +1,654 @@ +--- +slug: /en/getting-started/example-datasets/nypd_complaint_data +sidebar_label: NYPD Complaint Data +description: "Ingest and query Tab Separated Value data in 5 steps" +title: NYPD Complaint Data +--- + +Tab separated value, or TSV, files are common and may include field headings as the first line of the file. ClickHouse can ingest TSVs, and also can query TSVs without ingesting the files. This guide covers both of these cases. If you need to query or ingest CSV files, the same techniques work, simply substitute `TSV` with `CSV` in your format arguments. + +While working through this guide you will: +- **Investigate**: Query the structure and content of the TSV file. +- **Determine the target ClickHouse schema**: Choose proper data types and map the existing data to those types. +- **Create a ClickHouse table**. +- **Preprocess and stream** the data to ClickHouse. +- **Run some queries** against ClickHouse. + +The dataset used in this guide comes from the NYC Open Data team, and contains data about "all valid felony, misdemeanor, and violation crimes reported to the New York City Police Department (NYPD)". At the time of writing, the data file is 166MB, but it is updated regularly. + +**Source**: [data.cityofnewyork.us](https://data.cityofnewyork.us/Public-Safety/NYPD-Complaint-Data-Current-Year-To-Date-/5uac-w243) +**Terms of use**: https://www1.nyc.gov/home/terms-of-use.page + +## Prerequisites +- Download the dataset by visiting the [NYPD Complaint Data Current (Year To Date)](https://data.cityofnewyork.us/Public-Safety/NYPD-Complaint-Data-Current-Year-To-Date-/5uac-w243) page, clicking the Export button, and choosing **TSV for Excel**. +- Install [ClickHouse server and client](../../getting-started/install.md). +- [Launch](../../getting-started/install.md#launch) ClickHouse server, and connect with `clickhouse-client` + +### A note about the commands described in this guide +There are two types of commands in this guide: +- Some of the commands are querying the TSV files, these are run at the command prompt. +- The rest of the commands are querying ClickHouse, and these are run in the `clickhouse-client` or Play UI. + +:::note +The examples in this guide assume that you have saved the TSV file to `${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv`, please adjust the commands if needed. +::: + +## Familiarize yourself with the TSV file + +Before starting to work with the ClickHouse database familiarize yourself with the data. + +### Look at the fields in the source TSV file + +This is an example of a command to query a TSV file, but don't run it yet. +```sh +clickhouse-local --query \ +"describe file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames')" +``` + +Sample response +```response +CMPLNT_NUM Nullable(Float64) +ADDR_PCT_CD Nullable(Float64) +BORO_NM Nullable(String) +CMPLNT_FR_DT Nullable(String) +CMPLNT_FR_TM Nullable(String) +``` + +:::tip +Most of the time the above command will let you know which fields in the input data are numeric, and which are strings, and which are tuples. This is not always the case. Because ClickHouse is routineley used with datasets containing billions of records there is a default number (100) of rows examined to [infer the schema](../../guides/developer/working-with-json/json-semi-structured.md/#relying-on-schema-inference) in order to avoid parsing billions of rows to infer the schema. The response below may not match what you see, as the dataset is updated several times each year. Looking at the Data Dictionary you can see that CMPLNT_NUM is specified as text, and not numeric. By overriding the default of 100 rows for inference with the setting `SETTINGS input_format_max_rows_to_read_for_schema_inference=2000` +you can get a better idea of the content. + +Note: as of version 22.5 the default is now 25,000 rows for inferring the schema, so only change the setting if you are on an older version or if you need more than 25,000 rows to be sampled. +::: + +Run this command at your command prompt. You will be using `clickhouse-local` to query the data in the TSV file you downloaded. +```sh +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"describe file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames')" +``` + +Result: +```response +CMPLNT_NUM Nullable(String) +ADDR_PCT_CD Nullable(Float64) +BORO_NM Nullable(String) +CMPLNT_FR_DT Nullable(String) +CMPLNT_FR_TM Nullable(String) +CMPLNT_TO_DT Nullable(String) +CMPLNT_TO_TM Nullable(String) +CRM_ATPT_CPTD_CD Nullable(String) +HADEVELOPT Nullable(String) +HOUSING_PSA Nullable(Float64) +JURISDICTION_CODE Nullable(Float64) +JURIS_DESC Nullable(String) +KY_CD Nullable(Float64) +LAW_CAT_CD Nullable(String) +LOC_OF_OCCUR_DESC Nullable(String) +OFNS_DESC Nullable(String) +PARKS_NM Nullable(String) +PATROL_BORO Nullable(String) +PD_CD Nullable(Float64) +PD_DESC Nullable(String) +PREM_TYP_DESC Nullable(String) +RPT_DT Nullable(String) +STATION_NAME Nullable(String) +SUSP_AGE_GROUP Nullable(String) +SUSP_RACE Nullable(String) +SUSP_SEX Nullable(String) +TRANSIT_DISTRICT Nullable(Float64) +VIC_AGE_GROUP Nullable(String) +VIC_RACE Nullable(String) +VIC_SEX Nullable(String) +X_COORD_CD Nullable(Float64) +Y_COORD_CD Nullable(Float64) +Latitude Nullable(Float64) +Longitude Nullable(Float64) +Lat_Lon Tuple(Nullable(Float64), Nullable(Float64)) +New Georeferenced Column Nullable(String) +``` + +At this point you should check that the columns in the TSV file match the names and types specified in the **Columns in this Dataset** section of the [dataset web page](https://data.cityofnewyork.us/Public-Safety/NYPD-Complaint-Data-Current-Year-To-Date-/5uac-w243). The data types are not very specific, all numeric fields are set to `Nullable(Float64)`, and all other fields are `Nullable(String)`. When you create a ClickHouse table to store the data you can specify more appropriate and performant types. + +### Determine the proper schema + +In order to figure out what types should be used for the fields it is necessary to know what the data looks like. For example, the field `JURISDICTION_CODE` is a numeric: should it be a `UInt8`, or an `Enum`, or is `Float64` appropriate? + +```sql +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select JURISDICTION_CODE, count() FROM + file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') + GROUP BY JURISDICTION_CODE + ORDER BY JURISDICTION_CODE + FORMAT PrettyCompact" +``` + +Result: +```response +┌─JURISDICTION_CODE─┬─count()─┐ +│ 0 │ 188875 │ +│ 1 │ 4799 │ +│ 2 │ 13833 │ +│ 3 │ 656 │ +│ 4 │ 51 │ +│ 6 │ 5 │ +│ 7 │ 2 │ +│ 9 │ 13 │ +│ 11 │ 14 │ +│ 12 │ 5 │ +│ 13 │ 2 │ +│ 14 │ 70 │ +│ 15 │ 20 │ +│ 72 │ 159 │ +│ 87 │ 9 │ +│ 88 │ 75 │ +│ 97 │ 405 │ +└───────────────────┴─────────┘ +``` + +The query response shows that the `JURISDICTION_CODE` fits well in a `UInt8`. + +Similarly, look at some of the `String` fields and see if they are well suited to being `DateTime` or [`LowCardinality(String)`](../../sql-reference/data-types/lowcardinality.md) fields. + +For example, the field `PARKS_NM` is described as "Name of NYC park, playground or greenspace of occurrence, if applicable (state parks are not included)". The names of parks in New York City may be a good candidate for a `LowCardinality(String)`: + +```sh +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select count(distinct PARKS_NM) FROM + file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') + FORMAT PrettyCompact" +``` + +Result: +```response +┌─uniqExact(PARKS_NM)─┐ +│ 319 │ +└─────────────────────┘ +``` + +Have a look at some of the park names: +```sql +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select distinct PARKS_NM FROM + file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') + LIMIT 10 + FORMAT PrettyCompact" +``` + +Result: +```response +┌─PARKS_NM───────────────────┐ +│ (null) │ +│ ASSER LEVY PARK │ +│ JAMES J WALKER PARK │ +│ BELT PARKWAY/SHORE PARKWAY │ +│ PROSPECT PARK │ +│ MONTEFIORE SQUARE │ +│ SUTTON PLACE PARK │ +│ JOYCE KILMER PARK │ +│ ALLEY ATHLETIC PLAYGROUND │ +│ ASTORIA PARK │ +└────────────────────────────┘ +``` + +The dataset in use at the time of writing has only a few hundred distinct parks and playgrounds in the `PARK_NM` column. This is a small number based on the [LowCardinality](../../sql-reference/data-types/lowcardinality.md#lowcardinality-dscr) recommendation to stay below 10,000 distinct strings in a `LowCardinality(String)` field. + +### DateTime fields +Based on the **Columns in this Dataset** section of the [dataset web page](https://data.cityofnewyork.us/Public-Safety/NYPD-Complaint-Data-Current-Year-To-Date-/5uac-w243) there are date and time fields for the start and end of the reported event. Looking at the min and max of the `CMPLNT_FR_DT` and `CMPLT_TO_DT` gives an idea of whether or not the fields are always populated: + +```sh title="CMPLNT_FR_DT" +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select min(CMPLNT_FR_DT), max(CMPLNT_FR_DT) FROM +file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') +FORMAT PrettyCompact" +``` + +Result: +```response +┌─min(CMPLNT_FR_DT)─┬─max(CMPLNT_FR_DT)─┐ +│ 01/01/1973 │ 12/31/2021 │ +└───────────────────┴───────────────────┘ +``` + +```sh title="CMPLNT_TO_DT" +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select min(CMPLNT_TO_DT), max(CMPLNT_TO_DT) FROM +file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') +FORMAT PrettyCompact" +``` + +Result: +```response +┌─min(CMPLNT_TO_DT)─┬─max(CMPLNT_TO_DT)─┐ +│ │ 12/31/2021 │ +└───────────────────┴───────────────────┘ +``` + +```sh title="CMPLNT_FR_TM" +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select min(CMPLNT_FR_TM), max(CMPLNT_FR_TM) FROM +file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') +FORMAT PrettyCompact" +``` + +Result: +```response +┌─min(CMPLNT_FR_TM)─┬─max(CMPLNT_FR_TM)─┐ +│ 00:00:00 │ 23:59:00 │ +└───────────────────┴───────────────────┘ +``` + +```sh title="CMPLNT_TO_TM" +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select min(CMPLNT_TO_TM), max(CMPLNT_TO_TM) FROM +file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') +FORMAT PrettyCompact" +``` + +Result: +```response +┌─min(CMPLNT_TO_TM)─┬─max(CMPLNT_TO_TM)─┐ +│ (null) │ 23:59:00 │ +└───────────────────┴───────────────────┘ +``` + +## Make a plan + +Based on the above investigation: +- `JURISDICTION_CODE` should be cast as `UInt8`. +- `PARKS_NM` should be cast to `LowCardinality(String)` +- `CMPLNT_FR_DT` and `CMPLNT_FR_TM` are always populated (possibly with a default time of `00:00:00`) +- `CMPLNT_TO_DT` and `CMPLNT_TO_TM` may be empty +- Dates and times are stored in separate fields in the source +- Dates are `mm/dd/yyyy` format +- Times are `hh:mm:ss` format +- Dates and times can be concatenated into DateTime types +- There are some dates before January 1st 1970, which means we need a 64 bit DateTime + +:::note +There are many more changes to be made to the types, they all can be determined by following the same investigation steps. Look at the number of distinct strings in a field, the min and max of the numerics, and make your decisions. The table schema that is given later in the guide has many low cardinality strings and unsigned integer fields and very few floating point numerics. +::: + +## Concatenate the date and time fields + +To concatenate the date and time fields `CMPLNT_FR_DT` and `CMPLNT_FR_TM` into a single `String` that can be cast to a `DateTime`, select the two fields joined by the concatenation operator: `CMPLNT_FR_DT || ' ' || CMPLNT_FR_TM`. The `CMPLNT_TO_DT` and `CMPLNT_TO_TM` fields are handled similarly. + +```sh +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select CMPLNT_FR_DT || ' ' || CMPLNT_FR_TM AS complaint_begin FROM +file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') +LIMIT 10 +FORMAT PrettyCompact" +``` + +Result: +```response +┌─complaint_begin─────┐ +│ 07/29/2010 00:01:00 │ +│ 12/01/2011 12:00:00 │ +│ 04/01/2017 15:00:00 │ +│ 03/26/2018 17:20:00 │ +│ 01/01/2019 00:00:00 │ +│ 06/14/2019 00:00:00 │ +│ 11/29/2021 20:00:00 │ +│ 12/04/2021 00:35:00 │ +│ 12/05/2021 12:50:00 │ +│ 12/07/2021 20:30:00 │ +└─────────────────────┘ +``` + +## Convert the date and time String to a DateTime64 type + +Earlier in the guide we discovered that there are dates in the TSV file before January 1st 1970, which means that we need a 64 bit DateTime type for the dates. The dates also need to be converted from `MM/DD/YYYY` to `YYYY/MM/DD` format. Both of these can be done with [`parseDateTime64BestEffort()`](../../sql-reference/functions/type-conversion-functions.md#parsedatetime64besteffort). + +```sh +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"WITH (CMPLNT_FR_DT || ' ' || CMPLNT_FR_TM) AS CMPLNT_START, + (CMPLNT_TO_DT || ' ' || CMPLNT_TO_TM) AS CMPLNT_END +select parseDateTime64BestEffort(CMPLNT_START) AS complaint_begin, + parseDateTime64BestEffortOrNull(CMPLNT_END) AS complaint_end +FROM file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') +ORDER BY complaint_begin ASC +LIMIT 25 +FORMAT PrettyCompact" +``` + +Lines 2 and 3 above contain the concatenation from the previous step, and lines 4 and 5 above parse the strings into `DateTime64`. As the complaint end time is not guaranteed to exist `parseDateTime64BestEffortOrNull` is used. + +Result: +```response +┌─────────complaint_begin─┬───────────complaint_end─┐ +│ 1925-01-01 10:00:00.000 │ 2021-02-12 09:30:00.000 │ +│ 1925-01-01 11:37:00.000 │ 2022-01-16 11:49:00.000 │ +│ 1925-01-01 15:00:00.000 │ 2021-12-31 00:00:00.000 │ +│ 1925-01-01 15:00:00.000 │ 2022-02-02 22:00:00.000 │ +│ 1925-01-01 19:00:00.000 │ 2022-04-14 05:00:00.000 │ +│ 1955-09-01 19:55:00.000 │ 2022-08-01 00:45:00.000 │ +│ 1972-03-17 11:40:00.000 │ 2022-03-17 11:43:00.000 │ +│ 1972-05-23 22:00:00.000 │ 2022-05-24 09:00:00.000 │ +│ 1972-05-30 23:37:00.000 │ 2022-05-30 23:50:00.000 │ +│ 1972-07-04 02:17:00.000 │ ᴺᵁᴸᴸ │ +│ 1973-01-01 00:00:00.000 │ ᴺᵁᴸᴸ │ +│ 1975-01-01 00:00:00.000 │ ᴺᵁᴸᴸ │ +│ 1976-11-05 00:01:00.000 │ 1988-10-05 23:59:00.000 │ +│ 1977-01-01 00:00:00.000 │ 1977-01-01 23:59:00.000 │ +│ 1977-12-20 00:01:00.000 │ ᴺᵁᴸᴸ │ +│ 1981-01-01 00:01:00.000 │ ᴺᵁᴸᴸ │ +│ 1981-08-14 00:00:00.000 │ 1987-08-13 23:59:00.000 │ +│ 1983-01-07 00:00:00.000 │ 1990-01-06 00:00:00.000 │ +│ 1984-01-01 00:01:00.000 │ 1984-12-31 23:59:00.000 │ +│ 1985-01-01 12:00:00.000 │ 1987-12-31 15:00:00.000 │ +│ 1985-01-11 09:00:00.000 │ 1985-12-31 12:00:00.000 │ +│ 1986-03-16 00:05:00.000 │ 2022-03-16 00:45:00.000 │ +│ 1987-01-07 00:00:00.000 │ 1987-01-09 00:00:00.000 │ +│ 1988-04-03 18:30:00.000 │ 2022-08-03 09:45:00.000 │ +│ 1988-07-29 12:00:00.000 │ 1990-07-27 22:00:00.000 │ +└─────────────────────────┴─────────────────────────┘ +``` +:::note +The dates shown as `1925` above are from errors in the data. There are several records in the original data with dates in the years `1019` - `1022` that should be `2019` - `2022`. They are being stored as Jan 1st 1925 as that is the earliest date with a 64 bit DateTime. +::: + +## Create a table + +The decisions made above on the data types used for the columns are reflected in the table schema +below. We also need to decide on the `ORDER BY` and `PRIMARY KEY` used for the table. At least one +of `ORDER BY` or `PRIMARY KEY` must be specified. Here are some guidelines on deciding on the +columns to includes in `ORDER BY`, and more information is in the *Next Steps* section at the end +of this document. + +### Order By and Primary Key clauses + +- The `ORDER BY` tuple should include fields that are used in query filters +- To maximize compression on disk the `ORDER BY` tuple should be ordered by ascending cardinality +- If it exists, the `PRIMARY KEY` tuple must be a subset of the `ORDER BY` tuple +- If only `ORDER BY` is specified, then the same tuple will be used as `PRIMARY KEY` +- The primary key index is created using the `PRIMARY KEY` tuple if specified, otherwise the `ORDER BY` tuple +- The `PRIMARY KEY` index is kept in main memory + +Looking at the dataset and the questions that might be answered by querying it we might +decide that we would look at the types of crimes reported over time in the five boroughs of +New York City. These fields might be then included in the `ORDER BY`: + +| Column | Description (from the data dictionary) | +| ----------- | --------------------------------------------------- | +| OFNS_DESC | Description of offense corresponding with key code | +| RPT_DT | Date event was reported to police | +| BORO_NM | The name of the borough in which the incident occurred | + + +Querying the TSV file for the cardinality of the three candidate columns: + +```bash +clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ +--query \ +"select formatReadableQuantity(uniq(OFNS_DESC)) as cardinality_OFNS_DESC, + formatReadableQuantity(uniq(RPT_DT)) as cardinality_RPT_DT, + formatReadableQuantity(uniq(BORO_NM)) as cardinality_BORO_NM + FROM + file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames') + FORMAT PrettyCompact" +``` + +Result: +```response +┌─cardinality_OFNS_DESC─┬─cardinality_RPT_DT─┬─cardinality_BORO_NM─┐ +│ 60.00 │ 306.00 │ 6.00 │ +└───────────────────────┴────────────────────┴─────────────────────┘ +``` +Ordering by cardinality, the `ORDER BY` becomes: + +``` +ORDER BY ( BORO_NM, OFNS_DESC, RPT_DT ) +``` +:::note +The table below will use more easily read column names, the above names will be mapped to +``` +ORDER BY ( borough, offense_description, date_reported ) +``` +::: + +Putting together the changes to data types and the `ORDER BY` tuple gives this table structure: + +```sql +CREATE TABLE NYPD_Complaint ( + complaint_number String, + precinct UInt8, + borough LowCardinality(String), + complaint_begin DateTime64(0,'America/New_York'), + complaint_end DateTime64(0,'America/New_York'), + was_crime_completed String, + housing_authority String, + housing_level_code UInt32, + jurisdiction_code UInt8, + jurisdiction LowCardinality(String), + offense_code UInt8, + offense_level LowCardinality(String), + location_descriptor LowCardinality(String), + offense_description LowCardinality(String), + park_name LowCardinality(String), + patrol_borough LowCardinality(String), + PD_CD UInt16, + PD_DESC String, + location_type LowCardinality(String), + date_reported Date, + transit_station LowCardinality(String), + suspect_age_group LowCardinality(String), + suspect_race LowCardinality(String), + suspect_sex LowCardinality(String), + transit_district UInt8, + victim_age_group LowCardinality(String), + victim_race LowCardinality(String), + victim_sex LowCardinality(String), + NY_x_coordinate UInt32, + NY_y_coordinate UInt32, + Latitude Float64, + Longitude Float64 +) ENGINE = MergeTree + ORDER BY ( borough, offense_description, date_reported ) +``` + +### Finding the primary key of a table + +The ClickHouse `system` database, specifically `system.table` has all of the information about the table you +just created. This query shows the `ORDER BY` (sorting key), and the `PRIMARY KEY`: +```sql +SELECT + partition_key, + sorting_key, + primary_key, + table +FROM system.tables +WHERE table = 'NYPD_Complaint' +FORMAT Vertical +``` + +Response +```response +Query id: 6a5b10bf-9333-4090-b36e-c7f08b1d9e01 + +Row 1: +────── +partition_key: +sorting_key: borough, offense_description, date_reported +primary_key: borough, offense_description, date_reported +table: NYPD_Complaint + +1 row in set. Elapsed: 0.001 sec. +``` + +## Preprocess and Import Data {#preprocess-import-data} + +We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. + +### `clickhouse-local` arguments used + +:::tip +`table='input'` appears in the arguments to clickhouse-local below. clickhouse-local takes the provided input (`cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv`) and inserts the input into a table. By default the table is named `table`. In this guide the name of the table is set to `input` to make the data flow clearer. The final argument to clickhouse-local is a query that selects from the table (`FROM input`) which is then piped to `clickhouse-client` to populate the table `NYPD_Complaint`. +::: + +```sql +cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv \ + | clickhouse-local --table='input' --input-format='TSVWithNames' \ + --input_format_max_rows_to_read_for_schema_inference=2000 \ + --query " + WITH (CMPLNT_FR_DT || ' ' || CMPLNT_FR_TM) AS CMPLNT_START, + (CMPLNT_TO_DT || ' ' || CMPLNT_TO_TM) AS CMPLNT_END + SELECT + CMPLNT_NUM AS complaint_number, + ADDR_PCT_CD AS precinct, + BORO_NM AS borough, + parseDateTime64BestEffort(CMPLNT_START) AS complaint_begin, + parseDateTime64BestEffortOrNull(CMPLNT_END) AS complaint_end, + CRM_ATPT_CPTD_CD AS was_crime_completed, + HADEVELOPT AS housing_authority_development, + HOUSING_PSA AS housing_level_code, + JURISDICTION_CODE AS jurisdiction_code, + JURIS_DESC AS jurisdiction, + KY_CD AS offense_code, + LAW_CAT_CD AS offense_level, + LOC_OF_OCCUR_DESC AS location_descriptor, + OFNS_DESC AS offense_description, + PARKS_NM AS park_name, + PATROL_BORO AS patrol_borough, + PD_CD, + PD_DESC, + PREM_TYP_DESC AS location_type, + toDate(parseDateTimeBestEffort(RPT_DT)) AS date_reported, + STATION_NAME AS transit_station, + SUSP_AGE_GROUP AS suspect_age_group, + SUSP_RACE AS suspect_race, + SUSP_SEX AS suspect_sex, + TRANSIT_DISTRICT AS transit_district, + VIC_AGE_GROUP AS victim_age_group, + VIC_RACE AS victim_race, + VIC_SEX AS victim_sex, + X_COORD_CD AS NY_x_coordinate, + Y_COORD_CD AS NY_y_coordinate, + Latitude, + Longitude + FROM input" \ + | clickhouse-client --query='INSERT INTO NYPD_Complaint FORMAT TSV' +``` + +## Validate the Data {#validate-data} + +:::note +The dataset changes once or more per year, your counts may not match what is in this document. +::: + +Query: + +```sql +SELECT count() +FROM NYPD_Complaint +``` + +Result: + +```text +┌─count()─┐ +│ 208993 │ +└─────────┘ + +1 row in set. Elapsed: 0.001 sec. +``` + +The size of the dataset in ClickHouse is just 12% of the original TSV file, compare the size of the original TSV file with the size of the table: + +Query: + +```sql +SELECT formatReadableSize(total_bytes) +FROM system.tables +WHERE name = 'NYPD_Complaint' +``` + +Result: +```text +┌─formatReadableSize(total_bytes)─┐ +│ 8.63 MiB │ +└─────────────────────────────────┘ +``` + + +## Run Some Queries {#run-queries} + +### Query 1. Compare the number of complaints by month + +Query: + +```sql +SELECT + dateName('month', date_reported) AS month, + count() AS complaints, + bar(complaints, 0, 50000, 80) +FROM NYPD_Complaint +GROUP BY month +ORDER BY complaints DESC +``` + +Result: +```response +Query id: 7fbd4244-b32a-4acf-b1f3-c3aa198e74d9 + +┌─month─────┬─complaints─┬─bar(count(), 0, 50000, 80)───────────────────────────────┐ +│ March │ 34536 │ ███████████████████████████████████████████████████████▎ │ +│ May │ 34250 │ ██████████████████████████████████████████████████████▋ │ +│ April │ 32541 │ ████████████████████████████████████████████████████ │ +│ January │ 30806 │ █████████████████████████████████████████████████▎ │ +│ February │ 28118 │ ████████████████████████████████████████████▊ │ +│ November │ 7474 │ ███████████▊ │ +│ December │ 7223 │ ███████████▌ │ +│ October │ 7070 │ ███████████▎ │ +│ September │ 6910 │ ███████████ │ +│ August │ 6801 │ ██████████▊ │ +│ June │ 6779 │ ██████████▋ │ +│ July │ 6485 │ ██████████▍ │ +└───────────┴────────────┴──────────────────────────────────────────────────────────┘ + +12 rows in set. Elapsed: 0.006 sec. Processed 208.99 thousand rows, 417.99 KB (37.48 million rows/s., 74.96 MB/s.) +``` + +### Query 2. Compare total number of complaints by Borough + +Query: + +```sql +SELECT + borough, + count() AS complaints, + bar(complaints, 0, 125000, 60) +FROM NYPD_Complaint +GROUP BY borough +ORDER BY complaints DESC +``` + +Result: +```response +Query id: 8cdcdfd4-908f-4be0-99e3-265722a2ab8d + +┌─borough───────┬─complaints─┬─bar(count(), 0, 125000, 60)──┐ +│ BROOKLYN │ 57947 │ ███████████████████████████▋ │ +│ MANHATTAN │ 53025 │ █████████████████████████▍ │ +│ QUEENS │ 44875 │ █████████████████████▌ │ +│ BRONX │ 44260 │ █████████████████████▏ │ +│ STATEN ISLAND │ 8503 │ ████ │ +│ (null) │ 383 │ ▏ │ +└───────────────┴────────────┴──────────────────────────────┘ + +6 rows in set. Elapsed: 0.008 sec. Processed 208.99 thousand rows, 209.43 KB (27.14 million rows/s., 27.20 MB/s.) +``` + +## Next Steps + +[A Practical Introduction to Sparse Primary Indexes in ClickHouse](../../guides/improving-query-performance/sparse-primary-indexes/sparse-primary-indexes-intro.md) discusses the differences in ClickHouse indexing compared to traditional relational databases, how ClickHouse builds and uses a sparse primary index, and indexing best practices. From 478d2ad06a6c12e5bf5f87505be382da9844ef48 Mon Sep 17 00:00:00 2001 From: jthmath Date: Wed, 7 Sep 2022 10:23:52 +0800 Subject: [PATCH 423/582] fix ignore projection --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index d0b39684ba2..254bcd9f7f9 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1302,7 +1302,7 @@ private: *ctx->source_part->data_part_storage, it->name(), destination); hardlinked_files.insert(it->name()); } - else if (!endsWith(".tmp_proj", it->name())) // ignore projection tmp merge dir + else if (!endsWith(it->name(), ".tmp_proj")) // ignore projection tmp merge dir { // it's a projection part directory ctx->data_part_storage_builder->createProjection(destination); From de8f6bdce790df57dc89841620edd4ff1c08a9ef Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 7 Sep 2022 13:39:12 +0800 Subject: [PATCH 424/582] More safe Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 130 ++++++++++++++--------- src/Common/OpenTelemetryTraceContext.h | 2 +- 2 files changed, 82 insertions(+), 50 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 9efb278a670..9230e727eaf 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -88,7 +88,13 @@ void Span::addAttribute(std::exception_ptr e) noexcept SpanHolder::SpanHolder(std::string_view _operation_name) { - if (current_thread_trace_context.isTraceEnabled()) + if (!current_thread_trace_context.isTraceEnabled()) + { + return; + } + + /// Use try-catch to make sure the ctor is exception safe. + try { this->trace_id = current_thread_trace_context.trace_id; this->parent_span_id = current_thread_trace_context.span_id; @@ -97,9 +103,19 @@ SpanHolder::SpanHolder(std::string_view _operation_name) this->start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - // set current span id to this - current_thread_trace_context.span_id = this->span_id; + /// Add new intialization here } + catch (...) + { + tryLogCurrentException(__FUNCTION__); + + /// Clear related fields to make sure the span won't be recorded. + this->trace_id = UUID(); + return; + } + + /// Set current span as parent of other spans created later on this thread. + current_thread_trace_context.span_id = this->span_id; } void SpanHolder::finish() noexcept @@ -216,7 +232,7 @@ const TracingContextOnThread & CurrentContext() return current_thread_trace_context; } -void TracingContextOnThread::reset() +void TracingContextOnThread::reset() noexcept { this->trace_id = UUID(); this->span_id = 0; @@ -231,59 +247,75 @@ TracingContextHolder::TracingContextHolder( const Settings * settings_ptr, const std::weak_ptr & _span_log) { - if (current_thread_trace_context.isTraceEnabled()) + /// Use try-catch to make sure the ctor is exception safe. + /// If any exception is raised during the construction, the tracing is not enabled on current thread. + try { - /// - /// This is not the normal case, - /// it means that construction of current object is not at the start of current thread. - /// Usually this is due to: - /// 1. bad design - /// 2. right design but code changes so that original point where this object is constructing is not the new start execution of current thread - /// - /// In such case, we should use current context as parent of this new constructing object, - /// So this branch ensures this class can be instantiated multiple times on one same thread safely. - /// - this->is_context_owner = false; - this->root_span.trace_id = current_thread_trace_context.trace_id; - this->root_span.parent_span_id = current_thread_trace_context.span_id; + if (current_thread_trace_context.isTraceEnabled()) + { + /// + /// This is not the normal case, + /// it means that construction of current object is not at the start of current thread. + /// Usually this is due to: + /// 1. bad design + /// 2. right design but code changes so that original point where this object is constructing is not the new start execution of current thread + /// + /// In such case, we should use current context as parent of this new constructing object, + /// So this branch ensures this class can be instantiated multiple times on one same thread safely. + /// + this->is_context_owner = false; + this->root_span.trace_id = current_thread_trace_context.trace_id; + this->root_span.parent_span_id = current_thread_trace_context.span_id; + this->root_span.span_id = thread_local_rng(); + this->root_span.operation_name = _operation_name; + this->root_span.start_time_us + = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + /// Set the root span as parent of other spans created on current thread + current_thread_trace_context.span_id = this->root_span.span_id; + return; + } + + if (!_parent_trace_context.isTraceEnabled()) + { + if (settings_ptr == nullptr) + /// Skip tracing context initialization on current thread + return; + + // Start the trace with some configurable probability. + std::bernoulli_distribution should_start_trace{settings_ptr->opentelemetry_start_trace_probability}; + if (!should_start_trace(thread_local_rng)) + /// skip tracing context initialization on current thread + return; + + while (_parent_trace_context.trace_id == UUID()) + { + // Make sure the random generated trace_id is not 0 which is an invalid id. + _parent_trace_context.trace_id.toUnderType().items[0] = thread_local_rng(); //-V656 + _parent_trace_context.trace_id.toUnderType().items[1] = thread_local_rng(); //-V656 + } + _parent_trace_context.span_id = 0; + } + + this->root_span.trace_id = _parent_trace_context.trace_id; + this->root_span.parent_span_id = _parent_trace_context.span_id; this->root_span.span_id = thread_local_rng(); this->root_span.operation_name = _operation_name; this->root_span.start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - current_thread_trace_context.span_id = this->root_span.span_id; + /// Add new initialization here + } + catch(...) + { + tryLogCurrentException(__FUNCTION__); + + /// Clear related fields to make sure the tracing is not enabled. + this->root_span.trace_id = UUID(); return; } - if (!_parent_trace_context.isTraceEnabled()) - { - if (settings_ptr == nullptr) - /// skip tracing context initialization on current thread - return; - - // start the trace ourselves, with some configurable probability. - std::bernoulli_distribution should_start_trace{settings_ptr->opentelemetry_start_trace_probability}; - if (!should_start_trace(thread_local_rng)) - /// skip tracing context initialization on current thread - return; - - while (_parent_trace_context.trace_id == UUID()) - { - // make sure the random generated trace_id is not 0 which is an invalid id - _parent_trace_context.trace_id.toUnderType().items[0] = thread_local_rng(); //-V656 - _parent_trace_context.trace_id.toUnderType().items[1] = thread_local_rng(); //-V656 - } - _parent_trace_context.span_id = 0; - } - - this->root_span.trace_id = _parent_trace_context.trace_id; - this->root_span.parent_span_id = _parent_trace_context.span_id; - this->root_span.span_id = thread_local_rng(); - this->root_span.operation_name = _operation_name; - this->root_span.start_time_us - = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - - /// set up trace context on current thread + /// Set up trace context on current thread only when the root span is successfully intialized. current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; current_thread_trace_context.trace_flags = TRACE_FLAG_SAMPLED; @@ -313,7 +345,7 @@ TracingContextHolder::~TracingContextHolder() /// It's acceptable that the attribute is not recorded in case of any exception, /// so the exception is ignored to try to log the span. } - + this->root_span.finish_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 3964b5030fc..63136f8731d 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -74,7 +74,7 @@ struct TracingContextOnThread : TracingContext return *this; } - void reset(); + void reset() noexcept; /// Use weak_ptr instead of shared_ptr to hold a reference to the underlying system.opentelemetry_span_log table /// Since this object is kept on threads and passed across threads, a weak_ptr is more safe to prevent potential leak From fc05b05be3210667585fa490146bf659ede1d1f4 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 7 Sep 2022 15:14:43 +0800 Subject: [PATCH 425/582] Fix style and typo Signed-off-by: Frank Chen --- src/Common/OpenTelemetryTraceContext.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 9230e727eaf..7a1f94926d5 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -103,7 +103,7 @@ SpanHolder::SpanHolder(std::string_view _operation_name) this->start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - /// Add new intialization here + /// Add new initialization here } catch (...) { @@ -306,7 +306,7 @@ TracingContextHolder::TracingContextHolder( /// Add new initialization here } - catch(...) + catch (...) { tryLogCurrentException(__FUNCTION__); @@ -315,7 +315,7 @@ TracingContextHolder::TracingContextHolder( return; } - /// Set up trace context on current thread only when the root span is successfully intialized. + /// Set up trace context on current thread only when the root span is successfully initialized. current_thread_trace_context = _parent_trace_context; current_thread_trace_context.span_id = this->root_span.span_id; current_thread_trace_context.trace_flags = TRACE_FLAG_SAMPLED; From 497b65f41d94b9d6c4ac658e65e9c365c9bd6be8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 08:35:54 +0000 Subject: [PATCH 426/582] CheckLabels: Print message that 'can be tested' is missing --- tests/ci/run_check.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 0a269c07642..3e4657f66bb 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -87,14 +87,17 @@ def should_run_checks_for_pr(pr_info: PRInfo) -> Tuple[bool, str, str]: # Consider the labels and whether the user is trusted. print("Got labels", pr_info.labels) if FORCE_TESTS_LABEL in pr_info.labels: + print(f"Label '{FORCE_TESTS_LABEL}' set, forcing remaining checks") return True, f"Labeled '{FORCE_TESTS_LABEL}'", "pending" if DO_NOT_TEST_LABEL in pr_info.labels: + print(f"Label '{DO_NOT_TEST_LABEL}' set, skipping remaining checks") return False, f"Labeled '{DO_NOT_TEST_LABEL}'", "success" if CAN_BE_TESTED_LABEL not in pr_info.labels and not pr_is_by_trusted_user( pr_info.user_login, pr_info.user_orgs ): + print(f"PRs by untrusted users need the '{CAN_BE_TESTED_LABEL}' label - please contact a member of the core team") return False, "Needs 'can be tested' label", "failure" if OK_SKIP_LABELS.intersection(pr_info.labels): @@ -219,7 +222,7 @@ if __name__ == "__main__": elif SUBMODULE_CHANGED_LABEL in pr_info.labels: pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL) - print(f"change labels: add {pr_labels_to_add}, remove {pr_labels_to_remove}") + print(f"Change labels: add {pr_labels_to_add}, remove {pr_labels_to_remove}") if pr_labels_to_add: post_labels(gh, pr_info, pr_labels_to_add) From 01139d9d281d8199b08e2497e58486f319097a4e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 7 Sep 2022 08:51:02 +0000 Subject: [PATCH 427/582] Automatic style fix --- tests/ci/run_check.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 3e4657f66bb..39dbc938c8f 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -97,7 +97,9 @@ def should_run_checks_for_pr(pr_info: PRInfo) -> Tuple[bool, str, str]: if CAN_BE_TESTED_LABEL not in pr_info.labels and not pr_is_by_trusted_user( pr_info.user_login, pr_info.user_orgs ): - print(f"PRs by untrusted users need the '{CAN_BE_TESTED_LABEL}' label - please contact a member of the core team") + print( + f"PRs by untrusted users need the '{CAN_BE_TESTED_LABEL}' label - please contact a member of the core team" + ) return False, "Needs 'can be tested' label", "failure" if OK_SKIP_LABELS.intersection(pr_info.labels): From 81c98dadd2dd0c15afc2d23c8dc1875f409ec4f2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Sep 2022 11:01:06 +0200 Subject: [PATCH 428/582] Remove redundant change --- src/Storages/MergeTree/MergeTreeData.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8fe02359702..52046c2b158 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1902,7 +1902,6 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t { const auto settings = getSettings(); if (parts_to_remove.size() > 1 - && !settings->allow_remote_fs_zero_copy_replication && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold) { From c6b17bd7f957620779be907ad67e1a8acb2f3c12 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Sep 2022 09:14:03 +0000 Subject: [PATCH 429/582] Don't preprocess if storage finalized --- src/Coordination/KeeperServer.cpp | 4 +++- src/Coordination/KeeperStateMachine.cpp | 10 ++++++++-- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 11 ++++++++--- src/Coordination/KeeperStorage.h | 2 ++ 5 files changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d311dfbd679..d6a208cf826 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -617,7 +617,9 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ auto & entry_buf = entry->get_buf(); auto request_for_session = state_machine->parseRequest(entry_buf); request_for_session.zxid = next_zxid; - state_machine->preprocess(request_for_session); + if (!state_machine->preprocess(request_for_session)) + return nuraft::cb_func::ReturnCode::ReturnNull; + request_for_session.digest = state_machine->getNodesDigest(); entry = nuraft::cs_new(entry->get_term(), getZooKeeperLogEntry(request_for_session), entry->get_val_type()); break; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 3d6c80b5e41..c5a66ce29ca 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -191,12 +191,16 @@ KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer return request_for_session; } -void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) +bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) { if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) - return; + return true; std::lock_guard lock(storage_and_responses_lock); + + if (storage->isFinalized()) + return false; + try { storage->preprocessRequest( @@ -215,6 +219,8 @@ void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (keeper_context->digest_enabled && request_for_session.digest) assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); + + return true; } nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 9ddc4372d3b..fbd4fdc5ac2 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -33,7 +33,7 @@ public: static KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data); - void preprocess(const KeeperStorage::RequestForSession & request_for_session); + bool preprocess(const KeeperStorage::RequestForSession & request_for_session); nuraft::ptr pre_commit(uint64_t log_idx, nuraft::buffer & data) override; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 711f3874868..2b6ab2c8b39 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -516,7 +516,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const { - if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end()) + if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; return tryGetNodeFromStorage(path); @@ -524,7 +524,7 @@ std::shared_ptr KeeperStorage::UncommittedState::getNode(St Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const { - if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end()) + if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.acls; auto node_it = storage.container.find(path); @@ -1825,6 +1825,11 @@ void KeeperStorage::finalize() session_expiry_queue.clear(); } +bool KeeperStorage::isFinalized() const +{ + return finalized; +} + class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable { @@ -1892,7 +1897,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec if (!keeper_context->digest_enabled) return current_digest; - std::unordered_map> updated_nodes; + std::unordered_map> updated_nodes; for (const auto & delta : new_deltas) { diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 73714771bf3..a40cca8e778 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -425,6 +425,8 @@ public: void finalize(); + bool isFinalized() const; + /// Set of methods for creating snapshots /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. From 4cdf18cc81ec3d471836b70995f57d5c57630c58 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Sep 2022 09:20:47 +0000 Subject: [PATCH 430/582] Shutdown storage after RAFT --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d6a208cf826..8186ddd0c00 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -442,9 +442,9 @@ void KeeperServer::shutdownRaftServer() void KeeperServer::shutdown() { - state_machine->shutdownStorage(); state_manager->flushAndShutDownLogStore(); shutdownRaftServer(); + state_machine->shutdownStorage(); } namespace From a377a5f1e987ff9ce648fa4a250289464e51b5c0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Sep 2022 09:24:50 +0000 Subject: [PATCH 431/582] Change assert to LOGICAL_ERROR --- src/Coordination/KeeperStorage.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 2b6ab2c8b39..d5abb859961 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -830,7 +830,9 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) { auto ephemerals_it = ephemerals.find(session_id); - assert(ephemerals_it != ephemerals.end()); + if (ephemerals_it == ephemerals.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path"); + ephemerals_it->second.erase(path); if (ephemerals_it->second.empty()) ephemerals.erase(ephemerals_it); From 9c847ceec9efd890f594b5b85f5bb29128e7c265 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 7 Sep 2022 01:58:03 +0200 Subject: [PATCH 432/582] No hardlinks while making backup of MergeTree in atomic database. --- src/Backups/BackupEntryWrappedWith.h | 28 ++++++++++++++ .../MergeTree/DataPartStorageOnDisk.cpp | 37 +++++++++++++------ .../MergeTree/DataPartStorageOnDisk.h | 5 ++- src/Storages/MergeTree/IDataPartStorage.h | 5 ++- src/Storages/MergeTree/MergeTreeData.cpp | 26 +++++++++++-- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../test_concurrency.py | 6 ++- 7 files changed, 87 insertions(+), 22 deletions(-) create mode 100644 src/Backups/BackupEntryWrappedWith.h diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h new file mode 100644 index 00000000000..893a88db9fd --- /dev/null +++ b/src/Backups/BackupEntryWrappedWith.h @@ -0,0 +1,28 @@ +#pragma once + + +namespace DB +{ + +/// Wraps another backup entry and a value of any type. +template +class BackupEntryWrappedWith : public IBackupEntry +{ +public: + BackupEntryWrappedWith(BackupEntryPtr entry_, const T & custom_value_) : entry(entry_), custom_value(custom_value_) { } + BackupEntryWrappedWith(BackupEntryPtr entry_, T && custom_value_) : entry(entry_), custom_value(std::move(custom_value_)) { } + ~BackupEntryWrappedWith() override = default; + + UInt64 getSize() const override { return entry->getSize(); } + std::optional getChecksum() const override { return entry->getChecksum(); } + std::unique_ptr getReadBuffer() const override { return entry->getReadBuffer(); } + String getFilePath() const override { return entry->getFilePath(); } + DiskPtr tryGetDiskIfExists() const override { return entry->tryGetDiskIfExists(); } + DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); } + +private: + BackupEntryPtr entry; + T custom_value; +}; + +} diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 0154fd6e281..894eec12f0c 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -650,23 +650,31 @@ bool DataPartStorageOnDisk::shallParticipateInMerges(const IStoragePolicy & stor } void DataPartStorageOnDisk::backup( - TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, - BackupEntries & backup_entries) const + BackupEntries & backup_entries, + bool make_temporary_hard_links, + TemporaryFilesOnDisks * temp_dirs) const { fs::path part_path_on_disk = fs::path{root_path} / part_dir; fs::path part_path_in_backup = fs::path{path_in_backup} / part_dir; auto disk = volume->getDisk(); - auto temp_dir_it = temp_dirs.find(disk); - if (temp_dir_it == temp_dirs.end()) - temp_dir_it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/")).first; - auto temp_dir_owner = temp_dir_it->second; - fs::path temp_dir = temp_dir_owner->getPath(); - fs::path temp_part_dir = temp_dir / part_path_in_backup.relative_path(); - disk->createDirectories(temp_part_dir); + + fs::path temp_part_dir; + std::shared_ptr temp_dir_owner; + if (make_temporary_hard_links) + { + assert(temp_dirs); + auto temp_dir_it = temp_dirs->find(disk); + if (temp_dir_it == temp_dirs->end()) + temp_dir_it = temp_dirs->emplace(disk, std::make_shared(disk, "tmp/")).first; + temp_dir_owner = temp_dir_it->second; + fs::path temp_dir = temp_dir_owner->getPath(); + temp_part_dir = temp_dir / part_path_in_backup.relative_path(); + disk->createDirectories(temp_part_dir); + } /// For example, /// part_path_in_backup = /data/test/table/0_1_1_0 @@ -683,13 +691,18 @@ void DataPartStorageOnDisk::backup( continue; /// Skip *.proj files - they're actually directories and will be handled. String filepath_on_disk = part_path_on_disk / filepath; String filepath_in_backup = part_path_in_backup / filepath; - String hardlink_filepath = temp_part_dir / filepath; - disk->createHardLink(filepath_on_disk, hardlink_filepath); + if (make_temporary_hard_links) + { + String hardlink_filepath = temp_part_dir / filepath; + disk->createHardLink(filepath_on_disk, hardlink_filepath); + filepath_on_disk = hardlink_filepath; + } + UInt128 file_hash{checksum.file_hash.first, checksum.file_hash.second}; backup_entries.emplace_back( filepath_in_backup, - std::make_unique(disk, hardlink_filepath, checksum.file_size, file_hash, temp_dir_owner)); + std::make_unique(disk, filepath_on_disk, checksum.file_size, file_hash, temp_dir_owner)); } for (const auto & filepath : files_without_checksums) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 5b5cff8e636..f02ef26f811 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -89,11 +89,12 @@ public: bool shallParticipateInMerges(const IStoragePolicy &) const override; void backup( - TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, - BackupEntries & backup_entries) const override; + BackupEntries & backup_entries, + bool make_temporary_hard_links, + TemporaryFilesOnDisks * temp_dirs) const override; DataPartStoragePtr freeze( const std::string & to, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 946c1c5fd47..9da8a5eae03 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -177,11 +177,12 @@ public: /// Also creates a new tmp_dir for internal disk (if disk is mentioned the first time). using TemporaryFilesOnDisks = std::map>; virtual void backup( - TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, - BackupEntries & backup_entries) const = 0; + BackupEntries & backup_entries, + bool make_temporary_hard_links, + TemporaryFilesOnDisks * temp_dirs) const = 0; /// Creates hardlinks into 'to/dir_path' for every file in data part. /// Callback is called after hardlinks are created, but before 'delete-on-destroy.txt' marker is removed. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b7b68367e98..8b077e6fb13 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -4110,26 +4111,43 @@ void MergeTreeData::backupData(BackupEntriesCollector & backup_entries_collector backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup)); } -BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup) +BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup) const { BackupEntries backup_entries; std::map> temp_dirs; + /// Tables in atomic databases have UUID. When using atomic database we don't have to create hard links to make a backup, we can just + /// keep smart pointers to data parts instead. That's because the files of a data part are removed only by the destructor of the data part + /// and so keeping a smart pointer to a data part is enough to protect those files from deleting. + bool use_hard_links = !getStorageID().hasUUID(); + for (const auto & part : data_parts) { + BackupEntries new_backup_entries; + part->data_part_storage->backup( - temp_dirs, part->checksums, part->getFileNamesWithoutChecksums(), data_path_in_backup, backup_entries); + part->checksums, part->getFileNamesWithoutChecksums(), data_path_in_backup, new_backup_entries, use_hard_links, &temp_dirs); auto projection_parts = part->getProjectionParts(); for (const auto & [projection_name, projection_part] : projection_parts) { projection_part->data_part_storage->backup( - temp_dirs, projection_part->checksums, projection_part->getFileNamesWithoutChecksums(), fs::path{data_path_in_backup} / part->name, - backup_entries); + new_backup_entries, + use_hard_links, + &temp_dirs); } + + if (!use_hard_links) + { + /// Wrap backup entries with data parts in order to keep the data parts alive while the backup entries in use. + for (auto & [_, backup_entry] : new_backup_entries) + backup_entry = std::make_shared>(std::move(backup_entry), part); + } + + insertAtEnd(backup_entries, std::move(new_backup_entries)); } return backup_entries; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c91c7ba02a8..93f9e6157d8 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1243,7 +1243,7 @@ protected: bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space); /// Makes backup entries to backup the parts of this table. - static BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup); + BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup) const; class RestoredPartsHolder; diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index 2269ccda828..315c6b94507 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -175,7 +175,11 @@ def test_concurrent_backups_on_different_nodes(): @pytest.mark.parametrize( "db_engine, table_engine", - [("Replicated", "ReplicatedMergeTree"), ("Ordinary", "MergeTree")], + [ + ("Ordinary", "MergeTree"), + ("Atomic", "MergeTree"), + ("Replicated", "ReplicatedMergeTree"), + ], ) def test_create_or_drop_tables_during_backup(db_engine, table_engine): if db_engine == "Replicated": From bf8fed8be8cac07ef0cc0b56e05e3d6631976c27 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 09:45:07 +0000 Subject: [PATCH 433/582] Revert "fix: don't force-inline SSE3 code into generic code" This reverts commit d054ffd1109ab93ec366a64558c7c983cd23a11e. --- src/Functions/GatherUtils/sliceHasImplAnyAll.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/GatherUtils/sliceHasImplAnyAll.h b/src/Functions/GatherUtils/sliceHasImplAnyAll.h index 3ca8c6be9a7..21c80b742fd 100644 --- a/src/Functions/GatherUtils/sliceHasImplAnyAll.h +++ b/src/Functions/GatherUtils/sliceHasImplAnyAll.h @@ -419,7 +419,7 @@ DECLARE_SSE42_SPECIFIC_CODE ( // SSE4.2 Int64, UInt64 specialization template requires (std::is_same_v || std::is_same_v) -inline bool sliceHasImplAnyAllImplInt64( +inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, @@ -495,7 +495,7 @@ inline bool sliceHasImplAnyAllImplInt64( // SSE4.2 Int32, UInt32 specialization template requires (std::is_same_v || std::is_same_v) -inline bool sliceHasImplAnyAllImplInt32( +inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, @@ -580,7 +580,7 @@ inline bool sliceHasImplAnyAllImplInt32( // SSE4.2 Int16, UInt16 specialization template requires (std::is_same_v || std::is_same_v) -inline bool sliceHasImplAnyAllImplInt16( +inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, @@ -682,7 +682,7 @@ inline bool sliceHasImplAnyAllImplInt16( // SSE2 Int8, UInt8 specialization template requires (std::is_same_v || std::is_same_v) -inline bool sliceHasImplAnyAllImplInt8( +inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt8( const NumericArraySlice & first, const NumericArraySlice & second, const UInt8 * first_null_map, From b2ea21508dc356c9410c120cd694439b34b5a91b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 7 Sep 2022 12:04:05 +0200 Subject: [PATCH 434/582] Fix test --- .../queries/0_stateless/00965_shard_unresolvable_addresses.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql index 555e7a98380..5b763d2d853 100644 --- a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql +++ b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql @@ -2,7 +2,7 @@ SET prefer_localhost_replica = 1; -SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 198 } +SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 279 } SELECT count() FROM remote('127.0.0.1|localhos', system.one); -- Clear cache to avoid future errors in the logs From b716988991e85e99b08fb2b991ab5ffc45b680a2 Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 7 Sep 2022 15:53:19 +0800 Subject: [PATCH 435/582] Remove non-existed released trains --- docs/en/getting-started/install.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 0bdb956f0cb..1a17e63a274 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -59,7 +59,7 @@ clickhouse-client # or "clickhouse-client --password" if you set up a password. -You can replace `stable` with `lts` or `testing` to use different [release trains](../faq/operations/production.md) based on your needs. +You can replace `stable` with `lts` to use different [release kinds](../faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://packages.clickhouse.com/deb/pool/stable). @@ -106,7 +106,7 @@ clickhouse-client # or "clickhouse-client --password" if you set up a password. -If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). `prestable` is sometimes also available. +You can replace `stable` with `lts` to use different [release kinds](../faq/operations/production.md) based on your needs. Then run these commands to install packages: From c07f234f095f575f2fb6902b007c5b736061ec48 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 10:01:51 +0000 Subject: [PATCH 436/582] fix: disable ENABLE_MULTITARGET_CODE for SSE2 builds --- src/CMakeLists.txt | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fd8771c1529..3dc42746d67 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -42,6 +42,14 @@ endif () # See `src/Common/TargetSpecific.h` option(ENABLE_MULTITARGET_CODE "Enable platform-dependent code" ON) +if (NO_SSE3_OR_HIGHER) + # Optimized x86 code in DECLARE_*_SPECIFIC_CODE blocks (see `src/Common/TargetSpecific.h`) is sometimes marked FORCE_INLINE. As a + # result, its instruction set requirements (e.g. SSE4.2) leak into generic code. This is normally not a problem for standard x86 builds + # because generic code is compiled with SSE 4.2 anyways. But it breaks SSE2-only builds. Therefore disabling the multitarget code + # machinery and always use generic code. (The cleaner alternative is removing FORCE_INLINE but that impacts performance too much.) + set(ENABLE_MULTITARGET_CODE OFF) +endif() + if (ENABLE_MULTITARGET_CODE) add_definitions(-DENABLE_MULTITARGET_CODE=1) else() From 613c688331f9a1afea8f4c98ce3a6017348917f0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 7 Sep 2022 13:04:44 +0200 Subject: [PATCH 437/582] Add documentation for cache --- docs/en/operations/caches.md | 1 + docs/en/operations/storing-data.md | 110 +++++++++++++++++++++++++++++ 2 files changed, 111 insertions(+) diff --git a/docs/en/operations/caches.md b/docs/en/operations/caches.md index 910f57ec56b..3aeae7d1c9d 100644 --- a/docs/en/operations/caches.md +++ b/docs/en/operations/caches.md @@ -20,6 +20,7 @@ Additional cache types: - [Avro format](../interfaces/formats.md#data-format-avro) schemas cache. - [Dictionaries](../sql-reference/dictionaries/index.md) data cache. - Schema inference cache. +- [Filesystem cache](storing-data.md) over S3, Azure, Local and other disks. Indirectly used: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index fab78366892..19d5a9a1651 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -112,6 +112,116 @@ Example of disk configuration: ``` +## Using local cache {#using-local-cache} + +It is possible to configure cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. Cache uses `LRU` cache policy. + +Example of configuration for versions later or equal to 21.8: + +``` xml + + + + + s3 + ... + ... s3 configuration ... + + + cache + s3 + /s3_cache/ + 10000000 + + + +``` + +Example of configuration for versions earlier than 21.8: + +``` xml + + + + + s3 + ... + ... s3 configuration ... + 1 + 10000000 + + + +``` + +Cache configuration settings (the list corresponds to latest ClickHouse version, for earlier versions something might be unsupported): + +- `path` - path to cache Default: None, this settings is obligatory. + +- `max_size` - size of cache in bytes Default: None, this settings is obligatory. + +- `cache_on_write_operations` - turn on `write-through` cache. Default: `false`. The `write-through` cache is enabled if `cache_on_write_operations` is `true` and user setting `filesystem`. + +- `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. + +- `enable_cache_hits_threshold` - a number which defines the number of times some data needs to be read before it will be cached. Default: `0`, e.g. the data is cached at the first attempt to read it. + +- `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `true`. + +- `max_file_segment_size` - a max size for a single cache file. Default: `100 Mb`. + +- `max_elements` a limit for a number of cache files. + +Cache user settings (can be changes per query): + +- `enable_filesystem_cache` - allows to disable cache even if storage policy was configured with `cache` disk type. Default: `true`. + +- `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` - allows to use cache in query only if it already exists, otherwise cache will not be filled with the query data. Default: `false`. + +- `enable_filesystem_cache_on_write_operations` - turn on `write-through` cache. This setting works only if settings `cache_on_write_operations` in cache configuration is turned on. + +- `enable_filesystem_cache_log` - turn on writing to `system.filesystem_cache_log` table. Gives a detailed view of cache usage per query. Default: `false`. + +- `max_query_cache_size` - a limit for the cache size, which can be written to local cache storage. Requires enabled `enable_filesystem_query_cache_limit` in cache configuration. Default: `false`. + +- `skip_download_if_exceeds_query_cache` - allows to change the behaviour of setting `max_query_cache_size`. Default: `true`. If this setting is turned on and cache download limit during query was reached, no more cache will be downloaded to cache storage. If this setting is turned off and cache download limit during query was reached, cache will still be written by evicting previously written within current query cache data. E.g. second behaviour allows to preserve `last recentltly used` behaviour. + +Cache system tables: + +- `system.filesystem_cache` - system tables which shows current state of cache. + +- `system.filesystem_cache_log` - system table which shows detailed cache usage per query. Requires `enable_filesystem_cache_log` setting to be `true`. + +Cache commands: + +- `SYSTEM DROP FILESYSTEM CACHE (ON CLUSTER)` + +- `SHOW CACHES` -- show list of caches which were configured on the server. + +- `DESCRIBE CACHE ''` - show cache configuration and some general statistics for a specific cache. Cache name can be taken from `SHOW CACHES` command. + +Cache current metrics: + +- `FilesystemCacheSize` + +- `FilesystemCacheElements` + +Cache asynchronous metrics: + +- `FilesystemCacheBytes` + +- `FilesystemCacheFiles` + +Cache profile events: + +- `CachedReadBufferReadFromSourceBytes`, `CachedReadBufferReadFromCacheBytes,` + +- `CachedReadBufferReadFromSourceMicroseconds`, `CachedReadBufferReadFromCacheMicroseconds` + +- `CachedReadBufferCacheWriteBytes`, `CachedReadBufferCacheWriteMicroseconds` + +- `CachedWriteBufferCacheWriteBytes`, `CachedWriteBufferCacheWriteMicroseconds` + ## Storing Data on Web Server {#storing-data-on-webserver} There is a tool `clickhouse-static-files-uploader`, which prepares a data directory for a given table (`SELECT data_paths FROM system.tables WHERE name = 'table_name'`). For each table you need, you get a directory of files. These files can be uploaded to, for example, a web server with static files. After this preparation, you can load this table into any ClickHouse server via `DiskWeb`. From ade433797843cf38fa0dd4845bb91b57ca5fe14e Mon Sep 17 00:00:00 2001 From: Sachin Date: Mon, 8 Aug 2022 06:23:49 +0100 Subject: [PATCH 438/582] Add majority_insert_quorum setting majority_insert_quorum is defined as (number_of_replicas/2)+1. Insert will be successful only if majority of quorum have applied it. If insert_quorum and majority_insert_quorum both are specified, max of both will be used. --- docs/en/operations/settings/settings.md | 10 ++ src/Core/Settings.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 21 +-- .../MergeTree/ReplicatedMergeTreeSink.h | 9 +- src/Storages/StorageReplicatedMergeTree.cpp | 5 +- ...ity_insert_quorum_zookeeper_long.reference | 39 ++++++ ..._majority_insert_quorum_zookeeper_long.sql | 122 ++++++++++++++++++ 7 files changed, 196 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference create mode 100644 tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b4d44547328..ec426e68b62 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1196,6 +1196,16 @@ See also: - [insert_quorum_parallel](#settings-insert_quorum_parallel) - [select_sequential_consistency](#settings-select_sequential_consistency) +## majority_insert_quorum {#settings-majority_insert_quorum} + +Use majority number as quorum number. Majority is defined as (number_of_replicas/2) + 1. If insert_quorum and majority_insert_quorum both are specified, max(insert_quorum , majority number) will be used as quorum. + +Default value: false + +See also: + +- [insert_quorum](#settings-insert_quorum) + ## insert_quorum_timeout {#settings-insert_quorum_timeout} Write to a quorum timeout in milliseconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1a2b9e42a25..ec8b0785912 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -213,7 +213,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed", 0) \ \ - M(UInt64Auto, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ + M(UInt64Auto, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled, 'auto' - use majority", 0) \ M(Milliseconds, insert_quorum_timeout, 600000, "If the quorum of replicas did not meet in specified time (in milliseconds), exception will be thrown and insertion is aborted.", 0) \ M(Bool, insert_quorum_parallel, true, "For quorum INSERT queries - enable to make parallel inserts without linearizability", 0) \ M(UInt64, select_sequential_consistency, 0, "For SELECT queries from the replicated table, throw an exception if the replica does not have a chunk written with the quorum; do not read the parts that have not yet been written with the quorum.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index dc01df22886..0b067d01fbb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -52,6 +52,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( size_t max_parts_per_block_, bool quorum_parallel_, bool deduplicate_, + bool majority_quorum_, ContextPtr context_, bool is_attach_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) @@ -63,6 +64,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( , is_attach(is_attach_) , quorum_parallel(quorum_parallel_) , deduplicate(deduplicate_) + , majority_quorum(majority_quorum_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) , context(context_) , storage_snapshot(storage.getStorageSnapshot(metadata_snapshot, context)) @@ -86,7 +88,7 @@ static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) } -void ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) +void ReplicatedMergeTreeSink::setMajorityQuorumAndCheckQuorum(zkutil::ZooKeeperPtr & zookeeper) { quorum_info.status_path = storage.zookeeper_path + "/quorum/status"; @@ -105,6 +107,9 @@ void ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zoo if (status.get().error == Coordination::Error::ZOK) ++active_replicas; + if (majority_quorum) + quorum = std::max(quorum, replicas.size() / 2 + 1); + if (active_replicas < quorum) throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}).", active_replicas, quorum); @@ -148,8 +153,8 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) * And also check that during the insertion, the replica was not reinitialized or disabled (by the value of `is_active` node). * TODO Too complex logic, you can do better. */ - if (quorum) - checkQuorumPrecondition(zookeeper); + if (quorumEnabled()) + setMajorityQuorumAndCheckQuorum(zookeeper); storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); @@ -281,8 +286,8 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); - if (quorum) - checkQuorumPrecondition(zookeeper); + if (quorumEnabled()) + setMajorityQuorumAndCheckQuorum(zookeeper); Stopwatch watch; @@ -384,7 +389,7 @@ void ReplicatedMergeTreeSink::commitPart( * but for it the quorum has not yet been reached. * You can not do the next quorum record at this time.) */ - if (quorum) + if (quorumEnabled()) { ReplicatedMergeTreeQuorumEntry quorum_entry; quorum_entry.part_name = part->name; @@ -436,7 +441,7 @@ void ReplicatedMergeTreeSink::commitPart( { part->is_duplicate = true; ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); - if (quorum) + if (quorumEnabled()) { LOG_INFO(log, "Block with ID {} already exists locally as part {}; ignoring it, but checking quorum.", block_id, existing_part_name); @@ -593,7 +598,7 @@ void ReplicatedMergeTreeSink::commitPart( break; } - if (quorum) + if (quorumEnabled()) { if (is_already_existing_part) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 3a329b8d6c6..285af5bddd1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -32,6 +32,7 @@ public: size_t max_parts_per_block_, bool quorum_parallel_, bool deduplicate_, + bool majority_quorum_, ContextPtr context_, // special flag to determine the ALTER TABLE ATTACH PART without the query context, // needed to set the special LogEntryType::ATTACH_PART @@ -68,7 +69,8 @@ private: }; QuorumInfo quorum_info; - void checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); + /// set quorum if majority_quorum is true and checks active replicas + void setMajorityQuorumAndCheckQuorum(zkutil::ZooKeeperPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. void commitPart( @@ -93,6 +95,7 @@ private: bool quorum_parallel = false; const bool deduplicate = true; bool last_block_is_duplicate = false; + bool majority_quorum = false; using Logger = Poco::Logger; Poco::Logger * log; @@ -107,6 +110,10 @@ private: std::unique_ptr delayed_chunk; void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); + bool quorumEnabled() const + { + return majority_quorum || quorum != 0; + } }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4be97e01293..9bcea54b2ca 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4437,6 +4437,7 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con query_settings.max_partitions_per_insert_block, query_settings.insert_quorum_parallel, deduplicate, + query_settings.majority_insert_quorum, local_context); } @@ -5125,7 +5126,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition( MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. - ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, query_context, + ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, false, query_context, /*is_attach*/true); for (size_t i = 0; i < loaded_parts.size(); ++i) @@ -8394,7 +8395,7 @@ void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & rest void StorageReplicatedMergeTree::attachRestoredParts(MutableDataPartsVector && parts) { auto metadata_snapshot = getInMemoryMetadataPtr(); - auto sink = std::make_shared(*this, metadata_snapshot, 0, 0, 0, false, false, getContext(), /*is_attach*/true); + auto sink = std::make_shared(*this, metadata_snapshot, 0, 0, 0, false, false, false, getContext(), /*is_attach*/true); for (auto part : parts) sink->writeExistingPart(part); } diff --git a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference new file mode 100644 index 00000000000..39f11562863 --- /dev/null +++ b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference @@ -0,0 +1,39 @@ +1 +2 +3 +1 +2 +3 +majority_insert_quorum 1 1 Choose number_of_replicas/2 + 1 as insert_quorum. Note if insert_quorum and majority_insert_quorum both are specified max(insert_quorum, number_of_replicas/2+1) will be used \N \N 0 Bool +majority_insert_quorum 0 1 Choose number_of_replicas/2 + 1 as insert_quorum. Note if insert_quorum and majority_insert_quorum both are specified max(insert_quorum, number_of_replicas/2+1) will be used \N \N 0 Bool +majority_insert_quorum 1 1 Choose number_of_replicas/2 + 1 as insert_quorum. Note if insert_quorum and majority_insert_quorum both are specified max(insert_quorum, number_of_replicas/2+1) will be used \N \N 0 Bool +1 +1 +1 +1 +1 +1 +1 +2 +3 +1 +2 +3 +1 +2 +3 +11 +11 +11 +11 +11 +11 +11 +12 +13 +11 +12 +13 +11 +12 +13 diff --git a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql new file mode 100644 index 00000000000..22be1e54f6b --- /dev/null +++ b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql @@ -0,0 +1,122 @@ +-- Tags: long, zookeeper + +SET send_logs_level = 'fatal'; +SET insert_quorum_parallel = false; +SET select_sequential_consistency=1; + +DROP TABLE IF EXISTS quorum1; +DROP TABLE IF EXISTS quorum2; +DROP TABLE IF EXISTS quorum3; + +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum', '2') ORDER BY x PARTITION BY y; + +-- majority_insert_quorum = n/2 + 1 , so insert will be written to both replica +SET majority_insert_quorum = true; + +INSERT INTO quorum1 VALUES (1, '2018-11-15'); +INSERT INTO quorum1 VALUES (2, '2018-11-15'); +INSERT INTO quorum1 VALUES (3, '2018-12-16'); + +SELECT x FROM quorum1 ORDER BY x; +SELECT x FROM quorum2 ORDER BY x; + +DROP TABLE quorum1; +DROP TABLE quorum2; + +-- check majority_insert_quorum valid values +SET majority_insert_quorum = TrUe; +select * from system.settings where name like 'majority_insert_quorum'; +SET majority_insert_quorum = FalSE; +select * from system.settings where name like 'majority_insert_quorum'; +-- this is also allowed +SET majority_insert_quorum = 10; +select * from system.settings where name like 'majority_insert_quorum'; + +-- Create 3 replicas and stop sync 2 replicas +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum1', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum1', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum1', '3') ORDER BY x PARTITION BY y; + +SET majority_insert_quorum = true; + +-- Insert should be successful +-- stop replica 3 +SYSTEM STOP FETCHES quorum3; +INSERT INTO quorum1 VALUES (1, '2018-11-15'); +SELECT x FROM quorum1 ORDER BY x; +SELECT x FROM quorum2 ORDER BY x; +SELECT x FROM quorum3 ORDER BY x; -- {serverError 289} + +-- Sync replica 3 +SYSTEM START FETCHES quorum3; +SYSTEM SYNC REPLICA quorum3; +SELECT x FROM quorum3 ORDER BY x; + +-- Stop 2 replicas , so insert wont be successful +SYSTEM STOP FETCHES quorum2; +SYSTEM STOP FETCHES quorum3; +SET insert_quorum_timeout = 5000; +INSERT INTO quorum1 VALUES (2, '2018-11-15'); -- { serverError 319 } +SELECT x FROM quorum1 ORDER BY x; +SELECT x FROM quorum2 ORDER BY x; +SELECT x FROM quorum3 ORDER BY x; + +-- Sync replica 2 and 3 +SYSTEM START FETCHES quorum2; +SYSTEM SYNC REPLICA quorum2; +SYSTEM START FETCHES quorum3; +SYSTEM SYNC REPLICA quorum3; + +INSERT INTO quorum1 VALUES (3, '2018-11-15'); +SELECT x FROM quorum1 ORDER BY x; +SYSTEM SYNC REPLICA quorum2; +SYSTEM SYNC REPLICA quorum3; +SELECT x FROM quorum2 ORDER BY x; +SELECT x FROM quorum3 ORDER BY x; + +DROP TABLE quorum1; +DROP TABLE quorum2; +DROP TABLE quorum3; + +-- both insert_quorum and majority_insert_quorum are on, in that case max of both will be used as insert quorum +-- insert_quorum < n/2 +1 +SET majority_insert_quorum = true; +set insert_quorum = 1; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum2', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum2', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum2', '3') ORDER BY x PARTITION BY y; + +-- Insert should be successful +-- stop replica 3 +SYSTEM STOP FETCHES quorum3; +INSERT INTO quorum1 VALUES (11, '2018-11-15'); +SELECT x FROM quorum1 ORDER BY x; +SELECT x FROM quorum2 ORDER BY x; +SELECT x FROM quorum3 ORDER BY x; -- {serverError 289} + +-- Sync replica 3 +SYSTEM START FETCHES quorum3; +SYSTEM SYNC REPLICA quorum3; +SELECT x FROM quorum3 ORDER BY x; + +-- insert_quorum > n/2 +1 +set insert_quorum = 3; +SET majority_insert_quorum = false; +SYSTEM STOP FETCHES quorum3; +INSERT INTO quorum1 VALUES (12, '2018-11-15'); -- { serverError 319 } +SELECT x FROM quorum1 ORDER BY x; +SELECT x FROM quorum2 ORDER BY x; +SELECT x FROM quorum3 ORDER BY x; + +-- Sync replica 3 +SYSTEM START FETCHES quorum3; +SYSTEM SYNC REPLICA quorum3; +INSERT INTO quorum1 VALUES (13, '2018-11-15'); +SELECT x FROM quorum1 ORDER BY x; +SELECT x FROM quorum2 ORDER BY x; +SELECT x FROM quorum3 ORDER BY x; + +DROP TABLE quorum1; +DROP TABLE quorum2; +DROP TABLE quorum3; From 000ba8a60d426d2a7310113d1df8a2d3e7f180a3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Aug 2022 12:03:11 +0000 Subject: [PATCH 439/582] Use insert_qurum = auto instead of majority_insert_quorum --- docs/en/operations/settings/settings.md | 15 +---- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- ...ity_insert_quorum_zookeeper_long.reference | 18 ------ ..._majority_insert_quorum_zookeeper_long.sql | 57 +------------------ 4 files changed, 6 insertions(+), 87 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ec426e68b62..3869168becd 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1176,8 +1176,9 @@ Enables the quorum writes. - If `insert_quorum < 2`, the quorum writes are disabled. - If `insert_quorum >= 2`, the quorum writes are enabled. +- If `insert_quorum = 'auto'`, use majority number (`number_of_replicas / 2 + 1`) as quorum number. -Default value: 0. +Default value: 0 - disabled. Quorum writes @@ -1196,16 +1197,6 @@ See also: - [insert_quorum_parallel](#settings-insert_quorum_parallel) - [select_sequential_consistency](#settings-select_sequential_consistency) -## majority_insert_quorum {#settings-majority_insert_quorum} - -Use majority number as quorum number. Majority is defined as (number_of_replicas/2) + 1. If insert_quorum and majority_insert_quorum both are specified, max(insert_quorum , majority number) will be used as quorum. - -Default value: false - -See also: - -- [insert_quorum](#settings-insert_quorum) - ## insert_quorum_timeout {#settings-insert_quorum_timeout} Write to a quorum timeout in milliseconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. @@ -1269,7 +1260,7 @@ Possible values: Default value: 1. -By default, blocks inserted into replicated tables by the `INSERT` statement are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)). +By default, blocks inserted into replicated tables by the `INSERT` statement are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)). For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9bcea54b2ca..42aabedb3ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4430,14 +4430,13 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; // TODO: should we also somehow pass list of columns to deduplicate on to the ReplicatedMergeTreeSink? - // TODO: insert_quorum = 'auto' would be supported in https://github.com/ClickHouse/ClickHouse/pull/39970, now it's same as 0. return std::make_shared( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, query_settings.insert_quorum_parallel, deduplicate, - query_settings.majority_insert_quorum, + query_settings.insert_quorum.is_auto, local_context); } diff --git a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference index 39f11562863..579caf88d46 100644 --- a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference +++ b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.reference @@ -4,9 +4,6 @@ 1 2 3 -majority_insert_quorum 1 1 Choose number_of_replicas/2 + 1 as insert_quorum. Note if insert_quorum and majority_insert_quorum both are specified max(insert_quorum, number_of_replicas/2+1) will be used \N \N 0 Bool -majority_insert_quorum 0 1 Choose number_of_replicas/2 + 1 as insert_quorum. Note if insert_quorum and majority_insert_quorum both are specified max(insert_quorum, number_of_replicas/2+1) will be used \N \N 0 Bool -majority_insert_quorum 1 1 Choose number_of_replicas/2 + 1 as insert_quorum. Note if insert_quorum and majority_insert_quorum both are specified max(insert_quorum, number_of_replicas/2+1) will be used \N \N 0 Bool 1 1 1 @@ -22,18 +19,3 @@ majority_insert_quorum 1 1 Choose number_of_replicas/2 + 1 as insert_quorum. Not 1 2 3 -11 -11 -11 -11 -11 -11 -11 -12 -13 -11 -12 -13 -11 -12 -13 diff --git a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql index 22be1e54f6b..1f058b4d148 100644 --- a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql +++ b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql @@ -11,8 +11,8 @@ DROP TABLE IF EXISTS quorum3; CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum', '1') ORDER BY x PARTITION BY y; CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum', '2') ORDER BY x PARTITION BY y; --- majority_insert_quorum = n/2 + 1 , so insert will be written to both replica -SET majority_insert_quorum = true; +-- insert_quorum = n/2 + 1 , so insert will be written to both replica +SET insert_quorum = 'auto'; INSERT INTO quorum1 VALUES (1, '2018-11-15'); INSERT INTO quorum1 VALUES (2, '2018-11-15'); @@ -24,22 +24,11 @@ SELECT x FROM quorum2 ORDER BY x; DROP TABLE quorum1; DROP TABLE quorum2; --- check majority_insert_quorum valid values -SET majority_insert_quorum = TrUe; -select * from system.settings where name like 'majority_insert_quorum'; -SET majority_insert_quorum = FalSE; -select * from system.settings where name like 'majority_insert_quorum'; --- this is also allowed -SET majority_insert_quorum = 10; -select * from system.settings where name like 'majority_insert_quorum'; - -- Create 3 replicas and stop sync 2 replicas CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum1', '1') ORDER BY x PARTITION BY y; CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum1', '2') ORDER BY x PARTITION BY y; CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum1', '3') ORDER BY x PARTITION BY y; -SET majority_insert_quorum = true; - -- Insert should be successful -- stop replica 3 SYSTEM STOP FETCHES quorum3; @@ -78,45 +67,3 @@ SELECT x FROM quorum3 ORDER BY x; DROP TABLE quorum1; DROP TABLE quorum2; DROP TABLE quorum3; - --- both insert_quorum and majority_insert_quorum are on, in that case max of both will be used as insert quorum --- insert_quorum < n/2 +1 -SET majority_insert_quorum = true; -set insert_quorum = 1; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum2', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum2', '2') ORDER BY x PARTITION BY y; -CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02377/quorum2', '3') ORDER BY x PARTITION BY y; - --- Insert should be successful --- stop replica 3 -SYSTEM STOP FETCHES quorum3; -INSERT INTO quorum1 VALUES (11, '2018-11-15'); -SELECT x FROM quorum1 ORDER BY x; -SELECT x FROM quorum2 ORDER BY x; -SELECT x FROM quorum3 ORDER BY x; -- {serverError 289} - --- Sync replica 3 -SYSTEM START FETCHES quorum3; -SYSTEM SYNC REPLICA quorum3; -SELECT x FROM quorum3 ORDER BY x; - --- insert_quorum > n/2 +1 -set insert_quorum = 3; -SET majority_insert_quorum = false; -SYSTEM STOP FETCHES quorum3; -INSERT INTO quorum1 VALUES (12, '2018-11-15'); -- { serverError 319 } -SELECT x FROM quorum1 ORDER BY x; -SELECT x FROM quorum2 ORDER BY x; -SELECT x FROM quorum3 ORDER BY x; - --- Sync replica 3 -SYSTEM START FETCHES quorum3; -SYSTEM SYNC REPLICA quorum3; -INSERT INTO quorum1 VALUES (13, '2018-11-15'); -SELECT x FROM quorum1 ORDER BY x; -SELECT x FROM quorum2 ORDER BY x; -SELECT x FROM quorum3 ORDER BY x; - -DROP TABLE quorum1; -DROP TABLE quorum2; -DROP TABLE quorum3; From 6234d564d1c9940a635c20d4ad132ca414caae14 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 6 Sep 2022 12:09:03 +0000 Subject: [PATCH 440/582] Fix ReplicatedMergeTreeSink, add more info to logs --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 95 +++++++++++-------- .../MergeTree/ReplicatedMergeTreeSink.h | 24 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 18 ++-- 3 files changed, 80 insertions(+), 57 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 0b067d01fbb..6c7fbcb52d8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -41,42 +41,45 @@ struct ReplicatedMergeTreeSink::DelayedChunk String block_id; }; + DelayedChunk() = default; + explicit DelayedChunk(size_t replicas_num_) : replicas_num(replicas_num_) {} + + size_t replicas_num = 0; + std::vector partitions; }; ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( StorageReplicatedMergeTree & storage_, const StorageMetadataPtr & metadata_snapshot_, - size_t quorum_, + size_t quorum_size, size_t quorum_timeout_ms_, size_t max_parts_per_block_, bool quorum_parallel_, bool deduplicate_, - bool majority_quorum_, + bool majority_quorum, ContextPtr context_, bool is_attach_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) - , quorum(quorum_) + , required_quorum_size(majority_quorum ? std::nullopt : std::make_optional(quorum_size)) , quorum_timeout_ms(quorum_timeout_ms_) , max_parts_per_block(max_parts_per_block_) , is_attach(is_attach_) , quorum_parallel(quorum_parallel_) , deduplicate(deduplicate_) - , majority_quorum(majority_quorum_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) , context(context_) , storage_snapshot(storage.getStorageSnapshot(metadata_snapshot, context)) { /// The quorum value `1` has the same meaning as if it is disabled. - if (quorum == 1) - quorum = 0; + if (required_quorum_size == 1) + required_quorum_size = 0; } ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; - /// Allow to verify that the session in ZooKeeper is still alive. static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) { @@ -87,9 +90,11 @@ static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) throw Exception("ZooKeeper session has been expired.", ErrorCodes::NO_ZOOKEEPER); } - -void ReplicatedMergeTreeSink::setMajorityQuorumAndCheckQuorum(zkutil::ZooKeeperPtr & zookeeper) +size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) { + if (!isQuorumEnabled()) + return 0; + quorum_info.status_path = storage.zookeeper_path + "/quorum/status"; Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); @@ -107,12 +112,12 @@ void ReplicatedMergeTreeSink::setMajorityQuorumAndCheckQuorum(zkutil::ZooKeeperP if (status.get().error == Coordination::Error::ZOK) ++active_replicas; - if (majority_quorum) - quorum = std::max(quorum, replicas.size() / 2 + 1); + size_t replicas_number = replicas.size(); + size_t quorum_size = getQuorumSize(replicas_number); - if (active_replicas < quorum) - throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}).", - active_replicas, quorum); + if (active_replicas < quorum_size) + throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", + active_replicas, quorum_size, replicas_number); /** Is there a quorum for the last part for which a quorum is needed? * Write of all the parts with the included quorum is linearly ordered. @@ -138,8 +143,9 @@ void ReplicatedMergeTreeSink::setMajorityQuorumAndCheckQuorum(zkutil::ZooKeeperP quorum_info.is_active_node_value = is_active.data; quorum_info.is_active_node_version = is_active.stat.version; quorum_info.host_node_version = host.stat.version; -} + return replicas_number; +} void ReplicatedMergeTreeSink::consume(Chunk chunk) { @@ -153,8 +159,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) * And also check that during the insertion, the replica was not reinitialized or disabled (by the value of `is_active` node). * TODO Too complex logic, you can do better. */ - if (quorumEnabled()) - setMajorityQuorumAndCheckQuorum(zookeeper); + size_t replicas_num = checkQuorumPrecondition(zookeeper); storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); @@ -198,11 +203,11 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) } block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token); - LOG_DEBUG(log, "Wrote block with ID '{}', {} rows", block_id, current_block.block.rows()); + LOG_DEBUG(log, "Wrote block with ID '{}', {} rows on {} replicas", block_id, current_block.block.rows(), replicas_num); } else { - LOG_DEBUG(log, "Wrote block with {} rows", current_block.block.rows()); + LOG_DEBUG(log, "Wrote block with {} rows on {} replicas", current_block.block.rows(), replicas_num); } UInt64 elapsed_ns = watch.elapsed(); @@ -216,7 +221,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(zookeeper); - delayed_chunk = std::make_unique(); + delayed_chunk = std::make_unique(replicas_num); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(zookeeper); @@ -259,7 +264,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeepe try { - commitPart(zookeeper, part, partition.block_id, partition.temp_part.builder); + commitPart(zookeeper, part, partition.block_id, partition.temp_part.builder, delayed_chunk->replicas_num); last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; @@ -278,7 +283,6 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeepe delayed_chunk.reset(); } - void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) { /// NOTE: No delay in this case. That's Ok. @@ -286,15 +290,14 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); - if (quorumEnabled()) - setMajorityQuorumAndCheckQuorum(zookeeper); + size_t replicas_num = checkQuorumPrecondition(zookeeper); Stopwatch watch; try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - commitPart(zookeeper, part, "", part->data_part_storage->getBuilder()); + commitPart(zookeeper, part, "", part->data_part_storage->getBuilder(), replicas_num); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) @@ -304,12 +307,12 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt } } - void ReplicatedMergeTreeSink::commitPart( zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - DataPartStorageBuilderPtr builder) + DataPartStorageBuilderPtr builder, + size_t replicas_num) { metadata_snapshot->check(part->getColumns()); assertSessionIsNotExpired(zookeeper); @@ -372,7 +375,7 @@ void ReplicatedMergeTreeSink::commitPart( log_entry.source_replica = storage.replica_name; log_entry.new_part_name = part->name; /// TODO maybe add UUID here as well? - log_entry.quorum = quorum; + log_entry.quorum = getQuorumSize(replicas_num); log_entry.block_id = block_id; log_entry.new_part_type = part->getType(); @@ -389,11 +392,11 @@ void ReplicatedMergeTreeSink::commitPart( * but for it the quorum has not yet been reached. * You can not do the next quorum record at this time.) */ - if (quorumEnabled()) + if (isQuorumEnabled()) { ReplicatedMergeTreeQuorumEntry quorum_entry; quorum_entry.part_name = part->name; - quorum_entry.required_number_of_replicas = quorum; + quorum_entry.required_number_of_replicas = getQuorumSize(replicas_num); quorum_entry.replicas.insert(storage.replica_name); /** At this point, this node will contain information that the current replica received a part. @@ -441,7 +444,7 @@ void ReplicatedMergeTreeSink::commitPart( { part->is_duplicate = true; ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); - if (quorumEnabled()) + if (isQuorumEnabled()) { LOG_INFO(log, "Block with ID {} already exists locally as part {}; ignoring it, but checking quorum.", block_id, existing_part_name); @@ -451,7 +454,7 @@ void ReplicatedMergeTreeSink::commitPart( else quorum_path = storage.zookeeper_path + "/quorum/status"; - waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_value); + waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_value, replicas_num); } else { @@ -598,7 +601,7 @@ void ReplicatedMergeTreeSink::commitPart( break; } - if (quorumEnabled()) + if (isQuorumEnabled()) { if (is_already_existing_part) { @@ -610,7 +613,7 @@ void ReplicatedMergeTreeSink::commitPart( storage.updateQuorum(part->name, false); } - waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value); + waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value, replicas_num); } } @@ -632,10 +635,11 @@ void ReplicatedMergeTreeSink::waitForQuorum( zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, - const std::string & is_active_node_value) const + const std::string & is_active_node_value, + size_t replicas_num) const { /// We are waiting for quorum to be satisfied. - LOG_TRACE(log, "Waiting for quorum"); + LOG_TRACE(log, "Waiting for quorum '{}' for part {} on {} replicas", quorum_path, part_name, replicas_num); try { @@ -659,7 +663,7 @@ void ReplicatedMergeTreeSink::waitForQuorum( if (!event->tryWait(quorum_timeout_ms)) throw Exception("Timeout while waiting for quorum", ErrorCodes::TIMEOUT_EXCEEDED); - LOG_TRACE(log, "Quorum {} updated, will check quorum node still exists", quorum_path); + LOG_TRACE(log, "Quorum {} for part {} updated, will check quorum node still exists", quorum_path, part_name); } /// And what if it is possible that the current replica at this time has ceased to be active @@ -677,8 +681,23 @@ void ReplicatedMergeTreeSink::waitForQuorum( ErrorCodes::UNKNOWN_STATUS_OF_INSERT); } - LOG_TRACE(log, "Quorum satisfied"); + LOG_TRACE(log, "Quorum '{}' for part {} satisfied", quorum_path, part_name); } +size_t ReplicatedMergeTreeSink::getQuorumSize(size_t replicas_num) const +{ + if (!isQuorumEnabled()) + return 0; + + if (required_quorum_size) + return required_quorum_size.value(); + + return replicas_num / 2 + 1; +} + +bool ReplicatedMergeTreeSink::isQuorumEnabled() const +{ + return !required_quorum_size.has_value() || required_quorum_size.value() > 1; +} } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 285af5bddd1..48e94ef5659 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -69,25 +69,34 @@ private: }; QuorumInfo quorum_info; - /// set quorum if majority_quorum is true and checks active replicas - void setMajorityQuorumAndCheckQuorum(zkutil::ZooKeeperPtr & zookeeper); + + /// Checks active replicas. + /// Returns total number of replicas. + size_t checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. void commitPart( zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - DataPartStorageBuilderPtr part_builder); + DataPartStorageBuilderPtr part_builder, + size_t replicas_num); /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) /// Also checks that replica still alive. void waitForQuorum( zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, - const std::string & quorum_path, const std::string & is_active_node_value) const; + const std::string & quorum_path, const std::string & is_active_node_value, size_t replicas_num) const; StorageReplicatedMergeTree & storage; StorageMetadataPtr metadata_snapshot; - size_t quorum; + + /// Empty means use majority quorum. + std::optional required_quorum_size; + + size_t getQuorumSize(size_t replicas_num) const; + bool isQuorumEnabled() const; + size_t quorum_timeout_ms; size_t max_parts_per_block; @@ -95,7 +104,6 @@ private: bool quorum_parallel = false; const bool deduplicate = true; bool last_block_is_duplicate = false; - bool majority_quorum = false; using Logger = Poco::Logger; Poco::Logger * log; @@ -110,10 +118,6 @@ private: std::unique_ptr delayed_chunk; void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); - bool quorumEnabled() const - { - return majority_quorum || quorum != 0; - } }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 42aabedb3ff..b392a0a39d7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3641,8 +3641,8 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ if (quorum_entry.replicas.size() >= quorum_entry.required_number_of_replicas) { /// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum. - LOG_TRACE(log, "Got {} replicas confirmed quorum {}, going to remove node", - quorum_entry.replicas.size(), quorum_status_path); + LOG_TRACE(log, "Got {} (of {}) replicas confirmed quorum {}, going to remove node", + quorum_entry.replicas.size(), quorum_entry.required_number_of_replicas, quorum_status_path); Coordination::Requests ops; Coordination::Responses responses; @@ -3690,8 +3690,8 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ } else { - LOG_TRACE(log, "Quorum {} still not satisfied (have only {} replicas), updating node", - quorum_status_path, quorum_entry.replicas.size()); + LOG_TRACE(log, "Quorum {} still not satisfied (have only {} of {} replicas), updating node", + quorum_status_path, quorum_entry.replicas.size(), quorum_entry.required_number_of_replicas); /// We update the node, registering there one more replica. auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version); @@ -4304,12 +4304,12 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg auto added_parts = part_with_quorum.added_parts; for (const auto & added_part : added_parts) + { if (!getActiveContainingPart(added_part.second)) - throw Exception( - "Replica doesn't have part " + added_part.second - + " which was successfully written to quorum of other replicas." - " Send query to another replica or disable 'select_sequential_consistency' setting.", - ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); + throw Exception(ErrorCodes::REPLICA_IS_NOT_IN_QUORUM, + "Replica doesn't have part '{}' which was successfully written to quorum of other replicas. " + "Send query to another replica or disable 'select_sequential_consistency' setting", added_part.second); + } for (const auto & max_block : part_with_quorum.getMaxInsertedBlocks()) max_added_blocks[max_block.first] = max_block.second; From ef53741d2362c60ae311759535f03132ac744f44 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Sep 2022 11:22:18 +0000 Subject: [PATCH 441/582] Disable 02377_majority_insert_quorum_zookeeper for replicated-database --- .../02377_majority_insert_quorum_zookeeper_long.sql | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql index 1f058b4d148..803607f526e 100644 --- a/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql +++ b/tests/queries/0_stateless/02377_majority_insert_quorum_zookeeper_long.sql @@ -1,8 +1,11 @@ --- Tags: long, zookeeper +-- Tags: long, zookeeper, no-replicated-database + +-- no-replicated-database: +-- The number of replicas is doubled, so `SYSTEM STOP FETCHES` stop not enough replicas. -SET send_logs_level = 'fatal'; SET insert_quorum_parallel = false; -SET select_sequential_consistency=1; + +SET select_sequential_consistency = 1; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; From d1d6e5cb23c66cea0c391e5034714fceb614472a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 11:56:09 +0000 Subject: [PATCH 442/582] fix: build ragel assets for amd64 and aarch64 separately --- contrib/vectorscan-cmake/CMakeLists.txt | 27 +- .../rageled_files/{ => aarch64}/Parser.cpp | 2598 ++-- .../{ => aarch64}/control_verbs.cpp | 60 +- .../rageled_files/amd64/Parser.cpp | 10725 ++++++++++++++++ .../rageled_files/amd64/control_verbs.cpp | 547 + 5 files changed, 12615 insertions(+), 1342 deletions(-) rename contrib/vectorscan-cmake/rageled_files/{ => aarch64}/Parser.cpp (70%) rename contrib/vectorscan-cmake/rageled_files/{ => aarch64}/control_verbs.cpp (83%) create mode 100644 contrib/vectorscan-cmake/rageled_files/amd64/Parser.cpp create mode 100644 contrib/vectorscan-cmake/rageled_files/amd64/control_verbs.cpp diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index bf6f82eadc0..5596fdd23fd 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -4,8 +4,6 @@ if (ARCH_AMD64 OR ARCH_AARCH64) option (ENABLE_VECTORSCAN "Enable vectorscan library" ${ENABLE_LIBRARIES}) endif() -# TODO: vectorscan supports ARM yet some tests involving cyrillic letters fail (PR #38171) ... needs further investigation - # TODO PPC should generally work but needs manual generation of ppc/config.h file on a PPC machine if (NOT ENABLE_VECTORSCAN) @@ -236,17 +234,20 @@ set (SRCS # The original build system invokes ragel on src/parser/{Parser|control_verbs}.rl (+ a few more .rl files which are unneeded). To avoid a # build-time dependency on ragel (via contrib/ or find_program()), add the manually generated output of ragel to the sources. -# Please regenerate these files if you update vectorscan. -list (APPEND SRCS - "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/Parser.cpp" - "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/control_verbs.cpp" -) - -set_source_files_properties( - "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/Parser.cpp" - "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/control_verbs.cpp" - COMPILE_FLAGS -Wno-c++11-narrowing -) +# +# Please regenerate these files if you update vectorscan. They must be regenerated for each platform separately because ragel produces for +# weird reasons different constants in the output. +if (ARCH_AMD64) + list (APPEND SRCS + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/amd64/Parser.cpp" + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/amd64/control_verbs.cpp" + ) +elseif (ARCH_AARCH64) + list (APPEND SRCS + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/aarch64/Parser.cpp" + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/aarch64/control_verbs.cpp" + ) +endif() # Platform-dependent files if (ARCH_AMD64) diff --git a/contrib/vectorscan-cmake/rageled_files/Parser.cpp b/contrib/vectorscan-cmake/rageled_files/aarch64/Parser.cpp similarity index 70% rename from contrib/vectorscan-cmake/rageled_files/Parser.cpp rename to contrib/vectorscan-cmake/rageled_files/aarch64/Parser.cpp index b956d8f5801..fd58ead4128 100644 --- a/contrib/vectorscan-cmake/rageled_files/Parser.cpp +++ b/contrib/vectorscan-cmake/rageled_files/aarch64/Parser.cpp @@ -1,4 +1,4 @@ -#line 1 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1 "Parser.rl" /* * Copyright (c) 2015-2017, Intel Corporation * @@ -272,11 +272,11 @@ namespace ue2 { } -#line 1909 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1909 "Parser.rl" -#line 277 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 277 "Parser.cpp" static const short _regex_actions[] = { 0, 1, 0, 1, 1, 1, 2, 1, 3, 1, 4, 1, 7, 1, 8, 1, @@ -4116,7 +4116,7 @@ static const short _regex_actions[] = { static const int regex_en_main = 746; -#line 1911 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1911 "Parser.rl" /** \brief Main parser call, returns root Component or nullptr. */ @@ -4189,7 +4189,7 @@ static const short _regex_actions[] = { try { // Embed the Ragel machine here -#line 4187 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4187 "Parser.cpp" { cs = (int)regex_start; top = 0; @@ -4198,10 +4198,10 @@ static const short _regex_actions[] = { act = 0; } -#line 1982 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1982 "Parser.rl" -#line 4196 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4196 "Parser.cpp" { int _cpc; int _klen; @@ -4224,7 +4224,7 @@ static const short _regex_actions[] = { #line 1 "NONE" {ts = p;}} -#line 4218 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4218 "Parser.cpp" break; } @@ -4300,19 +4300,19 @@ static const short _regex_actions[] = { case 0: { if ( ( mode.utf8 ) -#line 4293 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4293 "Parser.cpp" ) _cpc += 1; break; } case 1: { if ( ( mode.ignore_space ) -#line 4298 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4298 "Parser.cpp" ) _cpc += 1; break; } case 2: { if ( ( inCharClassEarly ) -#line 4303 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4303 "Parser.cpp" ) _cpc += 1; break; } @@ -4352,159 +4352,159 @@ static const short _regex_actions[] = { { case 0: { { -#line 285 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 285 "Parser.rl" label.clear();} -#line 4345 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4345 "Parser.cpp" break; } case 1: { { -#line 286 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 286 "Parser.rl" label.push_back((( (*( p)))));} -#line 4353 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4353 "Parser.cpp" break; } case 2: { { -#line 287 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 287 "Parser.rl" octAccumulator = 0;} -#line 4361 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4361 "Parser.cpp" break; } case 3: { { -#line 288 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 288 "Parser.rl" accumulator = 0;} -#line 4369 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4369 "Parser.cpp" break; } case 4: { { -#line 289 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 289 "Parser.rl" octAccumulator = 0; pushOct(&octAccumulator, (( (*( p))))); } -#line 4380 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4380 "Parser.cpp" break; } case 5: { { -#line 293 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 293 "Parser.rl" accumulator = 0; pushDec(&accumulator, (( (*( p))))); } -#line 4391 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4391 "Parser.cpp" break; } case 6: { { -#line 297 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 297 "Parser.rl" repeatN = 0; repeatM = 0; } -#line 4399 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4399 "Parser.cpp" break; } case 7: { { -#line 298 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 298 "Parser.rl" pushDec(&repeatN, (( (*( p))))); } -#line 4407 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4407 "Parser.cpp" break; } case 8: { { -#line 299 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 299 "Parser.rl" pushDec(&repeatM, (( (*( p))))); } -#line 4415 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4415 "Parser.cpp" break; } case 9: { { -#line 300 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 300 "Parser.rl" pushOct(&octAccumulator, (( (*( p))))); } -#line 4423 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4423 "Parser.cpp" break; } case 10: { { -#line 301 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 301 "Parser.rl" pushDec(&accumulator, (( (*( p))))); } -#line 4431 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4431 "Parser.cpp" break; } case 11: { { -#line 302 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 302 "Parser.rl" accumulator *= 16; accumulator += (( (*( p)))) - '0'; } -#line 4442 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4442 "Parser.cpp" break; } case 12: { { -#line 306 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 306 "Parser.rl" accumulator *= 16; accumulator += 10 + (( (*( p)))) - 'a'; } -#line 4453 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4453 "Parser.cpp" break; } case 13: { { -#line 310 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 310 "Parser.rl" accumulator *= 16; accumulator += 10 + (( (*( p)))) - 'A'; } -#line 4464 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4464 "Parser.cpp" break; } case 14: { { -#line 430 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 430 "Parser.rl" newMode = mode; } -#line 4474 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4474 "Parser.cpp" break; } case 15: { { -#line 437 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 437 "Parser.rl" switch ((( (*( p))))) { case 'i': @@ -4525,13 +4525,13 @@ static const short _regex_actions[] = { } } -#line 4500 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4500 "Parser.cpp" break; } case 16: { { -#line 456 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 456 "Parser.rl" switch ((( (*( p))))) { case 'i': @@ -4552,42 +4552,42 @@ static const short _regex_actions[] = { } } -#line 4526 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4526 "Parser.cpp" break; } case 17: { { -#line 510 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 510 "Parser.rl" repeatM = repeatN;} -#line 4534 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4534 "Parser.cpp" break; } case 18: { { -#line 510 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 510 "Parser.rl" repeatM = ComponentRepeat::NoLimit;} -#line 4542 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4542 "Parser.cpp" break; } case 19: { { -#line 722 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 722 "Parser.rl" negated = !negated; } -#line 4550 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4550 "Parser.cpp" break; } case 20: { { -#line 723 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 723 "Parser.rl" {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -4596,13 +4596,13 @@ static const short _regex_actions[] = { } stack[top] = cs; top += 1;cs = 790;goto _again;}} -#line 4566 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4566 "Parser.cpp" break; } case 21: { { -#line 724 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 724 "Parser.rl" if (!inCharClass) { // not inside [..] currentCls->finalize(); currentSeq->addComponent(move(currentCls)); @@ -4610,16 +4610,16 @@ static const short _regex_actions[] = { {top -= 1;cs = stack[top];goto _again;} } -#line 4579 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4579 "Parser.cpp" break; } case 22: { { -#line 730 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 730 "Parser.rl" throw LocatedParseError("Malformed property"); } -#line 4587 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4587 "Parser.cpp" break; } @@ -4628,60 +4628,60 @@ static const short _regex_actions[] = { #line 1 "NONE" {te = p+1;}} -#line 4595 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4595 "Parser.cpp" break; } case 26: { { -#line 550 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 550 "Parser.rl" {te = p+1;{ -#line 550 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 550 "Parser.rl" throw LocatedParseError("(*UTF8) must be at start of " "expression, encountered"); } }} -#line 4609 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4609 "Parser.cpp" break; } case 27: { { -#line 554 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 554 "Parser.rl" {te = p+1;{ -#line 554 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 554 "Parser.rl" throw LocatedParseError("(*UTF) must be at start of " "expression, encountered"); } }} -#line 4623 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4623 "Parser.cpp" break; } case 28: { { -#line 558 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 558 "Parser.rl" {te = p+1;{ -#line 558 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 558 "Parser.rl" throw LocatedParseError("(*UCP) must be at start of " "expression, encountered"); } }} -#line 4637 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4637 "Parser.cpp" break; } case 29: { { -#line 564 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 564 "Parser.rl" {te = p+1;{ -#line 564 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 564 "Parser.rl" ParseMode temp_mode; assert(ts - 2 >= ptr); // parser needs the '(*' at the start too. @@ -4691,1868 +4691,1868 @@ static const short _regex_actions[] = { } }} -#line 4654 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4654 "Parser.cpp" break; } case 30: { { -#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 571 "Parser.rl" {te = p+1;{ -#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 571 "Parser.rl" throw LocatedParseError("Unknown control verb"); } }} -#line 4667 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4667 "Parser.cpp" break; } case 31: { { -#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 571 "Parser.rl" {te = p;p = p - 1;{ -#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 571 "Parser.rl" throw LocatedParseError("Unknown control verb"); } }} -#line 4680 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4680 "Parser.cpp" break; } case 32: { { -#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 571 "Parser.rl" {p = ((te))-1; { -#line 571 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 571 "Parser.rl" throw LocatedParseError("Unknown control verb"); } }} -#line 4694 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4694 "Parser.cpp" break; } case 33: { { -#line 581 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 581 "Parser.rl" {te = p+1;{ -#line 581 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 581 "Parser.rl" currentCls->add(CLASS_UCP_CC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4705 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4705 "Parser.cpp" break; } case 34: { { -#line 582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 582 "Parser.rl" {te = p+1;{ -#line 582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 582 "Parser.rl" currentCls->add(CLASS_UCP_CF, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4716 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4716 "Parser.cpp" break; } case 35: { { -#line 583 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 583 "Parser.rl" {te = p+1;{ -#line 583 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 583 "Parser.rl" currentCls->add(CLASS_UCP_CN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4727 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4727 "Parser.cpp" break; } case 36: { { -#line 585 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 585 "Parser.rl" {te = p+1;{ -#line 585 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 585 "Parser.rl" currentCls->add(CLASS_UCP_CS, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4738 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4738 "Parser.cpp" break; } case 37: { { -#line 587 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 587 "Parser.rl" {te = p+1;{ -#line 587 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 587 "Parser.rl" currentCls->add(CLASS_UCP_LL, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4749 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4749 "Parser.cpp" break; } case 38: { { -#line 588 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 588 "Parser.rl" {te = p+1;{ -#line 588 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 588 "Parser.rl" currentCls->add(CLASS_UCP_LM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4760 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4760 "Parser.cpp" break; } case 39: { { -#line 589 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 589 "Parser.rl" {te = p+1;{ -#line 589 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 589 "Parser.rl" currentCls->add(CLASS_UCP_LO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4771 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4771 "Parser.cpp" break; } case 40: { { -#line 590 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 590 "Parser.rl" {te = p+1;{ -#line 590 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 590 "Parser.rl" currentCls->add(CLASS_UCP_LT, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4782 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4782 "Parser.cpp" break; } case 41: { { -#line 591 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 591 "Parser.rl" {te = p+1;{ -#line 591 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 591 "Parser.rl" currentCls->add(CLASS_UCP_LU, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4793 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4793 "Parser.cpp" break; } case 42: { { -#line 592 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 592 "Parser.rl" {te = p+1;{ -#line 592 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 592 "Parser.rl" currentCls->add(CLASS_UCP_L_AND, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4804 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4804 "Parser.cpp" break; } case 43: { { -#line 594 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 594 "Parser.rl" {te = p+1;{ -#line 594 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 594 "Parser.rl" currentCls->add(CLASS_UCP_MC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4815 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4815 "Parser.cpp" break; } case 44: { { -#line 596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 596 "Parser.rl" {te = p+1;{ -#line 596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 596 "Parser.rl" currentCls->add(CLASS_UCP_MN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4826 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4826 "Parser.cpp" break; } case 45: { { -#line 598 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 598 "Parser.rl" {te = p+1;{ -#line 598 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 598 "Parser.rl" currentCls->add(CLASS_UCP_ND, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4837 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4837 "Parser.cpp" break; } case 46: { { -#line 599 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 599 "Parser.rl" {te = p+1;{ -#line 599 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 599 "Parser.rl" currentCls->add(CLASS_UCP_NL, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4848 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4848 "Parser.cpp" break; } case 47: { { -#line 600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 600 "Parser.rl" {te = p+1;{ -#line 600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 600 "Parser.rl" currentCls->add(CLASS_UCP_NO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4859 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4859 "Parser.cpp" break; } case 48: { { -#line 602 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 602 "Parser.rl" {te = p+1;{ -#line 602 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 602 "Parser.rl" currentCls->add(CLASS_UCP_PC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4870 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4870 "Parser.cpp" break; } case 49: { { -#line 603 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 603 "Parser.rl" {te = p+1;{ -#line 603 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 603 "Parser.rl" currentCls->add(CLASS_UCP_PD, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4881 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4881 "Parser.cpp" break; } case 50: { { -#line 604 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 604 "Parser.rl" {te = p+1;{ -#line 604 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 604 "Parser.rl" currentCls->add(CLASS_UCP_PE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4892 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4892 "Parser.cpp" break; } case 51: { { -#line 605 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 605 "Parser.rl" {te = p+1;{ -#line 605 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 605 "Parser.rl" currentCls->add(CLASS_UCP_PF, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4903 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4903 "Parser.cpp" break; } case 52: { { -#line 606 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 606 "Parser.rl" {te = p+1;{ -#line 606 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 606 "Parser.rl" currentCls->add(CLASS_UCP_PI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4914 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4914 "Parser.cpp" break; } case 53: { { -#line 607 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 607 "Parser.rl" {te = p+1;{ -#line 607 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 607 "Parser.rl" currentCls->add(CLASS_UCP_PO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4925 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4925 "Parser.cpp" break; } case 54: { { -#line 608 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 608 "Parser.rl" {te = p+1;{ -#line 608 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 608 "Parser.rl" currentCls->add(CLASS_UCP_PS, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4936 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4936 "Parser.cpp" break; } case 55: { { -#line 610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 610 "Parser.rl" {te = p+1;{ -#line 610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 610 "Parser.rl" currentCls->add(CLASS_UCP_SC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4947 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4947 "Parser.cpp" break; } case 56: { { -#line 611 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 611 "Parser.rl" {te = p+1;{ -#line 611 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 611 "Parser.rl" currentCls->add(CLASS_UCP_SK, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4958 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4958 "Parser.cpp" break; } case 57: { { -#line 612 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 612 "Parser.rl" {te = p+1;{ -#line 612 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 612 "Parser.rl" currentCls->add(CLASS_UCP_SM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4969 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4969 "Parser.cpp" break; } case 58: { { -#line 613 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 613 "Parser.rl" {te = p+1;{ -#line 613 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 613 "Parser.rl" currentCls->add(CLASS_UCP_SO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4980 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4980 "Parser.cpp" break; } case 59: { { -#line 615 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 615 "Parser.rl" {te = p+1;{ -#line 615 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 615 "Parser.rl" currentCls->add(CLASS_UCP_ZL, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 4991 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 4991 "Parser.cpp" break; } case 60: { { -#line 616 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 616 "Parser.rl" {te = p+1;{ -#line 616 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 616 "Parser.rl" currentCls->add(CLASS_UCP_ZP, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5002 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5002 "Parser.cpp" break; } case 61: { { -#line 617 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 617 "Parser.rl" {te = p+1;{ -#line 617 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 617 "Parser.rl" currentCls->add(CLASS_UCP_ZS, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5013 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5013 "Parser.cpp" break; } case 62: { { -#line 618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 618 "Parser.rl" {te = p+1;{ -#line 618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 618 "Parser.rl" currentCls->add(CLASS_UCP_XAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5024 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5024 "Parser.cpp" break; } case 63: { { -#line 619 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 619 "Parser.rl" {te = p+1;{ -#line 619 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 619 "Parser.rl" currentCls->add(CLASS_UCP_XPS, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5035 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5035 "Parser.cpp" break; } case 64: { { -#line 620 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 620 "Parser.rl" {te = p+1;{ -#line 620 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 620 "Parser.rl" currentCls->add(CLASS_UCP_XSP, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5046 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5046 "Parser.cpp" break; } case 65: { { -#line 621 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 621 "Parser.rl" {te = p+1;{ -#line 621 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 621 "Parser.rl" currentCls->add(CLASS_UCP_XWD, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5057 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5057 "Parser.cpp" break; } case 66: { { -#line 622 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 622 "Parser.rl" {te = p+1;{ -#line 622 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 622 "Parser.rl" currentCls->add(CLASS_SCRIPT_ARABIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5068 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5068 "Parser.cpp" break; } case 67: { { -#line 623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 623 "Parser.rl" {te = p+1;{ -#line 623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 623 "Parser.rl" currentCls->add(CLASS_SCRIPT_ARMENIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5079 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5079 "Parser.cpp" break; } case 68: { { -#line 624 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 624 "Parser.rl" {te = p+1;{ -#line 624 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 624 "Parser.rl" currentCls->add(CLASS_SCRIPT_AVESTAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5090 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5090 "Parser.cpp" break; } case 69: { { -#line 625 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 625 "Parser.rl" {te = p+1;{ -#line 625 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 625 "Parser.rl" currentCls->add(CLASS_SCRIPT_BALINESE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5101 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5101 "Parser.cpp" break; } case 70: { { -#line 626 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 626 "Parser.rl" {te = p+1;{ -#line 626 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 626 "Parser.rl" currentCls->add(CLASS_SCRIPT_BAMUM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5112 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5112 "Parser.cpp" break; } case 71: { { -#line 627 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 627 "Parser.rl" {te = p+1;{ -#line 627 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 627 "Parser.rl" currentCls->add(CLASS_SCRIPT_BATAK, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5123 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5123 "Parser.cpp" break; } case 72: { { -#line 628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 628 "Parser.rl" {te = p+1;{ -#line 628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 628 "Parser.rl" currentCls->add(CLASS_SCRIPT_BENGALI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5134 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5134 "Parser.cpp" break; } case 73: { { -#line 629 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 629 "Parser.rl" {te = p+1;{ -#line 629 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 629 "Parser.rl" currentCls->add(CLASS_SCRIPT_BOPOMOFO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5145 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5145 "Parser.cpp" break; } case 74: { { -#line 630 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 630 "Parser.rl" {te = p+1;{ -#line 630 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 630 "Parser.rl" currentCls->add(CLASS_SCRIPT_BRAHMI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5156 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5156 "Parser.cpp" break; } case 75: { { -#line 631 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 631 "Parser.rl" {te = p+1;{ -#line 631 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 631 "Parser.rl" currentCls->add(CLASS_SCRIPT_BRAILLE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5167 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5167 "Parser.cpp" break; } case 76: { { -#line 632 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 632 "Parser.rl" {te = p+1;{ -#line 632 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 632 "Parser.rl" currentCls->add(CLASS_SCRIPT_BUGINESE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5178 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5178 "Parser.cpp" break; } case 77: { { -#line 633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 633 "Parser.rl" {te = p+1;{ -#line 633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 633 "Parser.rl" currentCls->add(CLASS_SCRIPT_BUHID, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5189 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5189 "Parser.cpp" break; } case 78: { { -#line 634 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 634 "Parser.rl" {te = p+1;{ -#line 634 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 634 "Parser.rl" currentCls->add(CLASS_SCRIPT_CANADIAN_ABORIGINAL, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5200 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5200 "Parser.cpp" break; } case 79: { { -#line 635 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 635 "Parser.rl" {te = p+1;{ -#line 635 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 635 "Parser.rl" currentCls->add(CLASS_SCRIPT_CARIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5211 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5211 "Parser.cpp" break; } case 80: { { -#line 636 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 636 "Parser.rl" {te = p+1;{ -#line 636 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 636 "Parser.rl" currentCls->add(CLASS_SCRIPT_CHAM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5222 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5222 "Parser.cpp" break; } case 81: { { -#line 637 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 637 "Parser.rl" {te = p+1;{ -#line 637 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 637 "Parser.rl" currentCls->add(CLASS_SCRIPT_CHEROKEE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5233 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5233 "Parser.cpp" break; } case 82: { { -#line 638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 638 "Parser.rl" {te = p+1;{ -#line 638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 638 "Parser.rl" currentCls->add(CLASS_SCRIPT_COMMON, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5244 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5244 "Parser.cpp" break; } case 83: { { -#line 639 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 639 "Parser.rl" {te = p+1;{ -#line 639 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 639 "Parser.rl" currentCls->add(CLASS_SCRIPT_COPTIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5255 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5255 "Parser.cpp" break; } case 84: { { -#line 640 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 640 "Parser.rl" {te = p+1;{ -#line 640 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 640 "Parser.rl" currentCls->add(CLASS_SCRIPT_CUNEIFORM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5266 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5266 "Parser.cpp" break; } case 85: { { -#line 641 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 641 "Parser.rl" {te = p+1;{ -#line 641 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 641 "Parser.rl" currentCls->add(CLASS_SCRIPT_CYPRIOT, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5277 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5277 "Parser.cpp" break; } case 86: { { -#line 642 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 642 "Parser.rl" {te = p+1;{ -#line 642 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 642 "Parser.rl" currentCls->add(CLASS_SCRIPT_CYRILLIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5288 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5288 "Parser.cpp" break; } case 87: { { -#line 643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 643 "Parser.rl" {te = p+1;{ -#line 643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 643 "Parser.rl" currentCls->add(CLASS_SCRIPT_DESERET, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5299 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5299 "Parser.cpp" break; } case 88: { { -#line 644 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 644 "Parser.rl" {te = p+1;{ -#line 644 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 644 "Parser.rl" currentCls->add(CLASS_SCRIPT_DEVANAGARI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5310 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5310 "Parser.cpp" break; } case 89: { { -#line 645 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 645 "Parser.rl" {te = p+1;{ -#line 645 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 645 "Parser.rl" currentCls->add(CLASS_SCRIPT_EGYPTIAN_HIEROGLYPHS, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5321 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5321 "Parser.cpp" break; } case 90: { { -#line 646 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 646 "Parser.rl" {te = p+1;{ -#line 646 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 646 "Parser.rl" currentCls->add(CLASS_SCRIPT_ETHIOPIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5332 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5332 "Parser.cpp" break; } case 91: { { -#line 647 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 647 "Parser.rl" {te = p+1;{ -#line 647 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 647 "Parser.rl" currentCls->add(CLASS_SCRIPT_GEORGIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5343 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5343 "Parser.cpp" break; } case 92: { { -#line 648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 648 "Parser.rl" {te = p+1;{ -#line 648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 648 "Parser.rl" currentCls->add(CLASS_SCRIPT_GLAGOLITIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5354 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5354 "Parser.cpp" break; } case 93: { { -#line 649 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 649 "Parser.rl" {te = p+1;{ -#line 649 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 649 "Parser.rl" currentCls->add(CLASS_SCRIPT_GOTHIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5365 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5365 "Parser.cpp" break; } case 94: { { -#line 650 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 650 "Parser.rl" {te = p+1;{ -#line 650 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 650 "Parser.rl" currentCls->add(CLASS_SCRIPT_GREEK, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5376 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5376 "Parser.cpp" break; } case 95: { { -#line 651 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 651 "Parser.rl" {te = p+1;{ -#line 651 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 651 "Parser.rl" currentCls->add(CLASS_SCRIPT_GUJARATI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5387 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5387 "Parser.cpp" break; } case 96: { { -#line 652 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 652 "Parser.rl" {te = p+1;{ -#line 652 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 652 "Parser.rl" currentCls->add(CLASS_SCRIPT_GURMUKHI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5398 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5398 "Parser.cpp" break; } case 97: { { -#line 654 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 654 "Parser.rl" {te = p+1;{ -#line 654 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 654 "Parser.rl" currentCls->add(CLASS_SCRIPT_HANGUL, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5409 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5409 "Parser.cpp" break; } case 98: { { -#line 655 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 655 "Parser.rl" {te = p+1;{ -#line 655 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 655 "Parser.rl" currentCls->add(CLASS_SCRIPT_HANUNOO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5420 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5420 "Parser.cpp" break; } case 99: { { -#line 656 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 656 "Parser.rl" {te = p+1;{ -#line 656 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 656 "Parser.rl" currentCls->add(CLASS_SCRIPT_HEBREW, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5431 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5431 "Parser.cpp" break; } case 100: { { -#line 657 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 657 "Parser.rl" {te = p+1;{ -#line 657 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 657 "Parser.rl" currentCls->add(CLASS_SCRIPT_HIRAGANA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5442 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5442 "Parser.cpp" break; } case 101: { { -#line 658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 658 "Parser.rl" {te = p+1;{ -#line 658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 658 "Parser.rl" currentCls->add(CLASS_SCRIPT_IMPERIAL_ARAMAIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5453 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5453 "Parser.cpp" break; } case 102: { { -#line 659 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 659 "Parser.rl" {te = p+1;{ -#line 659 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 659 "Parser.rl" currentCls->add(CLASS_SCRIPT_INHERITED, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5464 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5464 "Parser.cpp" break; } case 103: { { -#line 660 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 660 "Parser.rl" {te = p+1;{ -#line 660 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 660 "Parser.rl" currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PAHLAVI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5475 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5475 "Parser.cpp" break; } case 104: { { -#line 661 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 661 "Parser.rl" {te = p+1;{ -#line 661 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 661 "Parser.rl" currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PARTHIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5486 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5486 "Parser.cpp" break; } case 105: { { -#line 662 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 662 "Parser.rl" {te = p+1;{ -#line 662 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 662 "Parser.rl" currentCls->add(CLASS_SCRIPT_JAVANESE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5497 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5497 "Parser.cpp" break; } case 106: { { -#line 663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 663 "Parser.rl" {te = p+1;{ -#line 663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 663 "Parser.rl" currentCls->add(CLASS_SCRIPT_KAITHI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5508 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5508 "Parser.cpp" break; } case 107: { { -#line 664 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 664 "Parser.rl" {te = p+1;{ -#line 664 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 664 "Parser.rl" currentCls->add(CLASS_SCRIPT_KANNADA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5519 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5519 "Parser.cpp" break; } case 108: { { -#line 665 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 665 "Parser.rl" {te = p+1;{ -#line 665 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 665 "Parser.rl" currentCls->add(CLASS_SCRIPT_KATAKANA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5530 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5530 "Parser.cpp" break; } case 109: { { -#line 666 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 666 "Parser.rl" {te = p+1;{ -#line 666 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 666 "Parser.rl" currentCls->add(CLASS_SCRIPT_KAYAH_LI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5541 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5541 "Parser.cpp" break; } case 110: { { -#line 667 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 667 "Parser.rl" {te = p+1;{ -#line 667 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 667 "Parser.rl" currentCls->add(CLASS_SCRIPT_KHAROSHTHI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5552 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5552 "Parser.cpp" break; } case 111: { { -#line 668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 668 "Parser.rl" {te = p+1;{ -#line 668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 668 "Parser.rl" currentCls->add(CLASS_SCRIPT_KHMER, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5563 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5563 "Parser.cpp" break; } case 112: { { -#line 669 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 669 "Parser.rl" {te = p+1;{ -#line 669 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 669 "Parser.rl" currentCls->add(CLASS_SCRIPT_LAO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5574 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5574 "Parser.cpp" break; } case 113: { { -#line 670 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 670 "Parser.rl" {te = p+1;{ -#line 670 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 670 "Parser.rl" currentCls->add(CLASS_SCRIPT_LATIN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5585 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5585 "Parser.cpp" break; } case 114: { { -#line 671 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 671 "Parser.rl" {te = p+1;{ -#line 671 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 671 "Parser.rl" currentCls->add(CLASS_SCRIPT_LEPCHA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5596 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5596 "Parser.cpp" break; } case 115: { { -#line 672 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 672 "Parser.rl" {te = p+1;{ -#line 672 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 672 "Parser.rl" currentCls->add(CLASS_SCRIPT_LIMBU, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5607 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5607 "Parser.cpp" break; } case 116: { { -#line 673 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 673 "Parser.rl" {te = p+1;{ -#line 673 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 673 "Parser.rl" currentCls->add(CLASS_SCRIPT_LINEAR_B, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5618 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5618 "Parser.cpp" break; } case 117: { { -#line 674 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 674 "Parser.rl" {te = p+1;{ -#line 674 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 674 "Parser.rl" currentCls->add(CLASS_SCRIPT_LISU, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5629 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5629 "Parser.cpp" break; } case 118: { { -#line 675 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 675 "Parser.rl" {te = p+1;{ -#line 675 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 675 "Parser.rl" currentCls->add(CLASS_SCRIPT_LYCIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5640 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5640 "Parser.cpp" break; } case 119: { { -#line 676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 676 "Parser.rl" {te = p+1;{ -#line 676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 676 "Parser.rl" currentCls->add(CLASS_SCRIPT_LYDIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5651 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5651 "Parser.cpp" break; } case 120: { { -#line 677 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 677 "Parser.rl" {te = p+1;{ -#line 677 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 677 "Parser.rl" currentCls->add(CLASS_SCRIPT_MALAYALAM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5662 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5662 "Parser.cpp" break; } case 121: { { -#line 678 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 678 "Parser.rl" {te = p+1;{ -#line 678 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 678 "Parser.rl" currentCls->add(CLASS_SCRIPT_MANDAIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5673 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5673 "Parser.cpp" break; } case 122: { { -#line 679 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 679 "Parser.rl" {te = p+1;{ -#line 679 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 679 "Parser.rl" currentCls->add(CLASS_SCRIPT_MEETEI_MAYEK, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5684 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5684 "Parser.cpp" break; } case 123: { { -#line 680 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 680 "Parser.rl" {te = p+1;{ -#line 680 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 680 "Parser.rl" currentCls->add(CLASS_SCRIPT_MONGOLIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5695 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5695 "Parser.cpp" break; } case 124: { { -#line 681 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 681 "Parser.rl" {te = p+1;{ -#line 681 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 681 "Parser.rl" currentCls->add(CLASS_SCRIPT_MYANMAR, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5706 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5706 "Parser.cpp" break; } case 125: { { -#line 682 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 682 "Parser.rl" {te = p+1;{ -#line 682 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 682 "Parser.rl" currentCls->add(CLASS_SCRIPT_NEW_TAI_LUE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5717 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5717 "Parser.cpp" break; } case 126: { { -#line 683 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 683 "Parser.rl" {te = p+1;{ -#line 683 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 683 "Parser.rl" currentCls->add(CLASS_SCRIPT_NKO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5728 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5728 "Parser.cpp" break; } case 127: { { -#line 684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 684 "Parser.rl" {te = p+1;{ -#line 684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 684 "Parser.rl" currentCls->add(CLASS_SCRIPT_OGHAM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5739 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5739 "Parser.cpp" break; } case 128: { { -#line 685 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 685 "Parser.rl" {te = p+1;{ -#line 685 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 685 "Parser.rl" currentCls->add(CLASS_SCRIPT_OL_CHIKI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5750 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5750 "Parser.cpp" break; } case 129: { { -#line 686 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 686 "Parser.rl" {te = p+1;{ -#line 686 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 686 "Parser.rl" currentCls->add(CLASS_SCRIPT_OLD_ITALIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5761 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5761 "Parser.cpp" break; } case 130: { { -#line 687 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 687 "Parser.rl" {te = p+1;{ -#line 687 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 687 "Parser.rl" currentCls->add(CLASS_SCRIPT_OLD_PERSIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5772 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5772 "Parser.cpp" break; } case 131: { { -#line 688 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 688 "Parser.rl" {te = p+1;{ -#line 688 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 688 "Parser.rl" currentCls->add(CLASS_SCRIPT_OLD_SOUTH_ARABIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5783 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5783 "Parser.cpp" break; } case 132: { { -#line 689 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 689 "Parser.rl" {te = p+1;{ -#line 689 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 689 "Parser.rl" currentCls->add(CLASS_SCRIPT_OLD_TURKIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5794 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5794 "Parser.cpp" break; } case 133: { { -#line 690 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 690 "Parser.rl" {te = p+1;{ -#line 690 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 690 "Parser.rl" currentCls->add(CLASS_SCRIPT_ORIYA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5805 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5805 "Parser.cpp" break; } case 134: { { -#line 691 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 691 "Parser.rl" {te = p+1;{ -#line 691 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 691 "Parser.rl" currentCls->add(CLASS_SCRIPT_OSMANYA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5816 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5816 "Parser.cpp" break; } case 135: { { -#line 692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 692 "Parser.rl" {te = p+1;{ -#line 692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 692 "Parser.rl" currentCls->add(CLASS_SCRIPT_PHAGS_PA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5827 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5827 "Parser.cpp" break; } case 136: { { -#line 693 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 693 "Parser.rl" {te = p+1;{ -#line 693 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 693 "Parser.rl" currentCls->add(CLASS_SCRIPT_PHOENICIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5838 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5838 "Parser.cpp" break; } case 137: { { -#line 694 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 694 "Parser.rl" {te = p+1;{ -#line 694 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 694 "Parser.rl" currentCls->add(CLASS_SCRIPT_REJANG, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5849 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5849 "Parser.cpp" break; } case 138: { { -#line 695 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 695 "Parser.rl" {te = p+1;{ -#line 695 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 695 "Parser.rl" currentCls->add(CLASS_SCRIPT_RUNIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5860 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5860 "Parser.cpp" break; } case 139: { { -#line 696 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 696 "Parser.rl" {te = p+1;{ -#line 696 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 696 "Parser.rl" currentCls->add(CLASS_SCRIPT_SAMARITAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5871 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5871 "Parser.cpp" break; } case 140: { { -#line 697 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 697 "Parser.rl" {te = p+1;{ -#line 697 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 697 "Parser.rl" currentCls->add(CLASS_SCRIPT_SAURASHTRA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5882 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5882 "Parser.cpp" break; } case 141: { { -#line 698 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 698 "Parser.rl" {te = p+1;{ -#line 698 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 698 "Parser.rl" currentCls->add(CLASS_SCRIPT_SHAVIAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5893 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5893 "Parser.cpp" break; } case 142: { { -#line 699 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 699 "Parser.rl" {te = p+1;{ -#line 699 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 699 "Parser.rl" currentCls->add(CLASS_SCRIPT_SINHALA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5904 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5904 "Parser.cpp" break; } case 143: { { -#line 700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 700 "Parser.rl" {te = p+1;{ -#line 700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 700 "Parser.rl" currentCls->add(CLASS_SCRIPT_SUNDANESE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5915 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5915 "Parser.cpp" break; } case 144: { { -#line 701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 701 "Parser.rl" {te = p+1;{ -#line 701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 701 "Parser.rl" currentCls->add(CLASS_SCRIPT_SYLOTI_NAGRI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5926 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5926 "Parser.cpp" break; } case 145: { { -#line 702 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 702 "Parser.rl" {te = p+1;{ -#line 702 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 702 "Parser.rl" currentCls->add(CLASS_SCRIPT_SYRIAC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5937 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5937 "Parser.cpp" break; } case 146: { { -#line 703 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 703 "Parser.rl" {te = p+1;{ -#line 703 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 703 "Parser.rl" currentCls->add(CLASS_SCRIPT_TAGALOG, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5948 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5948 "Parser.cpp" break; } case 147: { { -#line 704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 704 "Parser.rl" {te = p+1;{ -#line 704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 704 "Parser.rl" currentCls->add(CLASS_SCRIPT_TAGBANWA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5959 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5959 "Parser.cpp" break; } case 148: { { -#line 705 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 705 "Parser.rl" {te = p+1;{ -#line 705 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 705 "Parser.rl" currentCls->add(CLASS_SCRIPT_TAI_LE, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5970 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5970 "Parser.cpp" break; } case 149: { { -#line 706 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 706 "Parser.rl" {te = p+1;{ -#line 706 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 706 "Parser.rl" currentCls->add(CLASS_SCRIPT_TAI_THAM, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5981 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5981 "Parser.cpp" break; } case 150: { { -#line 707 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 707 "Parser.rl" {te = p+1;{ -#line 707 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 707 "Parser.rl" currentCls->add(CLASS_SCRIPT_TAI_VIET, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 5992 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 5992 "Parser.cpp" break; } case 151: { { -#line 708 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 708 "Parser.rl" {te = p+1;{ -#line 708 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 708 "Parser.rl" currentCls->add(CLASS_SCRIPT_TAMIL, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6003 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6003 "Parser.cpp" break; } case 152: { { -#line 709 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 709 "Parser.rl" {te = p+1;{ -#line 709 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 709 "Parser.rl" currentCls->add(CLASS_SCRIPT_TELUGU, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6014 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6014 "Parser.cpp" break; } case 153: { { -#line 710 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 710 "Parser.rl" {te = p+1;{ -#line 710 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 710 "Parser.rl" currentCls->add(CLASS_SCRIPT_THAANA, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6025 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6025 "Parser.cpp" break; } case 154: { { -#line 711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 711 "Parser.rl" {te = p+1;{ -#line 711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 711 "Parser.rl" currentCls->add(CLASS_SCRIPT_THAI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6036 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6036 "Parser.cpp" break; } case 155: { { -#line 712 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 712 "Parser.rl" {te = p+1;{ -#line 712 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 712 "Parser.rl" currentCls->add(CLASS_SCRIPT_TIBETAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6047 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6047 "Parser.cpp" break; } case 156: { { -#line 713 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 713 "Parser.rl" {te = p+1;{ -#line 713 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 713 "Parser.rl" currentCls->add(CLASS_SCRIPT_TIFINAGH, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6058 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6058 "Parser.cpp" break; } case 157: { { -#line 714 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 714 "Parser.rl" {te = p+1;{ -#line 714 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 714 "Parser.rl" currentCls->add(CLASS_SCRIPT_UGARITIC, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6069 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6069 "Parser.cpp" break; } case 158: { { -#line 715 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 715 "Parser.rl" {te = p+1;{ -#line 715 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 715 "Parser.rl" currentCls->add(CLASS_SCRIPT_VAI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6080 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6080 "Parser.cpp" break; } case 159: { { -#line 716 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 716 "Parser.rl" {te = p+1;{ -#line 716 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 716 "Parser.rl" currentCls->add(CLASS_SCRIPT_YI, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6091 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6091 "Parser.cpp" break; } case 160: { { -#line 717 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 717 "Parser.rl" {te = p+1;{ -#line 717 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 717 "Parser.rl" currentCls->add(CLASS_UCP_ANY, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6102 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6102 "Parser.cpp" break; } case 161: { { -#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 718 "Parser.rl" {te = p+1;{ -#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 718 "Parser.rl" throw LocatedParseError("Unknown property"); } }} -#line 6113 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6113 "Parser.cpp" break; } case 162: { { -#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 580 "Parser.rl" {te = p;p = p - 1;{ -#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 580 "Parser.rl" currentCls->add(CLASS_UCP_C, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6124 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6124 "Parser.cpp" break; } case 163: { { -#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 584 "Parser.rl" {te = p;p = p - 1;{ -#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 584 "Parser.rl" currentCls->add(CLASS_UCP_CO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6135 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6135 "Parser.cpp" break; } case 164: { { -#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 586 "Parser.rl" {te = p;p = p - 1;{ -#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 586 "Parser.rl" currentCls->add(CLASS_UCP_L, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6146 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6146 "Parser.cpp" break; } case 165: { { -#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 593 "Parser.rl" {te = p;p = p - 1;{ -#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 593 "Parser.rl" currentCls->add(CLASS_UCP_M, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6157 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6157 "Parser.cpp" break; } case 166: { { -#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 595 "Parser.rl" {te = p;p = p - 1;{ -#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 595 "Parser.rl" currentCls->add(CLASS_UCP_ME, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6168 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6168 "Parser.cpp" break; } case 167: { { -#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 597 "Parser.rl" {te = p;p = p - 1;{ -#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 597 "Parser.rl" currentCls->add(CLASS_UCP_N, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6179 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6179 "Parser.cpp" break; } case 168: { { -#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 601 "Parser.rl" {te = p;p = p - 1;{ -#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 601 "Parser.rl" currentCls->add(CLASS_UCP_P, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6190 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6190 "Parser.cpp" break; } case 169: { { -#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 609 "Parser.rl" {te = p;p = p - 1;{ -#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 609 "Parser.rl" currentCls->add(CLASS_UCP_S, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6201 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6201 "Parser.cpp" break; } case 170: { { -#line 614 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 614 "Parser.rl" {te = p;p = p - 1;{ -#line 614 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 614 "Parser.rl" currentCls->add(CLASS_UCP_Z, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6212 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6212 "Parser.cpp" break; } case 171: { { -#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 653 "Parser.rl" {te = p;p = p - 1;{ -#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 653 "Parser.rl" currentCls->add(CLASS_SCRIPT_HAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6223 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6223 "Parser.cpp" break; } case 172: { { -#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 718 "Parser.rl" {te = p;p = p - 1;{ -#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 718 "Parser.rl" throw LocatedParseError("Unknown property"); } }} -#line 6234 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6234 "Parser.cpp" break; } case 173: { { -#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 580 "Parser.rl" {p = ((te))-1; { -#line 580 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 580 "Parser.rl" currentCls->add(CLASS_UCP_C, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6246 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6246 "Parser.cpp" break; } case 174: { { -#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 584 "Parser.rl" {p = ((te))-1; { -#line 584 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 584 "Parser.rl" currentCls->add(CLASS_UCP_CO, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6258 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6258 "Parser.cpp" break; } case 175: { { -#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 586 "Parser.rl" {p = ((te))-1; { -#line 586 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 586 "Parser.rl" currentCls->add(CLASS_UCP_L, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6270 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6270 "Parser.cpp" break; } case 176: { { -#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 593 "Parser.rl" {p = ((te))-1; { -#line 593 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 593 "Parser.rl" currentCls->add(CLASS_UCP_M, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6282 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6282 "Parser.cpp" break; } case 177: { { -#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 595 "Parser.rl" {p = ((te))-1; { -#line 595 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 595 "Parser.rl" currentCls->add(CLASS_UCP_ME, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6294 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6294 "Parser.cpp" break; } case 178: { { -#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 597 "Parser.rl" {p = ((te))-1; { -#line 597 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 597 "Parser.rl" currentCls->add(CLASS_UCP_N, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6306 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6306 "Parser.cpp" break; } case 179: { { -#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 601 "Parser.rl" {p = ((te))-1; { -#line 601 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 601 "Parser.rl" currentCls->add(CLASS_UCP_P, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6318 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6318 "Parser.cpp" break; } case 180: { { -#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 609 "Parser.rl" {p = ((te))-1; { -#line 609 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 609 "Parser.rl" currentCls->add(CLASS_UCP_S, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6330 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6330 "Parser.cpp" break; } case 181: { { -#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 653 "Parser.rl" {p = ((te))-1; { -#line 653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 653 "Parser.rl" currentCls->add(CLASS_SCRIPT_HAN, negated); {top -= 1;cs = stack[top];goto _again;} } }} -#line 6342 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6342 "Parser.cpp" break; } case 182: { { -#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 718 "Parser.rl" {p = ((te))-1; { -#line 718 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 718 "Parser.rl" throw LocatedParseError("Unknown property"); } }} -#line 6354 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6354 "Parser.cpp" break; } case 183: { { -#line 733 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 733 "Parser.rl" {te = p+1;{ -#line 733 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 733 "Parser.rl" currentCls->add(CLASS_UCP_C, negated); if (!inCharClass) { @@ -6563,15 +6563,15 @@ static const short _regex_actions[] = { } }} -#line 6372 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6372 "Parser.cpp" break; } case 184: { { -#line 741 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 741 "Parser.rl" {te = p+1;{ -#line 741 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 741 "Parser.rl" currentCls->add(CLASS_UCP_L, negated); if (!inCharClass) { @@ -6582,15 +6582,15 @@ static const short _regex_actions[] = { } }} -#line 6390 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6390 "Parser.cpp" break; } case 185: { { -#line 749 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 749 "Parser.rl" {te = p+1;{ -#line 749 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 749 "Parser.rl" currentCls->add(CLASS_UCP_M, negated); if (!inCharClass) { @@ -6601,15 +6601,15 @@ static const short _regex_actions[] = { } }} -#line 6408 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6408 "Parser.cpp" break; } case 186: { { -#line 757 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 757 "Parser.rl" {te = p+1;{ -#line 757 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 757 "Parser.rl" currentCls->add(CLASS_UCP_N, negated); if (!inCharClass) { @@ -6620,15 +6620,15 @@ static const short _regex_actions[] = { } }} -#line 6426 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6426 "Parser.cpp" break; } case 187: { { -#line 765 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 765 "Parser.rl" {te = p+1;{ -#line 765 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 765 "Parser.rl" currentCls->add(CLASS_UCP_P, negated); if (!inCharClass) { @@ -6639,15 +6639,15 @@ static const short _regex_actions[] = { } }} -#line 6444 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6444 "Parser.cpp" break; } case 188: { { -#line 773 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 773 "Parser.rl" {te = p+1;{ -#line 773 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 773 "Parser.rl" currentCls->add(CLASS_UCP_S, negated); if (!inCharClass) { @@ -6658,15 +6658,15 @@ static const short _regex_actions[] = { } }} -#line 6462 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6462 "Parser.cpp" break; } case 189: { { -#line 781 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 781 "Parser.rl" {te = p+1;{ -#line 781 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 781 "Parser.rl" currentCls->add(CLASS_UCP_Z, negated); if (!inCharClass) { @@ -6677,451 +6677,451 @@ static const short _regex_actions[] = { } }} -#line 6480 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6480 "Parser.cpp" break; } case 190: { { -#line 790 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 790 "Parser.rl" {te = p+1;{ -#line 790 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 790 "Parser.rl" throw LocatedParseError("Unknown property"); } }} -#line 6491 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6491 "Parser.cpp" break; } case 191: { { -#line 796 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 796 "Parser.rl" {te = p+1;{ -#line 796 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 796 "Parser.rl" throw LocatedParseError("Unsupported POSIX collating " "element"); } }} -#line 6505 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6505 "Parser.cpp" break; } case 192: { { -#line 803 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 803 "Parser.rl" {te = p+1;{ -#line 803 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 803 "Parser.rl" currentCls->add(CLASS_ALNUM, false); } }} -#line 6518 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6518 "Parser.cpp" break; } case 193: { { -#line 806 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 806 "Parser.rl" {te = p+1;{ -#line 806 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 806 "Parser.rl" currentCls->add(CLASS_ALNUM, true); } }} -#line 6531 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6531 "Parser.cpp" break; } case 194: { { -#line 809 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 809 "Parser.rl" {te = p+1;{ -#line 809 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 809 "Parser.rl" currentCls->add(CLASS_ALPHA, false); } }} -#line 6544 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6544 "Parser.cpp" break; } case 195: { { -#line 812 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 812 "Parser.rl" {te = p+1;{ -#line 812 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 812 "Parser.rl" currentCls->add(CLASS_ALPHA, true); } }} -#line 6557 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6557 "Parser.cpp" break; } case 196: { { -#line 815 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 815 "Parser.rl" {te = p+1;{ -#line 815 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 815 "Parser.rl" currentCls->add(CLASS_ASCII, false); } }} -#line 6570 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6570 "Parser.cpp" break; } case 197: { { -#line 818 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 818 "Parser.rl" {te = p+1;{ -#line 818 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 818 "Parser.rl" currentCls->add(CLASS_ASCII, true); } }} -#line 6583 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6583 "Parser.cpp" break; } case 198: { { -#line 821 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 821 "Parser.rl" {te = p+1;{ -#line 821 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 821 "Parser.rl" currentCls->add(CLASS_BLANK, false); } }} -#line 6596 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6596 "Parser.cpp" break; } case 199: { { -#line 824 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 824 "Parser.rl" {te = p+1;{ -#line 824 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 824 "Parser.rl" currentCls->add(CLASS_BLANK, true); } }} -#line 6609 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6609 "Parser.cpp" break; } case 200: { { -#line 827 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 827 "Parser.rl" {te = p+1;{ -#line 827 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 827 "Parser.rl" currentCls->add(CLASS_CNTRL, false); } }} -#line 6622 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6622 "Parser.cpp" break; } case 201: { { -#line 830 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 830 "Parser.rl" {te = p+1;{ -#line 830 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 830 "Parser.rl" currentCls->add(CLASS_CNTRL, true); } }} -#line 6635 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6635 "Parser.cpp" break; } case 202: { { -#line 833 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 833 "Parser.rl" {te = p+1;{ -#line 833 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 833 "Parser.rl" currentCls->add(CLASS_DIGIT, false); } }} -#line 6648 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6648 "Parser.cpp" break; } case 203: { { -#line 836 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 836 "Parser.rl" {te = p+1;{ -#line 836 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 836 "Parser.rl" currentCls->add(CLASS_DIGIT, true); } }} -#line 6661 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6661 "Parser.cpp" break; } case 204: { { -#line 839 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 839 "Parser.rl" {te = p+1;{ -#line 839 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 839 "Parser.rl" currentCls->add(CLASS_GRAPH, false); } }} -#line 6674 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6674 "Parser.cpp" break; } case 205: { { -#line 842 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 842 "Parser.rl" {te = p+1;{ -#line 842 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 842 "Parser.rl" currentCls->add(CLASS_GRAPH, true); } }} -#line 6687 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6687 "Parser.cpp" break; } case 206: { { -#line 845 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 845 "Parser.rl" {te = p+1;{ -#line 845 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 845 "Parser.rl" currentCls->add(CLASS_LOWER, false); } }} -#line 6700 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6700 "Parser.cpp" break; } case 207: { { -#line 848 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 848 "Parser.rl" {te = p+1;{ -#line 848 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 848 "Parser.rl" currentCls->add(CLASS_LOWER, true); } }} -#line 6713 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6713 "Parser.cpp" break; } case 208: { { -#line 851 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 851 "Parser.rl" {te = p+1;{ -#line 851 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 851 "Parser.rl" currentCls->add(CLASS_PRINT, false); } }} -#line 6726 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6726 "Parser.cpp" break; } case 209: { { -#line 854 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 854 "Parser.rl" {te = p+1;{ -#line 854 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 854 "Parser.rl" currentCls->add(CLASS_PRINT, true); } }} -#line 6739 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6739 "Parser.cpp" break; } case 210: { { -#line 857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 857 "Parser.rl" {te = p+1;{ -#line 857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 857 "Parser.rl" currentCls->add(CLASS_PUNCT, false); } }} -#line 6752 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6752 "Parser.cpp" break; } case 211: { { -#line 860 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 860 "Parser.rl" {te = p+1;{ -#line 860 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 860 "Parser.rl" currentCls->add(CLASS_PUNCT, true); } }} -#line 6765 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6765 "Parser.cpp" break; } case 212: { { -#line 864 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 864 "Parser.rl" {te = p+1;{ -#line 864 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 864 "Parser.rl" currentCls->add(CLASS_SPACE, false); } }} -#line 6778 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6778 "Parser.cpp" break; } case 213: { { -#line 867 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 867 "Parser.rl" {te = p+1;{ -#line 867 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 867 "Parser.rl" currentCls->add(CLASS_SPACE, true); } }} -#line 6791 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6791 "Parser.cpp" break; } case 214: { { -#line 870 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 870 "Parser.rl" {te = p+1;{ -#line 870 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 870 "Parser.rl" currentCls->add(CLASS_UPPER, false); } }} -#line 6804 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6804 "Parser.cpp" break; } case 215: { { -#line 873 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 873 "Parser.rl" {te = p+1;{ -#line 873 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 873 "Parser.rl" currentCls->add(CLASS_UPPER, true); } }} -#line 6817 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6817 "Parser.cpp" break; } case 216: { { -#line 876 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 876 "Parser.rl" {te = p+1;{ -#line 876 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 876 "Parser.rl" currentCls->add(CLASS_WORD, false); } }} -#line 6830 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6830 "Parser.cpp" break; } case 217: { { -#line 879 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 879 "Parser.rl" {te = p+1;{ -#line 879 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 879 "Parser.rl" currentCls->add(CLASS_WORD, true); } }} -#line 6843 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6843 "Parser.cpp" break; } case 218: { { -#line 882 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 882 "Parser.rl" {te = p+1;{ -#line 882 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 882 "Parser.rl" currentCls->add(CLASS_XDIGIT, false); } }} -#line 6856 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6856 "Parser.cpp" break; } case 219: { { -#line 885 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 885 "Parser.rl" {te = p+1;{ -#line 885 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 885 "Parser.rl" currentCls->add(CLASS_XDIGIT, true); } }} -#line 6869 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6869 "Parser.cpp" break; } case 220: { { -#line 890 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 890 "Parser.rl" {te = p+1;{ -#line 890 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 890 "Parser.rl" throw LocatedParseError("Invalid POSIX named class"); } }} -#line 6882 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6882 "Parser.cpp" break; } case 221: { { -#line 893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 893 "Parser.rl" {te = p+1;{ -#line 893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 893 "Parser.rl" {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -7131,186 +7131,186 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 843;goto _again;}} }} -#line 6902 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6902 "Parser.cpp" break; } case 222: { { -#line 896 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 896 "Parser.rl" {te = p+1;{ -#line 896 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 896 "Parser.rl" /*noop*/} }} -#line 6913 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6913 "Parser.cpp" break; } case 223: { { -#line 898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 898 "Parser.rl" {te = p+1;{ -#line 898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 898 "Parser.rl" currentCls->add('\x08'); } }} -#line 6926 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6926 "Parser.cpp" break; } case 224: { { -#line 902 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 902 "Parser.rl" {te = p+1;{ -#line 902 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 902 "Parser.rl" currentCls->add('\x09'); } }} -#line 6939 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6939 "Parser.cpp" break; } case 225: { { -#line 906 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 906 "Parser.rl" {te = p+1;{ -#line 906 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 906 "Parser.rl" currentCls->add('\x0a'); } }} -#line 6952 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6952 "Parser.cpp" break; } case 226: { { -#line 910 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 910 "Parser.rl" {te = p+1;{ -#line 910 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 910 "Parser.rl" currentCls->add('\x0d'); } }} -#line 6965 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6965 "Parser.cpp" break; } case 227: { { -#line 914 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 914 "Parser.rl" {te = p+1;{ -#line 914 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 914 "Parser.rl" currentCls->add('\x0c'); } }} -#line 6978 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6978 "Parser.cpp" break; } case 228: { { -#line 918 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 918 "Parser.rl" {te = p+1;{ -#line 918 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 918 "Parser.rl" currentCls->add('\x07'); } }} -#line 6991 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 6991 "Parser.cpp" break; } case 229: { { -#line 922 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 922 "Parser.rl" {te = p+1;{ -#line 922 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 922 "Parser.rl" currentCls->add('\x1b'); } }} -#line 7004 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7004 "Parser.cpp" break; } case 230: { { -#line 926 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 926 "Parser.rl" {te = p+1;{ -#line 926 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 926 "Parser.rl" currentCls->add(CLASS_HORZ, false); } }} -#line 7017 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7017 "Parser.cpp" break; } case 231: { { -#line 930 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 930 "Parser.rl" {te = p+1;{ -#line 930 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 930 "Parser.rl" currentCls->add(CLASS_HORZ, true); } }} -#line 7030 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7030 "Parser.cpp" break; } case 232: { { -#line 934 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 934 "Parser.rl" {te = p+1;{ -#line 934 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 934 "Parser.rl" currentCls->add(CLASS_VERT, false); } }} -#line 7043 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7043 "Parser.cpp" break; } case 233: { { -#line 938 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 938 "Parser.rl" {te = p+1;{ -#line 938 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 938 "Parser.rl" currentCls->add(CLASS_VERT, true); } }} -#line 7056 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7056 "Parser.cpp" break; } case 234: { { -#line 942 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 942 "Parser.rl" {te = p+1;{ -#line 942 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 942 "Parser.rl" negated = false; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -7320,20 +7320,20 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 559;goto _again;}} }} -#line 7078 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7078 "Parser.cpp" break; } case 235: { { -#line 948 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 948 "Parser.rl" {te = p+1;{ -#line 948 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 948 "Parser.rl" negated = false; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -7343,20 +7343,20 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 818;goto _again;}} }} -#line 7100 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7100 "Parser.cpp" break; } case 236: { { -#line 954 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 954 "Parser.rl" {te = p+1;{ -#line 954 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 954 "Parser.rl" negated = true; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -7366,20 +7366,20 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 559;goto _again;}} }} -#line 7122 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7122 "Parser.cpp" break; } case 237: { { -#line 960 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 960 "Parser.rl" {te = p+1;{ -#line 960 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 960 "Parser.rl" negated = true; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -7389,43 +7389,43 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 818;goto _again;}} }} -#line 7144 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7144 "Parser.cpp" break; } case 238: { { -#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 970 "Parser.rl" {te = p+1;{ -#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 970 "Parser.rl" currentCls->add(octAccumulator); } }} -#line 7157 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7157 "Parser.cpp" break; } case 239: { { -#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 973 "Parser.rl" {te = p+1;{ -#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 973 "Parser.rl" currentCls->add(octAccumulator); } }} -#line 7170 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7170 "Parser.cpp" break; } case 240: { { -#line 977 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 977 "Parser.rl" {te = p+1;{ -#line 977 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 977 "Parser.rl" string oct(ts + 3, te - ts - 4); unsigned long val; @@ -7441,29 +7441,29 @@ static const short _regex_actions[] = { } }} -#line 7193 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7193 "Parser.cpp" break; } case 241: { { -#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 997 "Parser.rl" {te = p+1;{ -#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 997 "Parser.rl" currentCls->add(accumulator); } }} -#line 7206 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7206 "Parser.cpp" break; } case 242: { { -#line 1001 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1001 "Parser.rl" {te = p+1;{ -#line 1001 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1001 "Parser.rl" // whatever we found here currentCls->add(*(ts + 1)); @@ -7471,15 +7471,15 @@ static const short _regex_actions[] = { } }} -#line 7221 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7221 "Parser.cpp" break; } case 243: { { -#line 1007 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1007 "Parser.rl" {te = p+1;{ -#line 1007 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1007 "Parser.rl" string hex(ts + 3, te - ts - 4); unsigned long val; @@ -7495,15 +7495,15 @@ static const short _regex_actions[] = { } }} -#line 7244 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7244 "Parser.cpp" break; } case 244: { { -#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1025 "Parser.rl" {te = p+1;{ -#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1025 "Parser.rl" if (te - ts < 3) { assert(te - ts == 2); @@ -7515,113 +7515,113 @@ static const short _regex_actions[] = { } }} -#line 7263 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7263 "Parser.cpp" break; } case 245: { { -#line 1035 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1035 "Parser.rl" {te = p+1;{ -#line 1035 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1035 "Parser.rl" currentCls->add(CLASS_WORD, false); } }} -#line 7276 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7276 "Parser.cpp" break; } case 246: { { -#line 1039 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1039 "Parser.rl" {te = p+1;{ -#line 1039 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1039 "Parser.rl" currentCls->add(CLASS_WORD, true); } }} -#line 7289 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7289 "Parser.cpp" break; } case 247: { { -#line 1043 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1043 "Parser.rl" {te = p+1;{ -#line 1043 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1043 "Parser.rl" currentCls->add(CLASS_SPACE, false); } }} -#line 7302 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7302 "Parser.cpp" break; } case 248: { { -#line 1047 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1047 "Parser.rl" {te = p+1;{ -#line 1047 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1047 "Parser.rl" currentCls->add(CLASS_SPACE, true); } }} -#line 7315 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7315 "Parser.cpp" break; } case 249: { { -#line 1051 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1051 "Parser.rl" {te = p+1;{ -#line 1051 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1051 "Parser.rl" currentCls->add(CLASS_DIGIT, false); } }} -#line 7328 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7328 "Parser.cpp" break; } case 250: { { -#line 1055 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1055 "Parser.rl" {te = p+1;{ -#line 1055 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1055 "Parser.rl" currentCls->add(CLASS_DIGIT, true); } }} -#line 7341 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7341 "Parser.cpp" break; } case 251: { { -#line 1058 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1058 "Parser.rl" {te = p+1;{ -#line 1058 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1058 "Parser.rl" currentCls->addDash(); } }} -#line 7354 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7354 "Parser.cpp" break; } case 252: { { -#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 276 "Parser.rl" {te = p+1;{ -#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 276 "Parser.rl" ostringstream str; str << "'\\" << *(ts + 1) << "' at index " << ts - ptr @@ -7630,15 +7630,15 @@ static const short _regex_actions[] = { } }} -#line 7370 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7370 "Parser.cpp" break; } case 253: { { -#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 276 "Parser.rl" {te = p+1;{ -#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 276 "Parser.rl" ostringstream str; str << "'\\" << *(ts + 1) << "' at index " << ts - ptr @@ -7647,15 +7647,15 @@ static const short _regex_actions[] = { } }} -#line 7386 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7386 "Parser.cpp" break; } case 254: { { -#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 276 "Parser.rl" {te = p+1;{ -#line 276 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 276 "Parser.rl" ostringstream str; str << "'\\" << *(ts + 1) << "' at index " << ts - ptr @@ -7664,104 +7664,104 @@ static const short _regex_actions[] = { } }} -#line 7402 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7402 "Parser.cpp" break; } case 255: { { -#line 1075 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1075 "Parser.rl" {te = p+1;{ -#line 1075 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1075 "Parser.rl" // add the literal char currentCls->add(*(ts + 1)); } }} -#line 7416 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7416 "Parser.cpp" break; } case 256: { { -#line 1081 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1081 "Parser.rl" {te = p+1;{ -#line 1081 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1081 "Parser.rl" assert(mode.utf8); currentCls->add(readUtf8CodePoint2c(ts)); } }} -#line 7430 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7430 "Parser.cpp" break; } case 257: { { -#line 1086 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1086 "Parser.rl" {te = p+1;{ -#line 1086 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1086 "Parser.rl" assert(mode.utf8); currentCls->add(readUtf8CodePoint3c(ts)); } }} -#line 7444 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7444 "Parser.cpp" break; } case 258: { { -#line 1091 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1091 "Parser.rl" {te = p+1;{ -#line 1091 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1091 "Parser.rl" assert(mode.utf8); currentCls->add(readUtf8CodePoint4c(ts)); } }} -#line 7458 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7458 "Parser.cpp" break; } case 259: { { -#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1096 "Parser.rl" {te = p+1;{ -#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1096 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7472 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7472 "Parser.cpp" break; } case 260: { { -#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1102 "Parser.rl" {te = p+1;{ -#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1102 "Parser.rl" currentCls->add((u8)*ts); } }} -#line 7485 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7485 "Parser.cpp" break; } case 261: { { -#line 1106 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1106 "Parser.rl" {te = p+1;{ -#line 1106 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1106 "Parser.rl" currentCls->finalize(); currentSeq->addComponent(move(currentCls)); @@ -7769,109 +7769,109 @@ static const short _regex_actions[] = { {cs = 746;goto _again;}} }} -#line 7500 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7500 "Parser.cpp" break; } case 262: { { -#line 966 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 966 "Parser.rl" {te = p;p = p - 1;{ -#line 966 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 966 "Parser.rl" throw LocatedParseError("Malformed property"); } }} -#line 7511 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7511 "Parser.cpp" break; } case 263: { { -#line 967 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 967 "Parser.rl" {te = p;p = p - 1;{ -#line 967 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 967 "Parser.rl" throw LocatedParseError("Malformed property"); } }} -#line 7522 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7522 "Parser.cpp" break; } case 264: { { -#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 970 "Parser.rl" {te = p;p = p - 1;{ -#line 970 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 970 "Parser.rl" currentCls->add(octAccumulator); } }} -#line 7535 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7535 "Parser.cpp" break; } case 265: { { -#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 973 "Parser.rl" {te = p;p = p - 1;{ -#line 973 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 973 "Parser.rl" currentCls->add(octAccumulator); } }} -#line 7548 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7548 "Parser.cpp" break; } case 266: { { -#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 992 "Parser.rl" {te = p;p = p - 1;{ -#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 992 "Parser.rl" throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); } }} -#line 7561 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7561 "Parser.cpp" break; } case 267: { { -#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 997 "Parser.rl" {te = p;p = p - 1;{ -#line 997 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 997 "Parser.rl" currentCls->add(accumulator); } }} -#line 7574 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7574 "Parser.cpp" break; } case 268: { { -#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1021 "Parser.rl" {te = p;p = p - 1;{ -#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1021 "Parser.rl" throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); } }} -#line 7587 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7587 "Parser.cpp" break; } case 269: { { -#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1025 "Parser.rl" {te = p;p = p - 1;{ -#line 1025 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1025 "Parser.rl" if (te - ts < 3) { assert(te - ts == 2); @@ -7883,105 +7883,105 @@ static const short _regex_actions[] = { } }} -#line 7606 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7606 "Parser.cpp" break; } case 270: { { -#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1096 "Parser.rl" {te = p;p = p - 1;{ -#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1096 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7620 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7620 "Parser.cpp" break; } case 271: { { -#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1102 "Parser.rl" {te = p;p = p - 1;{ -#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1102 "Parser.rl" currentCls->add((u8)*ts); } }} -#line 7633 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7633 "Parser.cpp" break; } case 272: { { -#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 992 "Parser.rl" {p = ((te))-1; { -#line 992 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 992 "Parser.rl" throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); } }} -#line 7647 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7647 "Parser.cpp" break; } case 273: { { -#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1021 "Parser.rl" {p = ((te))-1; { -#line 1021 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1021 "Parser.rl" throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); } }} -#line 7661 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7661 "Parser.cpp" break; } case 274: { { -#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1096 "Parser.rl" {p = ((te))-1; { -#line 1096 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1096 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7676 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7676 "Parser.cpp" break; } case 275: { { -#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1102 "Parser.rl" {p = ((te))-1; { -#line 1102 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1102 "Parser.rl" currentCls->add((u8)*ts); } }} -#line 7690 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7690 "Parser.cpp" break; } case 276: { { -#line 1120 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1120 "Parser.rl" {te = p+1;{ -#line 1120 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1120 "Parser.rl" if (currentCls->isNegated()) { // Already seen a caret; the second one is not a meta-character. @@ -7994,32 +7994,32 @@ static const short _regex_actions[] = { } }} -#line 7710 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7710 "Parser.cpp" break; } case 277: { { -#line 1133 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1133 "Parser.rl" {te = p+1;{ -#line 1133 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1133 "Parser.rl" currentCls->add(']'); inCharClassEarly = false; } }} -#line 7724 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7724 "Parser.cpp" break; } case 278: { { -#line 1138 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1138 "Parser.rl" {te = p+1;{ -#line 1138 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1138 "Parser.rl" {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -8029,70 +8029,70 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 843;goto _again;}} }} -#line 7743 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7743 "Parser.cpp" break; } case 279: { { -#line 1139 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1139 "Parser.rl" {te = p+1;{ -#line 1139 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1139 "Parser.rl" /*noop*/} }} -#line 7754 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7754 "Parser.cpp" break; } case 280: { { -#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1142 "Parser.rl" {te = p+1;{ -#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1142 "Parser.rl" inCharClassEarly = false; {p = p - 1; } {cs = 819;goto _again;}} }} -#line 7768 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7768 "Parser.cpp" break; } case 281: { { -#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1142 "Parser.rl" {te = p;p = p - 1;{ -#line 1142 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1142 "Parser.rl" inCharClassEarly = false; {p = p - 1; } {cs = 819;goto _again;}} }} -#line 7782 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7782 "Parser.cpp" break; } case 282: { { -#line 1154 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1154 "Parser.rl" {te = p+1;{ -#line 1154 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1154 "Parser.rl" {cs = 746;goto _again;}} }} -#line 7794 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7794 "Parser.cpp" break; } case 283: { { -#line 1159 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1159 "Parser.rl" {te = p+1;{ -#line 1159 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1159 "Parser.rl" assert(mode.utf8); /* leverage ComponentClass to generate the vertices */ @@ -8103,15 +8103,15 @@ static const short _regex_actions[] = { } }} -#line 7812 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7812 "Parser.cpp" break; } case 284: { { -#line 1168 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1168 "Parser.rl" {te = p+1;{ -#line 1168 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1168 "Parser.rl" assert(mode.utf8); /* leverage ComponentClass to generate the vertices */ @@ -8122,15 +8122,15 @@ static const short _regex_actions[] = { } }} -#line 7830 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7830 "Parser.cpp" break; } case 285: { { -#line 1177 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1177 "Parser.rl" {te = p+1;{ -#line 1177 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1177 "Parser.rl" assert(mode.utf8); /* leverage ComponentClass to generate the vertices */ @@ -8141,103 +8141,103 @@ static const short _regex_actions[] = { } }} -#line 7848 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7848 "Parser.cpp" break; } case 286: { { -#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1186 "Parser.rl" {te = p+1;{ -#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1186 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7862 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7862 "Parser.cpp" break; } case 287: { { -#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1192 "Parser.rl" {te = p+1;{ -#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1192 "Parser.rl" addLiteral(currentSeq, *ts, mode); } }} -#line 7875 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7875 "Parser.cpp" break; } case 288: { { -#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1186 "Parser.rl" {te = p;p = p - 1;{ -#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1186 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7889 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7889 "Parser.cpp" break; } case 289: { { -#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1192 "Parser.rl" {te = p;p = p - 1;{ -#line 1192 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1192 "Parser.rl" addLiteral(currentSeq, *ts, mode); } }} -#line 7902 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7902 "Parser.cpp" break; } case 290: { { -#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1186 "Parser.rl" {p = ((te))-1; { -#line 1186 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1186 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7917 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7917 "Parser.cpp" break; } case 291: { { -#line 1202 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1202 "Parser.rl" {te = p+1;{ -#line 1202 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1202 "Parser.rl" {top -= 1;cs = stack[top];goto _again;} } }} -#line 7930 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7930 "Parser.cpp" break; } case 292: { { -#line 1207 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1207 "Parser.rl" {te = p+1;{ -#line 1207 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1207 "Parser.rl" assert(mode.utf8); currentCls->add(readUtf8CodePoint2c(ts)); @@ -8245,15 +8245,15 @@ static const short _regex_actions[] = { } }} -#line 7945 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7945 "Parser.cpp" break; } case 293: { { -#line 1213 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1213 "Parser.rl" {te = p+1;{ -#line 1213 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1213 "Parser.rl" assert(mode.utf8); currentCls->add(readUtf8CodePoint3c(ts)); @@ -8261,15 +8261,15 @@ static const short _regex_actions[] = { } }} -#line 7960 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7960 "Parser.cpp" break; } case 294: { { -#line 1219 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1219 "Parser.rl" {te = p+1;{ -#line 1219 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1219 "Parser.rl" assert(mode.utf8); currentCls->add(readUtf8CodePoint4c(ts)); @@ -8277,95 +8277,95 @@ static const short _regex_actions[] = { } }} -#line 7975 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7975 "Parser.cpp" break; } case 295: { { -#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1225 "Parser.rl" {te = p+1;{ -#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1225 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 7989 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 7989 "Parser.cpp" break; } case 296: { { -#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1231 "Parser.rl" {te = p+1;{ -#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1231 "Parser.rl" currentCls->add(*ts); inCharClassEarly = false; } }} -#line 8003 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8003 "Parser.cpp" break; } case 297: { { -#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1225 "Parser.rl" {te = p;p = p - 1;{ -#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1225 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 8017 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8017 "Parser.cpp" break; } case 298: { { -#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1231 "Parser.rl" {te = p;p = p - 1;{ -#line 1231 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1231 "Parser.rl" currentCls->add(*ts); inCharClassEarly = false; } }} -#line 8031 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8031 "Parser.cpp" break; } case 299: { { -#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1225 "Parser.rl" {p = ((te))-1; { -#line 1225 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1225 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 8046 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8046 "Parser.cpp" break; } case 300: { { -#line 1243 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1243 "Parser.rl" {te = p+1;{ -#line 1243 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1243 "Parser.rl" inComment = false; {cs = 746;goto _again;}} }} -#line 8057 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8057 "Parser.cpp" break; } @@ -8374,19 +8374,19 @@ static const short _regex_actions[] = { #line 1 "-" {te = p+1;}} -#line 8065 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8065 "Parser.cpp" break; } case 302: { { -#line 1255 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1255 "Parser.rl" {te = p+1;{ -#line 1255 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1255 "Parser.rl" inComment = false; {cs = 746;goto _again;}} }} -#line 8076 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8076 "Parser.cpp" break; } @@ -8395,42 +8395,42 @@ static const short _regex_actions[] = { #line 1 "-" {te = p+1;}} -#line 8084 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8084 "Parser.cpp" break; } case 304: { { -#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1491 "Parser.rl" {act = 288;}} -#line 8092 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8092 "Parser.cpp" break; } case 305: { { -#line 1508 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1508 "Parser.rl" {act = 290;}} -#line 8100 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8100 "Parser.cpp" break; } case 306: { { -#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1737 "Parser.rl" {act = 330;}} -#line 8108 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8108 "Parser.cpp" break; } case 307: { { -#line 362 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 362 "Parser.rl" {te = p+1;{ -#line 362 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 362 "Parser.rl" if (sequences.empty()) { throw LocatedParseError("Unmatched parentheses"); @@ -8440,98 +8440,98 @@ static const short _regex_actions[] = { } }} -#line 8125 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8125 "Parser.cpp" break; } case 308: { { -#line 1274 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1274 "Parser.rl" {te = p+1;{ -#line 1274 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1274 "Parser.rl" currentSeq->addAlternation(); } }} -#line 8138 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8138 "Parser.cpp" break; } case 309: { { -#line 1279 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1279 "Parser.rl" {te = p+1;{ -#line 1279 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1279 "Parser.rl" throw LocatedParseError("POSIX named classes are only " "supported inside a class"); } }} -#line 8152 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8152 "Parser.cpp" break; } case 310: { { -#line 1286 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1286 "Parser.rl" {te = p+1;{ -#line 1286 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1286 "Parser.rl" throw LocatedParseError("Unsupported POSIX collating " "element"); } }} -#line 8166 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8166 "Parser.cpp" break; } case 311: { { -#line 1293 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1293 "Parser.rl" {te = p+1;{ -#line 1293 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1293 "Parser.rl" {cs = 838;goto _again;}} }} -#line 8178 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8178 "Parser.cpp" break; } case 312: { { -#line 1297 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1297 "Parser.rl" {te = p+1;{ -#line 1297 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1297 "Parser.rl" /* noop */ } }} -#line 8189 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8189 "Parser.cpp" break; } case 313: { { -#line 1299 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1299 "Parser.rl" {te = p+1;{ -#line 1299 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1299 "Parser.rl" currentSeq->addComponent(generateComponent(CLASS_ANY, false, mode)); } }} -#line 8202 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8202 "Parser.cpp" break; } case 314: { { -#line 1303 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1303 "Parser.rl" {te = p+1;{ -#line 1303 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1303 "Parser.rl" if (mode.utf8) { throw LocatedParseError("\\C is unsupported in UTF8"); @@ -8540,15 +8540,15 @@ static const short _regex_actions[] = { } }} -#line 8218 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8218 "Parser.cpp" break; } case 315: { { -#line 1317 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1317 "Parser.rl" {te = p+1;{ -#line 1317 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1317 "Parser.rl" if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, ComponentRepeat::REPEAT_NONGREEDY)) { @@ -8557,15 +8557,15 @@ static const short _regex_actions[] = { } }} -#line 8234 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8234 "Parser.cpp" break; } case 316: { { -#line 1324 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1324 "Parser.rl" {te = p+1;{ -#line 1324 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1324 "Parser.rl" if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, ComponentRepeat::REPEAT_POSSESSIVE)) { @@ -8574,15 +8574,15 @@ static const short _regex_actions[] = { } }} -#line 8250 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8250 "Parser.cpp" break; } case 317: { { -#line 1338 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1338 "Parser.rl" {te = p+1;{ -#line 1338 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1338 "Parser.rl" if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, ComponentRepeat::REPEAT_NONGREEDY)) { @@ -8591,15 +8591,15 @@ static const short _regex_actions[] = { } }} -#line 8266 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8266 "Parser.cpp" break; } case 318: { { -#line 1345 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1345 "Parser.rl" {te = p+1;{ -#line 1345 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1345 "Parser.rl" if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, ComponentRepeat::REPEAT_POSSESSIVE)) { @@ -8608,15 +8608,15 @@ static const short _regex_actions[] = { } }} -#line 8282 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8282 "Parser.cpp" break; } case 319: { { -#line 1359 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1359 "Parser.rl" {te = p+1;{ -#line 1359 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1359 "Parser.rl" if (!currentSeq->addRepeat( 0, 1, ComponentRepeat::REPEAT_NONGREEDY)) { @@ -8625,15 +8625,15 @@ static const short _regex_actions[] = { } }} -#line 8298 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8298 "Parser.cpp" break; } case 320: { { -#line 1366 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1366 "Parser.rl" {te = p+1;{ -#line 1366 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1366 "Parser.rl" if (!currentSeq->addRepeat( 0, 1, ComponentRepeat::REPEAT_POSSESSIVE)) { @@ -8642,15 +8642,15 @@ static const short _regex_actions[] = { } }} -#line 8314 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8314 "Parser.cpp" break; } case 321: { { -#line 1383 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1383 "Parser.rl" {te = p+1;{ -#line 1383 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1383 "Parser.rl" if (repeatN > repeatM || repeatM == 0) { throwInvalidRepeat(); @@ -8662,15 +8662,15 @@ static const short _regex_actions[] = { } }} -#line 8333 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8333 "Parser.cpp" break; } case 322: { { -#line 1393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1393 "Parser.rl" {te = p+1;{ -#line 1393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1393 "Parser.rl" if (repeatN > repeatM || repeatM == 0) { throwInvalidRepeat(); @@ -8682,31 +8682,31 @@ static const short _regex_actions[] = { } }} -#line 8352 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8352 "Parser.cpp" break; } case 323: { { -#line 322 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 322 "Parser.rl" {te = p+1;{ -#line 322 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 322 "Parser.rl" inComment = true; {cs = 849;goto _again;}} }} -#line 8365 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8365 "Parser.cpp" break; } case 324: { { -#line 1410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1410 "Parser.rl" {te = p+1;{ -#line 1410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1410 "Parser.rl" {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -8716,27 +8716,27 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 787;goto _again;}} }} -#line 8384 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8384 "Parser.cpp" break; } case 325: { { -#line 1414 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1414 "Parser.rl" {te = p+1;{ -#line 1414 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1414 "Parser.rl" assert(0); {p += 1; goto _out; } } }} -#line 8395 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8395 "Parser.cpp" break; } case 326: { { -#line 1421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1421 "Parser.rl" {te = p+1;{ -#line 1421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1421 "Parser.rl" auto bound = mode.multiline ? ComponentBoundary::BEGIN_LINE : ComponentBoundary::BEGIN_STRING; @@ -8744,15 +8744,15 @@ static const short _regex_actions[] = { } }} -#line 8410 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8410 "Parser.cpp" break; } case 327: { { -#line 1428 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1428 "Parser.rl" {te = p+1;{ -#line 1428 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1428 "Parser.rl" auto bound = mode.multiline ? ComponentBoundary::END_LINE : ComponentBoundary::END_STRING_OPTIONAL_LF; @@ -8760,188 +8760,188 @@ static const short _regex_actions[] = { } }} -#line 8425 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8425 "Parser.cpp" break; } case 328: { { -#line 1434 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1434 "Parser.rl" {te = p+1;{ -#line 1434 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1434 "Parser.rl" auto bound = ComponentBoundary::BEGIN_STRING; currentSeq->addComponent(std::make_unique(bound)); } }} -#line 8439 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8439 "Parser.cpp" break; } case 329: { { -#line 1439 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1439 "Parser.rl" {te = p+1;{ -#line 1439 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1439 "Parser.rl" auto bound = ComponentBoundary::END_STRING_OPTIONAL_LF; currentSeq->addComponent(std::make_unique(bound)); } }} -#line 8453 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8453 "Parser.cpp" break; } case 330: { { -#line 1444 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1444 "Parser.rl" {te = p+1;{ -#line 1444 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1444 "Parser.rl" auto bound = ComponentBoundary::END_STRING; currentSeq->addComponent(std::make_unique(bound)); } }} -#line 8467 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8467 "Parser.cpp" break; } case 331: { { -#line 1449 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1449 "Parser.rl" {te = p+1;{ -#line 1449 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1449 "Parser.rl" currentSeq->addComponent( std::make_unique(ts - ptr, false, mode)); } }} -#line 8481 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8481 "Parser.cpp" break; } case 332: { { -#line 1454 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1454 "Parser.rl" {te = p+1;{ -#line 1454 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1454 "Parser.rl" currentSeq->addComponent( std::make_unique(ts - ptr, true, mode)); } }} -#line 8495 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8495 "Parser.cpp" break; } case 333: { { -#line 1464 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1464 "Parser.rl" {te = p+1;{ -#line 1464 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1464 "Parser.rl" addLiteral(currentSeq, '\x09', mode); } }} -#line 8508 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8508 "Parser.cpp" break; } case 334: { { -#line 1468 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1468 "Parser.rl" {te = p+1;{ -#line 1468 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1468 "Parser.rl" addLiteral(currentSeq, '\x0a', mode); } }} -#line 8521 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8521 "Parser.cpp" break; } case 335: { { -#line 1472 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1472 "Parser.rl" {te = p+1;{ -#line 1472 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1472 "Parser.rl" addLiteral(currentSeq, '\x0d', mode); } }} -#line 8534 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8534 "Parser.cpp" break; } case 336: { { -#line 1476 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1476 "Parser.rl" {te = p+1;{ -#line 1476 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1476 "Parser.rl" addLiteral(currentSeq, '\x0c', mode); } }} -#line 8547 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8547 "Parser.cpp" break; } case 337: { { -#line 1480 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1480 "Parser.rl" {te = p+1;{ -#line 1480 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1480 "Parser.rl" addLiteral(currentSeq, '\x07', mode); } }} -#line 8560 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8560 "Parser.cpp" break; } case 338: { { -#line 1484 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1484 "Parser.rl" {te = p+1;{ -#line 1484 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1484 "Parser.rl" addLiteral(currentSeq, '\x1b', mode); } }} -#line 8573 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8573 "Parser.cpp" break; } case 339: { { -#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1488 "Parser.rl" {te = p+1;{ -#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1488 "Parser.rl" addLiteral(currentSeq, octAccumulator, mode); } }} -#line 8586 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8586 "Parser.cpp" break; } case 340: { { -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" {te = p+1;{ -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" if (accumulator == 0) { throw LocatedParseError("Numbered reference cannot be zero"); @@ -8950,15 +8950,15 @@ static const short _regex_actions[] = { } }} -#line 8602 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8602 "Parser.cpp" break; } case 341: { { -#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 486 "Parser.rl" {te = p+1;{ -#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 486 "Parser.rl" // Accumulator is a negative offset. if (accumulator == 0) { @@ -8972,15 +8972,15 @@ static const short _regex_actions[] = { } }} -#line 8623 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8623 "Parser.cpp" break; } case 342: { { -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" {te = p+1;{ -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" if (accumulator == 0) { throw LocatedParseError("Numbered reference cannot be zero"); @@ -8989,15 +8989,15 @@ static const short _regex_actions[] = { } }} -#line 8639 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8639 "Parser.cpp" break; } case 343: { { -#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 486 "Parser.rl" {te = p+1;{ -#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 486 "Parser.rl" // Accumulator is a negative offset. if (accumulator == 0) { @@ -9011,85 +9011,85 @@ static const short _regex_actions[] = { } }} -#line 8660 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8660 "Parser.cpp" break; } case 344: { { -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" {te = p+1;{ -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" currentSeq->addComponent(std::make_unique(label)); } }} -#line 8673 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8673 "Parser.cpp" break; } case 345: { { -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" {te = p+1;{ -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" currentSeq->addComponent(std::make_unique(label)); } }} -#line 8686 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8686 "Parser.cpp" break; } case 346: { { -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" {te = p+1;{ -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" currentSeq->addComponent(std::make_unique(label)); } }} -#line 8699 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8699 "Parser.cpp" break; } case 347: { { -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" {te = p+1;{ -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" currentSeq->addComponent(std::make_unique(label)); } }} -#line 8712 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8712 "Parser.cpp" break; } case 348: { { -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" {te = p+1;{ -#line 498 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 498 "Parser.rl" currentSeq->addComponent(std::make_unique(label)); } }} -#line 8725 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8725 "Parser.cpp" break; } case 349: { { -#line 1549 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1549 "Parser.rl" {te = p+1;{ -#line 1549 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1549 "Parser.rl" ostringstream str; str << "Onigiruma subroutine call at index " << ts - ptr << @@ -9098,15 +9098,15 @@ static const short _regex_actions[] = { } }} -#line 8741 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8741 "Parser.cpp" break; } case 350: { { -#line 1560 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1560 "Parser.rl" {te = p+1;{ -#line 1560 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1560 "Parser.rl" string oct(ts + 3, te - ts - 4); unsigned long val; @@ -9122,29 +9122,29 @@ static const short _regex_actions[] = { } }} -#line 8764 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8764 "Parser.cpp" break; } case 351: { { -#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1578 "Parser.rl" {te = p+1;{ -#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1578 "Parser.rl" addEscapedHex(currentSeq, accumulator, mode); } }} -#line 8777 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8777 "Parser.cpp" break; } case 352: { { -#line 1582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1582 "Parser.rl" {te = p+1;{ -#line 1582 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1582 "Parser.rl" string hex(ts + 3, te - ts - 4); unsigned long val; @@ -9160,15 +9160,15 @@ static const short _regex_actions[] = { } }} -#line 8800 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8800 "Parser.cpp" break; } case 353: { { -#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1600 "Parser.rl" {te = p+1;{ -#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1600 "Parser.rl" if (te - ts < 3) { assert(te - ts == 2); @@ -9180,15 +9180,15 @@ static const short _regex_actions[] = { } }} -#line 8819 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8819 "Parser.cpp" break; } case 354: { { -#line 1610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1610 "Parser.rl" {te = p+1;{ -#line 1610 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1610 "Parser.rl" ostringstream str; str << "'\\" << *(ts + 1) << "' at index " << ts - ptr @@ -9197,172 +9197,172 @@ static const short _regex_actions[] = { } }} -#line 8835 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8835 "Parser.cpp" break; } case 355: { { -#line 1618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1618 "Parser.rl" {te = p+1;{ -#line 1618 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1618 "Parser.rl" auto cc = generateComponent(CLASS_WORD, false, mode); currentSeq->addComponent(move(cc)); } }} -#line 8849 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8849 "Parser.cpp" break; } case 356: { { -#line 1623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1623 "Parser.rl" {te = p+1;{ -#line 1623 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1623 "Parser.rl" auto cc = generateComponent(CLASS_WORD, true, mode); currentSeq->addComponent(move(cc)); } }} -#line 8863 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8863 "Parser.cpp" break; } case 357: { { -#line 1628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1628 "Parser.rl" {te = p+1;{ -#line 1628 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1628 "Parser.rl" auto cc = generateComponent(CLASS_SPACE, false, mode); currentSeq->addComponent(move(cc)); } }} -#line 8877 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8877 "Parser.cpp" break; } case 358: { { -#line 1633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1633 "Parser.rl" {te = p+1;{ -#line 1633 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1633 "Parser.rl" auto cc = generateComponent(CLASS_SPACE, true, mode); currentSeq->addComponent(move(cc)); } }} -#line 8891 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8891 "Parser.cpp" break; } case 359: { { -#line 1638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1638 "Parser.rl" {te = p+1;{ -#line 1638 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1638 "Parser.rl" auto cc = generateComponent(CLASS_DIGIT, false, mode); currentSeq->addComponent(move(cc)); } }} -#line 8905 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8905 "Parser.cpp" break; } case 360: { { -#line 1643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1643 "Parser.rl" {te = p+1;{ -#line 1643 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1643 "Parser.rl" auto cc = generateComponent(CLASS_DIGIT, true, mode); currentSeq->addComponent(move(cc)); } }} -#line 8919 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8919 "Parser.cpp" break; } case 361: { { -#line 1648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1648 "Parser.rl" {te = p+1;{ -#line 1648 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1648 "Parser.rl" auto cc = generateComponent(CLASS_HORZ, false, mode); currentSeq->addComponent(move(cc)); } }} -#line 8933 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8933 "Parser.cpp" break; } case 362: { { -#line 1653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1653 "Parser.rl" {te = p+1;{ -#line 1653 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1653 "Parser.rl" auto cc = generateComponent(CLASS_HORZ, true, mode); currentSeq->addComponent(move(cc)); } }} -#line 8947 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8947 "Parser.cpp" break; } case 363: { { -#line 1658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1658 "Parser.rl" {te = p+1;{ -#line 1658 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1658 "Parser.rl" auto cc = generateComponent(CLASS_VERT, false, mode); currentSeq->addComponent(move(cc)); } }} -#line 8961 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8961 "Parser.cpp" break; } case 364: { { -#line 1663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1663 "Parser.rl" {te = p+1;{ -#line 1663 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1663 "Parser.rl" auto cc = generateComponent(CLASS_VERT, true, mode); currentSeq->addComponent(move(cc)); } }} -#line 8975 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8975 "Parser.cpp" break; } case 365: { { -#line 1668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1668 "Parser.rl" {te = p+1;{ -#line 1668 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1668 "Parser.rl" assert(!currentCls && !inCharClass); currentCls = getComponentClass(mode); negated = false; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -9372,22 +9372,22 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 559;goto _again;}} }} -#line 8999 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 8999 "Parser.cpp" break; } case 366: { { -#line 1676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1676 "Parser.rl" {te = p+1;{ -#line 1676 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1676 "Parser.rl" assert(!currentCls && !inCharClass); currentCls = getComponentClass(mode); negated = false; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -9397,22 +9397,22 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 818;goto _again;}} }} -#line 9023 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9023 "Parser.cpp" break; } case 367: { { -#line 1684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1684 "Parser.rl" {te = p+1;{ -#line 1684 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1684 "Parser.rl" assert(!currentCls && !inCharClass); currentCls = getComponentClass(mode); negated = true; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -9422,22 +9422,22 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 559;goto _again;}} }} -#line 9047 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9047 "Parser.cpp" break; } case 368: { { -#line 1692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1692 "Parser.rl" {te = p+1;{ -#line 1692 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1692 "Parser.rl" assert(!currentCls && !inCharClass); currentCls = getComponentClass(mode); negated = true; {p = p - 1; } {{ -#line 1903 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1903 "Parser.rl" DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); if ((int)stack.size() == top) { @@ -9447,15 +9447,15 @@ static const short _regex_actions[] = { stack[top] = cs; top += 1;cs = 818;goto _again;}} }} -#line 9071 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9071 "Parser.cpp" break; } case 369: { { -#line 1704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1704 "Parser.rl" {te = p+1;{ -#line 1704 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1704 "Parser.rl" ostringstream str; str << "\\R at index " << ts - ptr << " not supported."; @@ -9463,15 +9463,15 @@ static const short _regex_actions[] = { } }} -#line 9086 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9086 "Parser.cpp" break; } case 370: { { -#line 1711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1711 "Parser.rl" {te = p+1;{ -#line 1711 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1711 "Parser.rl" ostringstream str; str << "\\K at index " << ts - ptr << " not supported."; @@ -9479,15 +9479,15 @@ static const short _regex_actions[] = { } }} -#line 9101 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9101 "Parser.cpp" break; } case 371: { { -#line 1726 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1726 "Parser.rl" {te = p+1;{ -#line 1726 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1726 "Parser.rl" ostringstream str; str << "\\G at index " << ts - ptr << " not supported."; @@ -9495,72 +9495,72 @@ static const short _regex_actions[] = { } }} -#line 9116 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9116 "Parser.cpp" break; } case 372: { { -#line 1732 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1732 "Parser.rl" {te = p+1;{ -#line 1732 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1732 "Parser.rl" currentSeq->addComponent(std::make_unique(ts - ptr, mode)); } }} -#line 9129 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9129 "Parser.cpp" break; } case 373: { { -#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1737 "Parser.rl" {te = p+1;{ -#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1737 "Parser.rl" addLiteral(currentSeq, *(ts + 1), mode); } }} -#line 9142 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9142 "Parser.cpp" break; } case 374: { { -#line 316 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 316 "Parser.rl" {te = p+1;{ -#line 316 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 316 "Parser.rl" inComment = true; {cs = 848;goto _again;}} }} -#line 9155 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9155 "Parser.cpp" break; } case 375: { { -#line 433 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 433 "Parser.rl" {te = p+1;{ -#line 433 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 433 "Parser.rl" mode = newMode; currentSeq->addComponent(std::make_unique()); } }} -#line 9169 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9169 "Parser.cpp" break; } case 376: { { -#line 355 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 355 "Parser.rl" {te = p+1;{ -#line 355 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 355 "Parser.rl" PUSH_SEQUENCE; mode = newMode; @@ -9569,15 +9569,15 @@ static const short _regex_actions[] = { } }} -#line 9185 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9185 "Parser.cpp" break; } case 377: { { -#line 369 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 369 "Parser.rl" {te = p+1;{ -#line 369 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 369 "Parser.rl" PUSH_SEQUENCE; currentSeq = enterSequence(currentSeq, @@ -9586,15 +9586,15 @@ static const short _regex_actions[] = { } }} -#line 9201 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9201 "Parser.cpp" break; } case 378: { { -#line 375 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 375 "Parser.rl" {te = p+1;{ -#line 375 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 375 "Parser.rl" PUSH_SEQUENCE; currentSeq = enterSequence(currentSeq, @@ -9603,15 +9603,15 @@ static const short _regex_actions[] = { } }} -#line 9217 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9217 "Parser.cpp" break; } case 379: { { -#line 381 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 381 "Parser.rl" {te = p+1;{ -#line 381 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 381 "Parser.rl" PUSH_SEQUENCE; currentSeq = enterSequence(currentSeq, @@ -9620,15 +9620,15 @@ static const short _regex_actions[] = { } }} -#line 9233 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9233 "Parser.cpp" break; } case 380: { { -#line 387 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 387 "Parser.rl" {te = p+1;{ -#line 387 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 387 "Parser.rl" PUSH_SEQUENCE; currentSeq = enterSequence(currentSeq, @@ -9637,43 +9637,43 @@ static const short _regex_actions[] = { } }} -#line 9249 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9249 "Parser.cpp" break; } case 381: { { -#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 393 "Parser.rl" {te = p+1;{ -#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 393 "Parser.rl" throw LocatedParseError("Embedded code is not supported"); } }} -#line 9262 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9262 "Parser.cpp" break; } case 382: { { -#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 393 "Parser.rl" {te = p+1;{ -#line 393 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 393 "Parser.rl" throw LocatedParseError("Embedded code is not supported"); } }} -#line 9275 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9275 "Parser.cpp" break; } case 383: { { -#line 416 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 416 "Parser.rl" {te = p+1;{ -#line 416 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 416 "Parser.rl" PUSH_SEQUENCE; currentSeq = enterSequence(currentSeq, @@ -9681,15 +9681,15 @@ static const short _regex_actions[] = { } }} -#line 9290 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9290 "Parser.cpp" break; } case 384: { { -#line 336 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 336 "Parser.rl" {te = p+1;{ -#line 336 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 336 "Parser.rl" assert(!label.empty()); // should be guaranteed by machine char c = *label.begin(); @@ -9707,43 +9707,43 @@ static const short _regex_actions[] = { } }} -#line 9315 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9315 "Parser.cpp" break; } case 385: { { -#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 399 "Parser.rl" {te = p+1;{ -#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 399 "Parser.rl" throw LocatedParseError("Subpattern reference unsupported"); } }} -#line 9328 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9328 "Parser.cpp" break; } case 386: { { -#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 399 "Parser.rl" {te = p+1;{ -#line 399 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 399 "Parser.rl" throw LocatedParseError("Subpattern reference unsupported"); } }} -#line 9341 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9341 "Parser.cpp" break; } case 387: { { -#line 1783 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1783 "Parser.rl" {te = p+1;{ -#line 1783 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1783 "Parser.rl" auto a = std::make_unique( ComponentAssertion::LOOKAHEAD, ComponentAssertion::POS); @@ -9756,15 +9756,15 @@ static const short _regex_actions[] = { } }} -#line 9361 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9361 "Parser.cpp" break; } case 388: { { -#line 1794 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1794 "Parser.rl" {te = p+1;{ -#line 1794 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1794 "Parser.rl" auto a = std::make_unique( ComponentAssertion::LOOKAHEAD, ComponentAssertion::NEG); @@ -9777,15 +9777,15 @@ static const short _regex_actions[] = { } }} -#line 9381 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9381 "Parser.cpp" break; } case 389: { { -#line 1805 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1805 "Parser.rl" {te = p+1;{ -#line 1805 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1805 "Parser.rl" auto a = std::make_unique( ComponentAssertion::LOOKBEHIND, ComponentAssertion::POS); @@ -9798,15 +9798,15 @@ static const short _regex_actions[] = { } }} -#line 9401 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9401 "Parser.cpp" break; } case 390: { { -#line 1816 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1816 "Parser.rl" {te = p+1;{ -#line 1816 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1816 "Parser.rl" auto a = std::make_unique( ComponentAssertion::LOOKBEHIND, ComponentAssertion::NEG); @@ -9819,29 +9819,29 @@ static const short _regex_actions[] = { } }} -#line 9421 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9421 "Parser.cpp" break; } case 391: { { -#line 1828 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1828 "Parser.rl" {te = p+1;{ -#line 1828 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1828 "Parser.rl" throw LocatedParseError("Pattern recursion not supported"); } }} -#line 9434 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9434 "Parser.cpp" break; } case 392: { { -#line 402 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 402 "Parser.rl" {te = p+1;{ -#line 402 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 402 "Parser.rl" if (accumulator == 0) { throw LocatedParseError("Numbered reference cannot be zero"); @@ -9852,15 +9852,15 @@ static const short _regex_actions[] = { } }} -#line 9452 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9452 "Parser.cpp" break; } case 393: { { -#line 410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 410 "Parser.rl" {te = p+1;{ -#line 410 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 410 "Parser.rl" PUSH_SEQUENCE; assert(!label.empty()); @@ -9869,15 +9869,15 @@ static const short _regex_actions[] = { } }} -#line 9468 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9468 "Parser.cpp" break; } case 394: { { -#line 1844 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1844 "Parser.rl" {te = p+1;{ -#line 1844 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1844 "Parser.rl" ostringstream str; str << "Callout at index " << ts - ptr << " not supported."; @@ -9885,29 +9885,29 @@ static const short _regex_actions[] = { } }} -#line 9483 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9483 "Parser.cpp" break; } case 395: { { -#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1852 "Parser.rl" {te = p+1;{ -#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1852 "Parser.rl" throw LocatedParseError("Unrecognised character after (?"); } }} -#line 9496 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9496 "Parser.cpp" break; } case 396: { { -#line 1857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1857 "Parser.rl" {te = p+1;{ -#line 1857 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1857 "Parser.rl" assert(mode.utf8); /* leverage ComponentClass to generate the vertices */ @@ -9918,15 +9918,15 @@ static const short _regex_actions[] = { } }} -#line 9514 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9514 "Parser.cpp" break; } case 397: { { -#line 1866 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1866 "Parser.rl" {te = p+1;{ -#line 1866 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1866 "Parser.rl" assert(mode.utf8); /* leverage ComponentClass to generate the vertices */ @@ -9937,15 +9937,15 @@ static const short _regex_actions[] = { } }} -#line 9532 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9532 "Parser.cpp" break; } case 398: { { -#line 1875 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1875 "Parser.rl" {te = p+1;{ -#line 1875 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1875 "Parser.rl" assert(mode.utf8); /* leverage ComponentClass to generate the vertices */ @@ -9956,30 +9956,30 @@ static const short _regex_actions[] = { } }} -#line 9550 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9550 "Parser.cpp" break; } case 399: { { -#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1884 "Parser.rl" {te = p+1;{ -#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1884 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 9564 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9564 "Parser.cpp" break; } case 400: { { -#line 1893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1893 "Parser.rl" {te = p+1;{ -#line 1893 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1893 "Parser.rl" if (mode.ignore_space == false) { addLiteral(currentSeq, *ts, mode); @@ -9987,29 +9987,29 @@ static const short _regex_actions[] = { } }} -#line 9579 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9579 "Parser.cpp" break; } case 401: { { -#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1898 "Parser.rl" {te = p+1;{ -#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1898 "Parser.rl" addLiteral(currentSeq, *ts, mode); } }} -#line 9592 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9592 "Parser.cpp" break; } case 402: { { -#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 328 "Parser.rl" {te = p;p = p - 1;{ -#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 328 "Parser.rl" PUSH_SEQUENCE; auto seq = std::make_unique(); @@ -10018,15 +10018,15 @@ static const short _regex_actions[] = { } }} -#line 9608 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9608 "Parser.cpp" break; } case 403: { { -#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 421 "Parser.rl" {te = p;p = p - 1;{ -#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 421 "Parser.rl" assert(!currentCls); assert(!inCharClass); // not reentrant @@ -10037,15 +10037,15 @@ static const short _regex_actions[] = { {cs = 836;goto _again;}} }} -#line 9626 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9626 "Parser.cpp" break; } case 404: { { -#line 1310 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1310 "Parser.rl" {te = p;p = p - 1;{ -#line 1310 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1310 "Parser.rl" if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, ComponentRepeat::REPEAT_GREEDY)) { @@ -10054,15 +10054,15 @@ static const short _regex_actions[] = { } }} -#line 9642 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9642 "Parser.cpp" break; } case 405: { { -#line 1331 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1331 "Parser.rl" {te = p;p = p - 1;{ -#line 1331 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1331 "Parser.rl" if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, ComponentRepeat::REPEAT_GREEDY)) { @@ -10071,15 +10071,15 @@ static const short _regex_actions[] = { } }} -#line 9658 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9658 "Parser.cpp" break; } case 406: { { -#line 1352 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1352 "Parser.rl" {te = p;p = p - 1;{ -#line 1352 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1352 "Parser.rl" if (!currentSeq->addRepeat( 0, 1, ComponentRepeat::REPEAT_GREEDY)) { @@ -10088,15 +10088,15 @@ static const short _regex_actions[] = { } }} -#line 9674 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9674 "Parser.cpp" break; } case 407: { { -#line 1373 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1373 "Parser.rl" {te = p;p = p - 1;{ -#line 1373 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1373 "Parser.rl" if (repeatN > repeatM || repeatM == 0) { throwInvalidRepeat(); @@ -10108,29 +10108,29 @@ static const short _regex_actions[] = { } }} -#line 9693 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9693 "Parser.cpp" break; } case 408: { { -#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1488 "Parser.rl" {te = p;p = p - 1;{ -#line 1488 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1488 "Parser.rl" addLiteral(currentSeq, octAccumulator, mode); } }} -#line 9706 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9706 "Parser.cpp" break; } case 409: { { -#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1491 "Parser.rl" {te = p;p = p - 1;{ -#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1491 "Parser.rl" // If there are enough capturing sub expressions, this may be // a back reference @@ -10143,15 +10143,15 @@ static const short _regex_actions[] = { } }} -#line 9726 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9726 "Parser.cpp" break; } case 410: { { -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" {te = p;p = p - 1;{ -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" if (accumulator == 0) { throw LocatedParseError("Numbered reference cannot be zero"); @@ -10160,15 +10160,15 @@ static const short _regex_actions[] = { } }} -#line 9742 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9742 "Parser.cpp" break; } case 411: { { -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" {te = p;p = p - 1;{ -#line 479 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 479 "Parser.rl" if (accumulator == 0) { throw LocatedParseError("Numbered reference cannot be zero"); @@ -10177,15 +10177,15 @@ static const short _regex_actions[] = { } }} -#line 9758 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9758 "Parser.cpp" break; } case 412: { { -#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 486 "Parser.rl" {te = p;p = p - 1;{ -#line 486 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 486 "Parser.rl" // Accumulator is a negative offset. if (accumulator == 0) { @@ -10199,71 +10199,71 @@ static const short _regex_actions[] = { } }} -#line 9779 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9779 "Parser.cpp" break; } case 413: { { -#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1557 "Parser.rl" {te = p;p = p - 1;{ -#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1557 "Parser.rl" throw LocatedParseError("Invalid reference after \\g"); } }} -#line 9792 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9792 "Parser.cpp" break; } case 414: { { -#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1574 "Parser.rl" {te = p;p = p - 1;{ -#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1574 "Parser.rl" throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); } }} -#line 9805 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9805 "Parser.cpp" break; } case 415: { { -#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1578 "Parser.rl" {te = p;p = p - 1;{ -#line 1578 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1578 "Parser.rl" addEscapedHex(currentSeq, accumulator, mode); } }} -#line 9818 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9818 "Parser.cpp" break; } case 416: { { -#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1596 "Parser.rl" {te = p;p = p - 1;{ -#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1596 "Parser.rl" throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); } }} -#line 9831 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9831 "Parser.cpp" break; } case 417: { { -#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1600 "Parser.rl" {te = p;p = p - 1;{ -#line 1600 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1600 "Parser.rl" if (te - ts < 3) { assert(te - ts == 2); @@ -10275,39 +10275,39 @@ static const short _regex_actions[] = { } }} -#line 9850 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9850 "Parser.cpp" break; } case 418: { { -#line 1700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1700 "Parser.rl" {te = p;p = p - 1;{ -#line 1700 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1700 "Parser.rl" throw LocatedParseError("Malformed property"); } }} -#line 9861 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9861 "Parser.cpp" break; } case 419: { { -#line 1701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1701 "Parser.rl" {te = p;p = p - 1;{ -#line 1701 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1701 "Parser.rl" throw LocatedParseError("Malformed property"); } }} -#line 9872 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9872 "Parser.cpp" break; } case 420: { { -#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1719 "Parser.rl" {te = p;p = p - 1;{ -#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1719 "Parser.rl" ostringstream str; str << "\\k at index " << ts - ptr << " not supported."; @@ -10315,15 +10315,15 @@ static const short _regex_actions[] = { } }} -#line 9887 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9887 "Parser.cpp" break; } case 421: { { -#line 1742 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1742 "Parser.rl" {te = p;p = p - 1;{ -#line 1742 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1742 "Parser.rl" assert(ts + 1 == pe); ostringstream str; @@ -10332,73 +10332,73 @@ static const short _regex_actions[] = { } }} -#line 9903 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9903 "Parser.cpp" break; } case 422: { { -#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 396 "Parser.rl" {te = p;p = p - 1;{ -#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 396 "Parser.rl" throw LocatedParseError("Conditional subpattern unsupported"); } }} -#line 9916 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9916 "Parser.cpp" break; } case 423: { { -#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1852 "Parser.rl" {te = p;p = p - 1;{ -#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1852 "Parser.rl" throw LocatedParseError("Unrecognised character after (?"); } }} -#line 9929 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9929 "Parser.cpp" break; } case 424: { { -#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1884 "Parser.rl" {te = p;p = p - 1;{ -#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1884 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 9943 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9943 "Parser.cpp" break; } case 425: { { -#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1898 "Parser.rl" {te = p;p = p - 1;{ -#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1898 "Parser.rl" addLiteral(currentSeq, *ts, mode); } }} -#line 9956 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9956 "Parser.cpp" break; } case 426: { { -#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 328 "Parser.rl" {p = ((te))-1; { -#line 328 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 328 "Parser.rl" PUSH_SEQUENCE; auto seq = std::make_unique(); @@ -10407,16 +10407,16 @@ static const short _regex_actions[] = { } }} -#line 9973 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9973 "Parser.cpp" break; } case 427: { { -#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 421 "Parser.rl" {p = ((te))-1; { -#line 421 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 421 "Parser.rl" assert(!currentCls); assert(!inCharClass); // not reentrant @@ -10427,61 +10427,61 @@ static const short _regex_actions[] = { {cs = 836;goto _again;}} }} -#line 9992 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 9992 "Parser.cpp" break; } case 428: { { -#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1557 "Parser.rl" {p = ((te))-1; { -#line 1557 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1557 "Parser.rl" throw LocatedParseError("Invalid reference after \\g"); } }} -#line 10006 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10006 "Parser.cpp" break; } case 429: { { -#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1574 "Parser.rl" {p = ((te))-1; { -#line 1574 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1574 "Parser.rl" throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); } }} -#line 10020 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10020 "Parser.cpp" break; } case 430: { { -#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1596 "Parser.rl" {p = ((te))-1; { -#line 1596 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1596 "Parser.rl" throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); } }} -#line 10034 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10034 "Parser.cpp" break; } case 431: { { -#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1719 "Parser.rl" {p = ((te))-1; { -#line 1719 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1719 "Parser.rl" ostringstream str; str << "\\k at index " << ts - ptr << " not supported."; @@ -10489,68 +10489,68 @@ static const short _regex_actions[] = { } }} -#line 10050 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10050 "Parser.cpp" break; } case 432: { { -#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 396 "Parser.rl" {p = ((te))-1; { -#line 396 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 396 "Parser.rl" throw LocatedParseError("Conditional subpattern unsupported"); } }} -#line 10064 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10064 "Parser.cpp" break; } case 433: { { -#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1852 "Parser.rl" {p = ((te))-1; { -#line 1852 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1852 "Parser.rl" throw LocatedParseError("Unrecognised character after (?"); } }} -#line 10078 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10078 "Parser.cpp" break; } case 434: { { -#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1884 "Parser.rl" {p = ((te))-1; { -#line 1884 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1884 "Parser.rl" assert(mode.utf8); throwInvalidUtf8(); } }} -#line 10093 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10093 "Parser.cpp" break; } case 435: { { -#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1898 "Parser.rl" {p = ((te))-1; { -#line 1898 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1898 "Parser.rl" addLiteral(currentSeq, *ts, mode); } }} -#line 10107 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10107 "Parser.cpp" break; } @@ -10561,7 +10561,7 @@ static const short _regex_actions[] = { case 288: { p = ((te))-1; { -#line 1491 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1491 "Parser.rl" // If there are enough capturing sub expressions, this may be // a back reference @@ -10577,7 +10577,7 @@ static const short _regex_actions[] = { case 290: { p = ((te))-1; { -#line 1508 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1508 "Parser.rl" // if there are enough left parens to this point, back ref if (accumulator < groupIndex) { @@ -10614,7 +10614,7 @@ static const short _regex_actions[] = { case 330: { p = ((te))-1; { -#line 1737 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1737 "Parser.rl" addLiteral(currentSeq, *(ts + 1), mode); } @@ -10623,7 +10623,7 @@ static const short _regex_actions[] = { }} } -#line 10179 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10179 "Parser.cpp" break; } @@ -10650,7 +10650,7 @@ else { #line 1 "NONE" {ts = 0;}} -#line 10205 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/Parser.cpp" +#line 10205 "Parser.cpp" break; } @@ -10667,7 +10667,7 @@ else { _out: {} } -#line 1983 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/Parser.rl" +#line 1983 "Parser.rl" if (p != pe && *p != '\0') { diff --git a/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp b/contrib/vectorscan-cmake/rageled_files/aarch64/control_verbs.cpp similarity index 83% rename from contrib/vectorscan-cmake/rageled_files/control_verbs.cpp rename to contrib/vectorscan-cmake/rageled_files/aarch64/control_verbs.cpp index edcfaec0c97..6204fc0a314 100644 --- a/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp +++ b/contrib/vectorscan-cmake/rageled_files/aarch64/control_verbs.cpp @@ -1,4 +1,4 @@ -#line 1 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 1 "control_verbs.rl" /* * Copyright (c) 2017, Intel Corporation * @@ -54,7 +54,7 @@ namespace ue2 { UNUSED int act; -#line 56 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 56 "control_verbs.cpp" static const signed char _ControlVerbs_actions[] = { 0, 1, 0, 1, 1, 1, 2, 1, 3, 1, 4, 1, 5, 1, 6, 1, @@ -269,19 +269,19 @@ static const signed char _ControlVerbs_actions[] = { static const int ControlVerbs_en_main = 75; -#line 269 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 269 "control_verbs.cpp" { cs = (int)ControlVerbs_start; ts = 0; te = 0; } -#line 105 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 105 "control_verbs.rl" try { -#line 278 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 278 "control_verbs.cpp" { int _klen; unsigned int _trans = 0; @@ -301,7 +301,7 @@ static const signed char _ControlVerbs_actions[] = { #line 1 "NONE" {ts = p;}} -#line 297 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 297 "control_verbs.cpp" break; } @@ -383,43 +383,43 @@ static const signed char _ControlVerbs_actions[] = { #line 1 "NONE" {te = p+1;}} -#line 378 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 378 "control_verbs.cpp" break; } case 3: { { -#line 76 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 76 "control_verbs.rl" {te = p+1;{ -#line 76 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 76 "control_verbs.rl" mode.utf8 = true; } }} -#line 391 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 391 "control_verbs.cpp" break; } case 4: { { -#line 80 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 80 "control_verbs.rl" {te = p+1;{ -#line 80 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 80 "control_verbs.rl" mode.ucp = true; } }} -#line 404 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 404 "control_verbs.cpp" break; } case 5: { { -#line 84 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 84 "control_verbs.rl" {te = p+1;{ -#line 84 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 84 "control_verbs.rl" ostringstream str; str << "Unsupported control verb " << string(ts, te - ts); @@ -427,15 +427,15 @@ static const signed char _ControlVerbs_actions[] = { } }} -#line 419 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 419 "control_verbs.cpp" break; } case 6: { { -#line 90 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 90 "control_verbs.rl" {te = p+1;{ -#line 90 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 90 "control_verbs.rl" ostringstream str; str << "Unknown control verb " << string(ts, te - ts); @@ -443,53 +443,53 @@ static const signed char _ControlVerbs_actions[] = { } }} -#line 434 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 434 "control_verbs.cpp" break; } case 7: { { -#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 97 "control_verbs.rl" {te = p+1;{ -#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 97 "control_verbs.rl" {p = p - 1; } {p += 1; goto _out; } } }} -#line 448 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 448 "control_verbs.cpp" break; } case 8: { { -#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 97 "control_verbs.rl" {te = p;p = p - 1;{ -#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 97 "control_verbs.rl" {p = p - 1; } {p += 1; goto _out; } } }} -#line 462 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 462 "control_verbs.cpp" break; } case 9: { { -#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 97 "control_verbs.rl" {p = ((te))-1; { -#line 97 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 97 "control_verbs.rl" {p = p - 1; } {p += 1; goto _out; } } }} -#line 477 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 477 "control_verbs.cpp" break; } @@ -515,7 +515,7 @@ static const signed char _ControlVerbs_actions[] = { #line 1 "NONE" {ts = 0;}} -#line 502 "../../repo/ClickHouse1/contrib/vectorscan-cmake/rageled_files/control_verbs.cpp" +#line 502 "control_verbs.cpp" break; } @@ -530,7 +530,7 @@ static const signed char _ControlVerbs_actions[] = { _out: {} } -#line 108 "../../repo/ClickHouse1/contrib/vectorscan/src/parser/control_verbs.rl" +#line 108 "control_verbs.rl" } catch (LocatedParseError &error) { if (ts >= ptr && ts <= pe) { diff --git a/contrib/vectorscan-cmake/rageled_files/amd64/Parser.cpp b/contrib/vectorscan-cmake/rageled_files/amd64/Parser.cpp new file mode 100644 index 00000000000..ffda1515582 --- /dev/null +++ b/contrib/vectorscan-cmake/rageled_files/amd64/Parser.cpp @@ -0,0 +1,10725 @@ +#line 1 "Parser.rl" +/* +* Copyright (c) 2015-2017, Intel Corporation +* +* Redistribution and use in source and binary forms, with or without +* modification, are permitted provided that the following conditions are met: +* +* * Redistributions of source code must retain the above copyright notice, +* this list of conditions and the following disclaimer. +* * Redistributions in binary form must reproduce the above copyright +* notice, this list of conditions and the following disclaimer in the +* documentation and/or other materials provided with the distribution. +* * Neither the name of Intel Corporation nor the names of its contributors +* may be used to endorse or promote products derived from this software +* without specific prior written permission. +* +* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +* POSSIBILITY OF SUCH DAMAGE. +*/ + +/** \file +* \brief Parser code (generated with Ragel from Parser.rl). +*/ + +#include "config.h" + +/* Parser.cpp is a built source, may not be in same dir as parser files */ +#include "parser/check_refs.h" +#include "parser/control_verbs.h" +#include "parser/ComponentAlternation.h" +#include "parser/ComponentAssertion.h" +#include "parser/ComponentAtomicGroup.h" +#include "parser/ComponentBackReference.h" +#include "parser/ComponentBoundary.h" +#include "parser/ComponentByte.h" +#include "parser/ComponentClass.h" +#include "parser/ComponentCondReference.h" +#include "parser/ComponentEmpty.h" +#include "parser/ComponentEUS.h" +#include "parser/Component.h" +#include "parser/ComponentRepeat.h" +#include "parser/ComponentSequence.h" +#include "parser/ComponentWordBoundary.h" +#include "parser/parse_error.h" +#include "parser/Parser.h" +#include "ue2common.h" +#include "util/compare.h" +#include "util/flat_containers.h" +#include "util/unicode_def.h" +#include "util/verify_types.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace std; + +namespace ue2 { + +#define PUSH_SEQUENCE do {\ + sequences.push_back(ExprState(currentSeq, (size_t)(ts - ptr), \ + mode)); \ + } while(0) +#define POP_SEQUENCE do {\ + currentSeq = sequences.back().seq; \ + mode = sequences.back().mode; \ + sequences.pop_back(); \ + } while(0) + + namespace { + + /** \brief Structure representing current state as we're parsing (current + * sequence, current options). Stored in the 'sequences' vector. */ + struct ExprState { + ExprState(ComponentSequence *seq_in, size_t offset, + const ParseMode &mode_in) : + seq(seq_in), seqOffset(offset), mode(mode_in) {} + + ComponentSequence *seq; //!< current sequence + size_t seqOffset; //!< offset seq was entered, for error reporting + ParseMode mode; //!< current mode flags + }; + + } // namespace + + static + unsigned parseAsDecimal(unsigned oct) { + // The input was parsed as octal, but should have been parsed as decimal. + // Deconstruct the octal number and reconstruct into decimal + unsigned ret = 0; + unsigned multiplier = 1; + while (oct) { + ret += (oct & 0x7) * multiplier; + oct >>= 3; + multiplier *= 10; + } + return ret; + } + + /** \brief Maximum value for a positive integer. We use INT_MAX, as that's what + * PCRE uses. */ + static constexpr u32 MAX_NUMBER = INT_MAX; + + static + void pushDec(u32 *acc, char raw_digit) { + assert(raw_digit >= '0' && raw_digit <= '9'); + u32 digit_val = raw_digit - '0'; + + // Ensure that we don't overflow. + u64a val = ((u64a)*acc * 10) + digit_val; + if (val > MAX_NUMBER) { + throw LocatedParseError("Number is too big"); + } + + *acc = verify_u32(val); + } + + static + void pushOct(u32 *acc, char raw_digit) { + assert(raw_digit >= '0' && raw_digit <= '7'); + u32 digit_val = raw_digit - '0'; + + // Ensure that we don't overflow. + u64a val = ((u64a)*acc * 8) + digit_val; + if (val > MAX_NUMBER) { + throw LocatedParseError("Number is too big"); + } + + *acc = verify_u32(val); + } + + static + void throwInvalidRepeat(void) { + throw LocatedParseError("Invalid repeat"); + } + + static + void throwInvalidUtf8(void) { + throw ParseError("Expression is not valid UTF-8."); + } + + /** + * Adds the given child component to the parent sequence, returning a pointer + * to the new (child) "current sequence". + */ + static + ComponentSequence *enterSequence(ComponentSequence *parent, + unique_ptr child) { + assert(parent); + assert(child); + + ComponentSequence *seq = child.get(); + parent->addComponent(move(child)); + return seq; + } + + static + void addLiteral(ComponentSequence *currentSeq, char c, const ParseMode &mode) { + if (mode.utf8 && mode.caseless) { + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + assert(cc); + cc->add(c); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } else { + currentSeq->addComponent(getLiteralComponentClass(c, mode.caseless)); + } + } + + static + void addEscaped(ComponentSequence *currentSeq, unichar accum, + const ParseMode &mode, const char *err_msg) { + if (mode.utf8) { + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + assert(cc); + cc->add(accum); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } else { + if (accum > 255) { + throw LocatedParseError(err_msg); + } + addLiteral(currentSeq, (char)accum, mode); + } + } + + static + void addEscapedOctal(ComponentSequence *currentSeq, unichar accum, + const ParseMode &mode) { + addEscaped(currentSeq, accum, mode, "Octal value is greater than \\377"); + } + + static + void addEscapedHex(ComponentSequence *currentSeq, unichar accum, + const ParseMode &mode) { + addEscaped(currentSeq, accum, mode, + "Hexadecimal value is greater than \\xFF"); + } + +#define SLASH_C_ERROR "\\c must be followed by an ASCII character" + + static + u8 decodeCtrl(char raw) { + if (raw & 0x80) { + throw LocatedParseError(SLASH_C_ERROR); + } + return mytoupper(raw) ^ 0x40; + } + + static + unichar readUtf8CodePoint2c(const char *s) { + auto *ts = (const u8 *)s; + assert(ts[0] >= 0xc0 && ts[0] < 0xe0); + assert(ts[1] >= 0x80 && ts[1] < 0xc0); + unichar val = ts[0] & 0x1f; + val <<= 6; + val |= ts[1] & 0x3f; + DEBUG_PRINTF("utf8 %02hhx %02hhx ->\\x{%x}\n", ts[0], + ts[1], val); + return val; + } + + static + unichar readUtf8CodePoint3c(const char *s) { + auto *ts = (const u8 *)s; + assert(ts[0] >= 0xe0 && ts[0] < 0xf0); + assert(ts[1] >= 0x80 && ts[1] < 0xc0); + assert(ts[2] >= 0x80 && ts[2] < 0xc0); + unichar val = ts[0] & 0x0f; + val <<= 6; + val |= ts[1] & 0x3f; + val <<= 6; + val |= ts[2] & 0x3f; + DEBUG_PRINTF("utf8 %02hhx %02hhx %02hhx ->\\x{%x}\n", ts[0], + ts[1], ts[2], val); + return val; + } + + static + unichar readUtf8CodePoint4c(const char *s) { + auto *ts = (const u8 *)s; + assert(ts[0] >= 0xf0 && ts[0] < 0xf8); + assert(ts[1] >= 0x80 && ts[1] < 0xc0); + assert(ts[2] >= 0x80 && ts[2] < 0xc0); + assert(ts[3] >= 0x80 && ts[3] < 0xc0); + unichar val = ts[0] & 0x07; + val <<= 6; + val |= ts[1] & 0x3f; + val <<= 6; + val |= ts[2] & 0x3f; + val <<= 6; + val |= ts[3] & 0x3f; + DEBUG_PRINTF("utf8 %02hhx %02hhx %02hhx %02hhx ->\\x{%x}\n", ts[0], + ts[1], ts[2], ts[3], val); + return val; + } + + +#line 1909 "Parser.rl" + + + +#line 277 "Parser.cpp" +static const short _regex_actions[] = { + 0, 1, 0, 1, 1, 1, 2, 1, + 3, 1, 4, 1, 7, 1, 8, 1, + 9, 1, 10, 1, 11, 1, 12, 1, + 13, 1, 15, 1, 16, 1, 17, 1, + 18, 1, 19, 1, 20, 1, 21, 1, + 22, 1, 23, 1, 24, 1, 25, 1, + 26, 1, 27, 1, 28, 1, 29, 1, + 30, 1, 31, 1, 32, 1, 33, 1, + 34, 1, 35, 1, 36, 1, 37, 1, + 38, 1, 39, 1, 40, 1, 41, 1, + 42, 1, 43, 1, 44, 1, 45, 1, + 46, 1, 47, 1, 48, 1, 49, 1, + 50, 1, 51, 1, 52, 1, 53, 1, + 54, 1, 55, 1, 56, 1, 57, 1, + 58, 1, 59, 1, 60, 1, 61, 1, + 62, 1, 63, 1, 64, 1, 65, 1, + 66, 1, 67, 1, 68, 1, 69, 1, + 70, 1, 71, 1, 72, 1, 73, 1, + 74, 1, 75, 1, 76, 1, 77, 1, + 78, 1, 79, 1, 80, 1, 81, 1, + 82, 1, 83, 1, 84, 1, 85, 1, + 86, 1, 87, 1, 88, 1, 89, 1, + 90, 1, 91, 1, 92, 1, 93, 1, + 94, 1, 95, 1, 96, 1, 97, 1, + 98, 1, 99, 1, 100, 1, 101, 1, + 102, 1, 103, 1, 104, 1, 105, 1, + 106, 1, 107, 1, 108, 1, 109, 1, + 110, 1, 111, 1, 112, 1, 113, 1, + 114, 1, 115, 1, 116, 1, 117, 1, + 118, 1, 119, 1, 120, 1, 121, 1, + 122, 1, 123, 1, 124, 1, 125, 1, + 126, 1, 127, 1, 128, 1, 129, 1, + 130, 1, 131, 1, 132, 1, 133, 1, + 134, 1, 135, 1, 136, 1, 137, 1, + 138, 1, 139, 1, 140, 1, 141, 1, + 142, 1, 143, 1, 144, 1, 145, 1, + 146, 1, 147, 1, 148, 1, 149, 1, + 150, 1, 151, 1, 152, 1, 153, 1, + 154, 1, 155, 1, 156, 1, 157, 1, + 158, 1, 159, 1, 160, 1, 161, 1, + 162, 1, 163, 1, 164, 1, 165, 1, + 166, 1, 167, 1, 168, 1, 169, 1, + 170, 1, 171, 1, 172, 1, 173, 1, + 174, 1, 175, 1, 176, 1, 177, 1, + 178, 1, 179, 1, 180, 1, 181, 1, + 182, 1, 183, 1, 184, 1, 185, 1, + 186, 1, 187, 1, 188, 1, 189, 1, + 190, 1, 191, 1, 192, 1, 193, 1, + 194, 1, 195, 1, 196, 1, 197, 1, + 198, 1, 199, 1, 200, 1, 201, 1, + 202, 1, 203, 1, 204, 1, 205, 1, + 206, 1, 207, 1, 208, 1, 209, 1, + 210, 1, 211, 1, 212, 1, 213, 1, + 214, 1, 215, 1, 216, 1, 217, 1, + 218, 1, 219, 1, 220, 1, 221, 1, + 222, 1, 223, 1, 224, 1, 225, 1, + 226, 1, 227, 1, 228, 1, 229, 1, + 230, 1, 231, 1, 232, 1, 233, 1, + 234, 1, 235, 1, 236, 1, 237, 1, + 240, 1, 242, 1, 243, 1, 244, 1, + 245, 1, 246, 1, 247, 1, 248, 1, + 249, 1, 250, 1, 251, 1, 252, 1, + 253, 1, 254, 1, 255, 1, 256, 1, + 257, 1, 258, 1, 259, 1, 260, 1, + 261, 1, 262, 1, 263, 1, 264, 1, + 265, 1, 266, 1, 267, 1, 268, 1, + 269, 1, 270, 1, 271, 1, 272, 1, + 273, 1, 274, 1, 275, 1, 276, 1, + 277, 1, 278, 1, 279, 1, 280, 1, + 281, 1, 282, 1, 283, 1, 284, 1, + 285, 1, 286, 1, 287, 1, 288, 1, + 289, 1, 290, 1, 291, 1, 292, 1, + 293, 1, 294, 1, 295, 1, 296, 1, + 297, 1, 298, 1, 299, 1, 300, 1, + 301, 1, 302, 1, 303, 1, 307, 1, + 308, 1, 309, 1, 310, 1, 311, 1, + 312, 1, 313, 1, 314, 1, 315, 1, + 316, 1, 317, 1, 318, 1, 319, 1, + 320, 1, 321, 1, 322, 1, 323, 1, + 324, 1, 325, 1, 326, 1, 327, 1, + 328, 1, 329, 1, 330, 1, 331, 1, + 332, 1, 333, 1, 334, 1, 335, 1, + 336, 1, 337, 1, 338, 1, 342, 1, + 343, 1, 344, 1, 345, 1, 346, 1, + 347, 1, 348, 1, 349, 1, 350, 1, + 352, 1, 353, 1, 354, 1, 355, 1, + 356, 1, 357, 1, 358, 1, 359, 1, + 360, 1, 361, 1, 362, 1, 363, 1, + 364, 1, 365, 1, 366, 1, 367, 1, + 368, 1, 369, 1, 370, 1, 371, 1, + 372, 1, 373, 1, 374, 1, 375, 1, + 376, 1, 377, 1, 378, 1, 379, 1, + 380, 1, 381, 1, 382, 1, 383, 1, + 384, 1, 385, 1, 386, 1, 387, 1, + 388, 1, 389, 1, 390, 1, 391, 1, + 392, 1, 393, 1, 394, 1, 395, 1, + 396, 1, 397, 1, 398, 1, 399, 1, + 400, 1, 401, 1, 402, 1, 403, 1, + 404, 1, 405, 1, 406, 1, 407, 1, + 408, 1, 409, 1, 410, 1, 411, 1, + 412, 1, 413, 1, 414, 1, 415, 1, + 416, 1, 417, 1, 418, 1, 419, 1, + 420, 1, 421, 1, 422, 1, 423, 1, + 424, 1, 425, 1, 426, 1, 427, 1, + 428, 1, 429, 1, 430, 1, 431, 1, + 432, 1, 433, 1, 434, 1, 435, 1, + 436, 2, 3, 0, 2, 4, 5, 2, + 5, 1, 2, 9, 10, 2, 9, 238, + 2, 9, 239, 2, 9, 339, 2, 10, + 1, 2, 10, 340, 2, 10, 341, 2, + 11, 241, 2, 11, 351, 2, 12, 241, + 2, 12, 351, 2, 13, 241, 2, 13, + 351, 2, 14, 375, 2, 14, 376, 2, + 25, 0, 2, 25, 3, 2, 25, 6, + 2, 25, 14, 3, 25, 5, 306, 3, + 25, 10, 305, 3, 25, 14, 15, 4, + 25, 9, 304, 10, 0 + }; + + static const short _regex_key_offsets[] = { + 0, 0, 2, 4, 6, 7, 29, 37, + 45, 52, 60, 61, 69, 77, 85, 92, + 100, 103, 105, 114, 121, 129, 137, 140, + 146, 154, 157, 164, 171, 179, 186, 190, + 197, 200, 203, 205, 208, 211, 213, 216, + 219, 221, 222, 224, 225, 233, 235, 238, + 241, 242, 250, 258, 266, 274, 281, 289, + 296, 304, 311, 319, 321, 324, 331, 335, + 338, 341, 342, 344, 345, 347, 349, 350, + 351, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 369, + 370, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 384, 385, + 386, 387, 388, 389, 390, 392, 393, 394, + 395, 396, 397, 399, 400, 401, 402, 403, + 404, 405, 406, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 429, + 430, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 443, 444, 445, + 446, 447, 448, 450, 451, 452, 453, 454, + 455, 456, 457, 458, 459, 461, 462, 463, + 464, 465, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, + 512, 513, 514, 515, 516, 517, 519, 520, + 521, 522, 523, 524, 525, 526, 527, 528, + 529, 530, 531, 532, 533, 534, 535, 536, + 537, 538, 539, 540, 541, 542, 543, 544, + 545, 546, 547, 548, 549, 550, 551, 552, + 553, 554, 555, 556, 557, 558, 559, 561, + 562, 563, 564, 565, 566, 567, 568, 569, + 570, 571, 572, 573, 574, 575, 576, 577, + 578, 579, 580, 582, 583, 584, 585, 586, + 587, 588, 589, 590, 591, 592, 593, 594, + 595, 596, 597, 601, 602, 603, 604, 605, + 606, 607, 608, 609, 610, 611, 612, 613, + 614, 615, 616, 617, 618, 620, 621, 622, + 623, 624, 625, 626, 627, 628, 629, 631, + 632, 633, 634, 635, 636, 637, 640, 641, + 642, 643, 644, 645, 646, 647, 648, 650, + 651, 652, 653, 654, 655, 656, 658, 659, + 660, 661, 662, 663, 664, 665, 666, 667, + 668, 669, 670, 671, 672, 673, 674, 675, + 676, 677, 678, 679, 680, 681, 682, 683, + 684, 685, 686, 687, 688, 689, 690, 691, + 692, 693, 694, 695, 696, 697, 698, 699, + 700, 701, 702, 704, 705, 706, 707, 708, + 709, 710, 714, 715, 716, 717, 718, 719, + 720, 721, 722, 723, 724, 725, 726, 727, + 728, 729, 730, 731, 732, 733, 734, 735, + 736, 737, 738, 739, 740, 741, 742, 743, + 744, 745, 746, 747, 748, 749, 750, 752, + 753, 754, 755, 756, 757, 758, 759, 760, + 761, 762, 763, 764, 765, 766, 767, 768, + 769, 770, 771, 773, 774, 775, 776, 777, + 778, 779, 780, 781, 782, 783, 784, 785, + 786, 787, 788, 789, 790, 791, 792, 793, + 794, 795, 796, 797, 798, 799, 800, 801, + 802, 803, 805, 806, 807, 808, 809, 810, + 811, 812, 813, 814, 815, 816, 817, 820, + 822, 823, 824, 825, 826, 827, 828, 829, + 830, 833, 834, 835, 836, 837, 838, 839, + 840, 841, 842, 843, 844, 845, 846, 847, + 849, 850, 851, 853, 854, 855, 856, 857, + 858, 859, 860, 861, 862, 863, 864, 865, + 866, 867, 868, 869, 870, 871, 872, 873, + 874, 875, 876, 877, 879, 881, 883, 886, + 889, 891, 906, 909, 912, 914, 928, 933, + 938, 942, 946, 949, 952, 956, 960, 963, + 966, 970, 974, 978, 981, 984, 988, 992, + 996, 1000, 1003, 1006, 1010, 1014, 1018, 1022, + 1025, 1028, 1032, 1036, 1040, 1044, 1047, 1050, + 1054, 1058, 1062, 1066, 1069, 1072, 1076, 1080, + 1084, 1088, 1091, 1094, 1099, 1103, 1107, 1111, + 1114, 1117, 1121, 1125, 1129, 1132, 1135, 1139, + 1143, 1147, 1151, 1154, 1157, 1161, 1165, 1169, + 1173, 1176, 1179, 1183, 1187, 1191, 1194, 1197, + 1201, 1205, 1209, 1213, 1217, 1220, 1223, 1228, + 1233, 1237, 1241, 1244, 1247, 1251, 1255, 1258, + 1261, 1265, 1269, 1273, 1276, 1279, 1283, 1287, + 1291, 1295, 1298, 1301, 1305, 1309, 1313, 1317, + 1320, 1323, 1327, 1331, 1335, 1339, 1342, 1345, + 1349, 1353, 1357, 1361, 1364, 1367, 1371, 1375, + 1379, 1383, 1386, 1389, 1394, 1398, 1402, 1406, + 1409, 1412, 1416, 1420, 1424, 1427, 1430, 1434, + 1438, 1442, 1446, 1449, 1452, 1456, 1460, 1464, + 1468, 1471, 1474, 1478, 1482, 1486, 1489, 1492, + 1496, 1500, 1504, 1508, 1512, 1515, 1518, 1521, + 1524, 1526, 1528, 1531, 1538, 1540, 1542, 1544, + 1546, 1548, 1550, 1577, 1579, 1581, 1583, 1585, + 1592, 1599, 1613, 1615, 1621, 1624, 1633, 1634, + 1637, 1640, 1647, 1649, 1651, 1653, 1656, 1701, + 1703, 1705, 1709, 1713, 1715, 1716, 1716, 1722, + 1724, 1726, 1728, 1730, 1733, 1734, 1735, 1742, + 1748, 1754, 1756, 1758, 1760, 1761, 1764, 1787, + 1790, 1795, 1804, 1806, 1807, 1809, 1814, 1817, + 1819, 1821, 1822, 1824, 1834, 1840, 1841, 1846, + 1850, 1858, 1860, 1869, 1873, 1874, 1875, 1879, + 1880, 1883, 1883, 1890, 1904, 1906, 1908, 1910, + 1913, 1952, 1954, 1956, 1958, 1960, 1961, 1961, + 1962, 1963, 1970, 1976, 1982, 1985, 1987, 1998, + 2000, 2002, 2004, 2005, 2016, 2018, 2020, 2022, + 2023, 2024, 0 + }; + + static const char _regex_trans_keys[] = { + -128, -65, -128, -65, -128, -65, 41, 33, + 35, 38, 39, 40, 41, 43, 45, 58, + 60, 61, 62, 63, 67, 80, 105, 109, + 115, 120, 123, 48, 57, 41, 95, 48, + 57, 65, 90, 97, 122, 39, 95, 48, + 57, 65, 90, 97, 122, 95, 48, 57, + 65, 90, 97, 122, 39, 95, 48, 57, + 65, 90, 97, 122, 41, 41, 95, 48, + 57, 65, 90, 97, 122, 41, 95, 48, + 57, 65, 90, 97, 122, 41, 95, 48, + 57, 65, 90, 97, 122, 95, 48, 57, + 65, 90, 97, 122, 62, 95, 48, 57, + 65, 90, 97, 122, 33, 60, 61, 33, + 61, 38, 41, 95, 48, 57, 65, 90, + 97, 122, 95, 48, 57, 65, 90, 97, + 122, 41, 95, 48, 57, 65, 90, 97, + 122, 41, 95, 48, 57, 65, 90, 97, + 122, 41, 48, 57, 41, 58, 105, 109, + 115, 120, 62, 95, 48, 57, 65, 90, + 97, 122, 41, 48, 57, 95, 48, 57, + 65, 90, 97, 122, 95, 48, 57, 65, + 90, 97, 122, 41, 95, 48, 57, 65, + 90, 97, 122, 95, 48, 57, 65, 90, + 97, 122, 105, 109, 115, 120, 41, 45, + 58, 105, 109, 115, 120, 46, 92, 93, + 46, 92, 93, 46, 92, 58, 92, 93, + 58, 92, 93, 58, 92, 61, 92, 93, + 61, 92, 93, 61, 92, 39, 48, 57, + 62, 45, 95, 48, 57, 65, 90, 97, + 122, 48, 57, 125, 48, 57, 125, 48, + 57, 125, 95, 125, 48, 57, 65, 90, + 97, 122, 95, 125, 48, 57, 65, 90, + 97, 122, 95, 125, 48, 57, 65, 90, + 97, 122, 95, 125, 48, 57, 65, 90, + 97, 122, 95, 48, 57, 65, 90, 97, + 122, 39, 95, 48, 57, 65, 90, 97, + 122, 95, 48, 57, 65, 90, 97, 122, + 62, 95, 48, 57, 65, 90, 97, 122, + 95, 48, 57, 65, 90, 97, 122, 95, + 125, 48, 57, 65, 90, 97, 122, 48, + 55, 125, 48, 55, 125, 48, 57, 65, + 70, 97, 102, 44, 125, 48, 57, 125, + 48, 57, 125, 48, 57, 41, 41, 80, + 41, 41, 70, 41, 56, 41, 121, 97, + 109, 98, 105, 99, 101, 110, 105, 97, + 110, 101, 115, 116, 97, 110, 108, 109, + 116, 105, 110, 101, 115, 101, 117, 109, + 97, 107, 110, 103, 97, 108, 105, 112, + 111, 109, 111, 102, 111, 97, 104, 105, + 109, 105, 108, 108, 101, 103, 104, 105, + 110, 101, 115, 101, 105, 100, 110, 114, + 97, 100, 105, 97, 110, 95, 65, 98, + 111, 114, 105, 103, 105, 110, 97, 108, + 105, 97, 110, 97, 101, 109, 114, 111, + 107, 101, 101, 109, 111, 110, 116, 105, + 99, 110, 101, 105, 102, 111, 114, 109, + 112, 114, 114, 105, 111, 116, 105, 108, + 108, 105, 99, 115, 118, 101, 114, 101, + 116, 97, 110, 97, 103, 97, 114, 105, + 121, 112, 116, 105, 97, 110, 95, 72, + 105, 101, 114, 111, 103, 108, 121, 112, + 104, 115, 104, 105, 111, 112, 105, 99, + 111, 114, 103, 105, 97, 110, 97, 103, + 111, 108, 105, 116, 105, 99, 116, 104, + 105, 99, 101, 101, 107, 106, 114, 97, + 114, 97, 116, 105, 109, 117, 107, 104, + 105, 110, 117, 108, 110, 111, 111, 98, + 114, 101, 119, 114, 97, 103, 97, 110, + 97, 112, 101, 114, 105, 97, 108, 95, + 65, 114, 97, 109, 97, 105, 99, 104, + 115, 101, 114, 105, 116, 101, 100, 99, + 114, 105, 112, 116, 105, 111, 110, 97, + 108, 95, 80, 97, 104, 114, 108, 97, + 118, 105, 116, 104, 105, 97, 110, 118, + 97, 110, 101, 115, 101, 105, 110, 116, + 121, 116, 104, 105, 110, 97, 100, 97, + 97, 107, 97, 110, 97, 97, 104, 95, + 76, 105, 97, 109, 114, 111, 115, 104, + 116, 104, 105, 101, 114, 111, 116, 105, + 110, 112, 99, 104, 97, 109, 110, 115, + 98, 117, 101, 97, 114, 95, 66, 117, + 99, 100, 105, 97, 110, 105, 97, 110, + 108, 110, 97, 121, 97, 108, 97, 109, + 100, 97, 105, 99, 116, 101, 105, 95, + 77, 97, 121, 101, 107, 110, 103, 111, + 108, 105, 97, 110, 97, 110, 109, 97, + 114, 119, 95, 84, 97, 105, 95, 76, + 117, 101, 111, 104, 97, 109, 95, 100, + 67, 104, 105, 107, 105, 95, 73, 80, + 83, 84, 116, 97, 108, 105, 99, 101, + 114, 115, 105, 97, 110, 111, 117, 116, + 104, 95, 65, 114, 97, 98, 105, 97, + 110, 117, 114, 107, 105, 99, 105, 121, + 97, 109, 97, 110, 121, 97, 97, 111, + 103, 115, 95, 80, 97, 101, 110, 105, + 99, 105, 97, 110, 106, 97, 110, 103, + 110, 105, 99, 109, 117, 97, 114, 105, + 116, 97, 110, 114, 97, 115, 104, 116, + 114, 97, 97, 118, 105, 97, 110, 110, + 104, 97, 108, 97, 110, 100, 97, 110, + 101, 115, 101, 108, 114, 111, 116, 105, + 95, 78, 97, 103, 114, 105, 105, 97, + 99, 103, 105, 109, 97, 98, 108, 111, + 103, 97, 110, 119, 97, 95, 76, 84, + 86, 101, 104, 97, 109, 105, 101, 116, + 105, 108, 108, 117, 103, 117, 97, 97, + 105, 110, 97, 98, 102, 101, 116, 97, + 110, 105, 110, 97, 103, 104, 97, 114, + 105, 116, 105, 99, 105, 110, 115, 112, + 100, 123, 94, 125, 94, -128, -65, -128, + -65, -128, -65, 46, 92, 93, 46, 92, + 93, 46, 92, 58, 92, 93, 94, 97, + 98, 99, 100, 103, 108, 112, 115, 117, + 119, 120, 58, 92, 93, 58, 92, 93, + 58, 92, 58, 92, 93, 97, 98, 99, + 100, 103, 108, 112, 115, 117, 119, 120, + 58, 92, 93, 108, 115, 58, 92, 93, + 110, 112, 58, 92, 93, 117, 58, 92, + 93, 109, 58, 92, 93, 58, 92, 93, + 58, 92, 93, 104, 58, 92, 93, 97, + 58, 92, 93, 58, 92, 93, 58, 92, + 93, 99, 58, 92, 93, 105, 58, 92, + 93, 105, 58, 92, 93, 58, 92, 93, + 58, 92, 93, 108, 58, 92, 93, 97, + 58, 92, 93, 110, 58, 92, 93, 107, + 58, 92, 93, 58, 92, 93, 58, 92, + 93, 110, 58, 92, 93, 116, 58, 92, + 93, 114, 58, 92, 93, 108, 58, 92, + 93, 58, 92, 93, 58, 92, 93, 105, + 58, 92, 93, 103, 58, 92, 93, 105, + 58, 92, 93, 116, 58, 92, 93, 58, + 92, 93, 58, 92, 93, 114, 58, 92, + 93, 97, 58, 92, 93, 112, 58, 92, + 93, 104, 58, 92, 93, 58, 92, 93, + 58, 92, 93, 111, 58, 92, 93, 119, + 58, 92, 93, 101, 58, 92, 93, 114, + 58, 92, 93, 58, 92, 93, 58, 92, + 93, 114, 117, 58, 92, 93, 105, 58, + 92, 93, 110, 58, 92, 93, 116, 58, + 92, 93, 58, 92, 93, 58, 92, 93, + 110, 58, 92, 93, 99, 58, 92, 93, + 116, 58, 92, 93, 58, 92, 93, 58, + 92, 93, 112, 58, 92, 93, 97, 58, + 92, 93, 99, 58, 92, 93, 101, 58, + 92, 93, 58, 92, 93, 58, 92, 93, + 112, 58, 92, 93, 112, 58, 92, 93, + 101, 58, 92, 93, 114, 58, 92, 93, + 58, 92, 93, 58, 92, 93, 111, 58, + 92, 93, 114, 58, 92, 93, 100, 58, + 92, 93, 58, 92, 93, 58, 92, 93, + 100, 58, 92, 93, 105, 58, 92, 93, + 103, 58, 92, 93, 105, 58, 92, 93, + 116, 58, 92, 93, 58, 92, 93, 58, + 92, 93, 108, 115, 58, 92, 93, 110, + 112, 58, 92, 93, 117, 58, 92, 93, + 109, 58, 92, 93, 58, 92, 93, 58, + 92, 93, 104, 58, 92, 93, 97, 58, + 92, 93, 58, 92, 93, 58, 92, 93, + 99, 58, 92, 93, 105, 58, 92, 93, + 105, 58, 92, 93, 58, 92, 93, 58, + 92, 93, 108, 58, 92, 93, 97, 58, + 92, 93, 110, 58, 92, 93, 107, 58, + 92, 93, 58, 92, 93, 58, 92, 93, + 110, 58, 92, 93, 116, 58, 92, 93, + 114, 58, 92, 93, 108, 58, 92, 93, + 58, 92, 93, 58, 92, 93, 105, 58, + 92, 93, 103, 58, 92, 93, 105, 58, + 92, 93, 116, 58, 92, 93, 58, 92, + 93, 58, 92, 93, 114, 58, 92, 93, + 97, 58, 92, 93, 112, 58, 92, 93, + 104, 58, 92, 93, 58, 92, 93, 58, + 92, 93, 111, 58, 92, 93, 119, 58, + 92, 93, 101, 58, 92, 93, 114, 58, + 92, 93, 58, 92, 93, 58, 92, 93, + 114, 117, 58, 92, 93, 105, 58, 92, + 93, 110, 58, 92, 93, 116, 58, 92, + 93, 58, 92, 93, 58, 92, 93, 110, + 58, 92, 93, 99, 58, 92, 93, 116, + 58, 92, 93, 58, 92, 93, 58, 92, + 93, 112, 58, 92, 93, 97, 58, 92, + 93, 99, 58, 92, 93, 101, 58, 92, + 93, 58, 92, 93, 58, 92, 93, 112, + 58, 92, 93, 112, 58, 92, 93, 101, + 58, 92, 93, 114, 58, 92, 93, 58, + 92, 93, 58, 92, 93, 111, 58, 92, + 93, 114, 58, 92, 93, 100, 58, 92, + 93, 58, 92, 93, 58, 92, 93, 100, + 58, 92, 93, 105, 58, 92, 93, 103, + 58, 92, 93, 105, 58, 92, 93, 116, + 58, 92, 93, 58, 92, 93, 61, 92, + 93, 61, 92, 93, 61, 92, 48, 55, + 125, 48, 55, 125, 48, 57, 65, 70, + 97, 102, -128, -65, -128, -65, -128, -65, + -128, -65, -128, -65, -128, -65, 0, 32, + 35, 36, 40, 41, 42, 43, 46, 63, + 91, 92, 94, 123, 124, -128, -65, -64, + -33, -32, -17, -16, -9, -8, -1, 9, + 13, -128, -65, -128, -65, -128, -65, 42, + 63, 95, 48, 57, 65, 90, 97, 122, + 95, 48, 57, 65, 90, 97, 122, 39, + 48, 60, 63, 82, 95, 49, 55, 56, + 57, 65, 90, 97, 122, 48, 57, 105, + 109, 115, 120, 48, 57, 41, 48, 57, + 33, 61, 95, 48, 57, 65, 90, 97, + 122, 123, 41, 48, 57, 60, 61, 62, + 41, 45, 58, 105, 109, 115, 120, 43, + 63, 43, 63, 43, 63, 46, 58, 61, + 48, 65, 66, 67, 68, 69, 71, 72, + 75, 76, 78, 80, 81, 82, 83, 85, + 86, 87, 88, 90, 97, 98, 99, 100, + 101, 102, 103, 104, 107, 108, 110, 111, + 112, 114, 115, 116, 117, 118, 119, 120, + 122, 49, 55, 56, 57, 48, 55, 48, + 55, 48, 55, 56, 57, 48, 55, 56, + 57, 48, 57, 123, 39, 45, 60, 123, + 48, 57, 48, 57, 48, 57, 48, 57, + 48, 57, 39, 60, 123, 123, 123, 123, + 48, 57, 65, 70, 97, 102, 48, 57, + 65, 70, 97, 102, 48, 57, 65, 70, + 97, 102, 48, 57, 43, 63, 41, 85, + 41, 41, 67, 84, 65, 66, 67, 68, + 69, 71, 72, 73, 74, 75, 76, 77, + 78, 79, 80, 82, 83, 84, 85, 86, + 88, 89, 90, 110, 114, 118, 97, 101, + 111, 114, 117, 97, 99, 102, 104, 110, + 111, 115, 117, 121, 109, 112, 101, 103, + 116, 101, 108, 111, 114, 117, 97, 101, + 105, 103, 117, 109, 110, 97, 97, 104, + 38, 97, 101, 105, 108, 109, 111, 116, + 117, 121, 97, 99, 101, 110, 111, 121, + 101, 100, 101, 107, 108, 111, 103, 108, + 114, 115, 99, 100, 101, 102, 104, 105, + 111, 115, 101, 117, 97, 99, 104, 105, + 107, 109, 111, 117, 121, 97, 101, 104, + 105, 103, 97, 97, 112, 115, 119, 105, + 108, 112, 115, 67, 76, 77, 78, 80, + 83, 90, 45, 91, 92, 93, -128, -65, + -64, -33, -32, -17, -16, -9, -8, -1, + -128, -65, -128, -65, -128, -65, 46, 58, + 61, 48, 68, 69, 72, 76, 78, 80, + 81, 83, 85, 86, 87, 97, 98, 99, + 100, 101, 102, 103, 104, 108, 110, 111, + 112, 114, 115, 116, 117, 118, 119, 120, + 49, 55, 56, 57, 65, 90, 105, 122, + 48, 55, 48, 55, 48, 55, 48, 55, + 123, 123, 123, 123, 48, 57, 65, 70, + 97, 102, 48, 57, 65, 70, 97, 102, + 48, 57, 65, 70, 97, 102, 92, 93, + 94, 69, 81, 92, -128, -65, -64, -33, + -32, -17, -16, -9, -8, -1, -128, -65, + -128, -65, -128, -65, 69, 92, -128, -65, + -64, -33, -32, -17, -16, -9, -8, -1, + -128, -65, -128, -65, -128, -65, 69, 41, + 10, 0 + }; + + static const signed char _regex_single_lengths[] = { + 0, 0, 0, 0, 1, 20, 2, 2, + 1, 2, 1, 2, 2, 2, 1, 2, + 3, 2, 3, 1, 2, 2, 1, 6, + 2, 1, 1, 1, 2, 1, 4, 7, + 3, 3, 2, 3, 3, 2, 3, 3, + 2, 1, 0, 1, 2, 0, 1, 1, + 1, 2, 2, 2, 2, 1, 2, 1, + 2, 1, 2, 0, 1, 1, 2, 1, + 1, 1, 2, 1, 2, 2, 1, 1, + 2, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 3, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 2, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 4, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 3, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 2, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 4, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 2, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 3, 2, + 1, 1, 1, 1, 1, 1, 1, 1, + 3, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 2, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 0, 0, 0, 3, 3, + 2, 15, 3, 3, 2, 14, 5, 5, + 4, 4, 3, 3, 4, 4, 3, 3, + 4, 4, 4, 3, 3, 4, 4, 4, + 4, 3, 3, 4, 4, 4, 4, 3, + 3, 4, 4, 4, 4, 3, 3, 4, + 4, 4, 4, 3, 3, 4, 4, 4, + 4, 3, 3, 5, 4, 4, 4, 3, + 3, 4, 4, 4, 3, 3, 4, 4, + 4, 4, 3, 3, 4, 4, 4, 4, + 3, 3, 4, 4, 4, 3, 3, 4, + 4, 4, 4, 4, 3, 3, 5, 5, + 4, 4, 3, 3, 4, 4, 3, 3, + 4, 4, 4, 3, 3, 4, 4, 4, + 4, 3, 3, 4, 4, 4, 4, 3, + 3, 4, 4, 4, 4, 3, 3, 4, + 4, 4, 4, 3, 3, 4, 4, 4, + 4, 3, 3, 5, 4, 4, 4, 3, + 3, 4, 4, 4, 3, 3, 4, 4, + 4, 4, 3, 3, 4, 4, 4, 4, + 3, 3, 4, 4, 4, 3, 3, 4, + 4, 4, 4, 4, 3, 3, 3, 3, + 2, 0, 1, 1, 0, 0, 0, 0, + 0, 0, 15, 0, 0, 0, 2, 1, + 1, 6, 0, 4, 1, 3, 1, 1, + 3, 7, 2, 2, 2, 3, 41, 0, + 0, 0, 0, 0, 1, 0, 4, 0, + 0, 0, 0, 3, 1, 1, 1, 0, + 0, 0, 2, 2, 1, 3, 23, 3, + 5, 9, 2, 1, 2, 5, 3, 2, + 2, 1, 2, 10, 6, 1, 5, 4, + 8, 2, 9, 4, 1, 1, 4, 1, + 3, 0, 7, 4, 0, 0, 0, 3, + 31, 0, 0, 0, 0, 1, 0, 1, + 1, 1, 0, 0, 3, 2, 1, 0, + 0, 0, 1, 1, 0, 0, 0, 1, + 1, 1, 0 + }; + + static const signed char _regex_range_lengths[] = { + 0, 1, 1, 1, 0, 1, 3, 3, + 3, 3, 0, 3, 3, 3, 3, 3, + 0, 0, 3, 3, 3, 3, 1, 0, + 3, 1, 3, 3, 3, 3, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1, 0, 3, 1, 1, 1, + 0, 3, 3, 3, 3, 3, 3, 3, + 3, 3, 3, 1, 1, 3, 1, 1, + 1, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 1, 1, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1, 1, 3, 1, 1, 1, 1, + 1, 1, 6, 1, 1, 1, 0, 3, + 3, 4, 1, 1, 1, 3, 0, 1, + 0, 0, 0, 0, 0, 0, 2, 1, + 1, 2, 2, 1, 0, 0, 1, 1, + 1, 1, 1, 0, 0, 0, 3, 3, + 3, 1, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 5, 1, 1, 1, 0, + 4, 1, 1, 1, 1, 0, 0, 0, + 0, 3, 3, 3, 0, 0, 5, 1, + 1, 1, 0, 5, 1, 1, 1, 0, + 0, 0, 0 + }; + + static const short _regex_index_offsets[] = { + 0, 0, 2, 4, 6, 8, 30, 36, + 42, 47, 53, 55, 61, 67, 73, 78, + 84, 88, 91, 98, 103, 109, 115, 118, + 125, 131, 134, 139, 144, 150, 155, 160, + 168, 172, 176, 179, 183, 187, 190, 194, + 198, 201, 203, 205, 207, 213, 215, 218, + 221, 223, 229, 235, 241, 247, 252, 258, + 263, 269, 274, 280, 282, 285, 290, 294, + 297, 300, 302, 305, 307, 310, 313, 315, + 317, 320, 322, 324, 326, 328, 330, 332, + 334, 336, 338, 340, 342, 344, 346, 350, + 352, 354, 356, 358, 360, 362, 364, 366, + 368, 370, 372, 374, 376, 378, 380, 382, + 384, 386, 388, 390, 392, 395, 397, 399, + 401, 403, 405, 408, 410, 412, 414, 416, + 418, 420, 422, 425, 427, 429, 431, 433, + 435, 437, 439, 441, 443, 445, 447, 449, + 451, 453, 455, 457, 459, 461, 463, 466, + 468, 470, 472, 474, 476, 478, 480, 482, + 484, 486, 488, 490, 492, 494, 496, 498, + 500, 502, 504, 507, 509, 511, 513, 515, + 517, 519, 521, 523, 525, 528, 530, 532, + 534, 536, 538, 540, 542, 544, 546, 548, + 550, 552, 554, 556, 558, 560, 562, 564, + 566, 568, 570, 572, 574, 576, 578, 580, + 582, 584, 586, 588, 590, 592, 594, 596, + 598, 600, 602, 604, 606, 608, 610, 612, + 614, 616, 618, 620, 622, 624, 626, 628, + 630, 632, 634, 636, 638, 640, 643, 645, + 647, 649, 651, 653, 655, 657, 659, 661, + 663, 665, 667, 669, 671, 673, 675, 677, + 679, 681, 683, 685, 687, 689, 691, 693, + 695, 697, 699, 701, 703, 705, 707, 709, + 711, 713, 715, 717, 719, 721, 723, 726, + 728, 730, 732, 734, 736, 738, 740, 742, + 744, 746, 748, 750, 752, 754, 756, 758, + 760, 762, 764, 767, 769, 771, 773, 775, + 777, 779, 781, 783, 785, 787, 789, 791, + 793, 795, 797, 802, 804, 806, 808, 810, + 812, 814, 816, 818, 820, 822, 824, 826, + 828, 830, 832, 834, 836, 839, 841, 843, + 845, 847, 849, 851, 853, 855, 857, 860, + 862, 864, 866, 868, 870, 872, 876, 878, + 880, 882, 884, 886, 888, 890, 892, 895, + 897, 899, 901, 903, 905, 907, 910, 912, + 914, 916, 918, 920, 922, 924, 926, 928, + 930, 932, 934, 936, 938, 940, 942, 944, + 946, 948, 950, 952, 954, 956, 958, 960, + 962, 964, 966, 968, 970, 972, 974, 976, + 978, 980, 982, 984, 986, 988, 990, 992, + 994, 996, 998, 1001, 1003, 1005, 1007, 1009, + 1011, 1013, 1018, 1020, 1022, 1024, 1026, 1028, + 1030, 1032, 1034, 1036, 1038, 1040, 1042, 1044, + 1046, 1048, 1050, 1052, 1054, 1056, 1058, 1060, + 1062, 1064, 1066, 1068, 1070, 1072, 1074, 1076, + 1078, 1080, 1082, 1084, 1086, 1088, 1090, 1093, + 1095, 1097, 1099, 1101, 1103, 1105, 1107, 1109, + 1111, 1113, 1115, 1117, 1119, 1121, 1123, 1125, + 1127, 1129, 1131, 1134, 1136, 1138, 1140, 1142, + 1144, 1146, 1148, 1150, 1152, 1154, 1156, 1158, + 1160, 1162, 1164, 1166, 1168, 1170, 1172, 1174, + 1176, 1178, 1180, 1182, 1184, 1186, 1188, 1190, + 1192, 1194, 1197, 1199, 1201, 1203, 1205, 1207, + 1209, 1211, 1213, 1215, 1217, 1219, 1221, 1225, + 1228, 1230, 1232, 1234, 1236, 1238, 1240, 1242, + 1244, 1248, 1250, 1252, 1254, 1256, 1258, 1260, + 1262, 1264, 1266, 1268, 1270, 1272, 1274, 1276, + 1279, 1281, 1283, 1286, 1288, 1290, 1292, 1294, + 1296, 1298, 1300, 1302, 1304, 1306, 1308, 1310, + 1312, 1314, 1316, 1318, 1320, 1322, 1324, 1326, + 1328, 1330, 1332, 1334, 1336, 1338, 1340, 1344, + 1348, 1351, 1367, 1371, 1375, 1378, 1393, 1399, + 1405, 1410, 1415, 1419, 1423, 1428, 1433, 1437, + 1441, 1446, 1451, 1456, 1460, 1464, 1469, 1474, + 1479, 1484, 1488, 1492, 1497, 1502, 1507, 1512, + 1516, 1520, 1525, 1530, 1535, 1540, 1544, 1548, + 1553, 1558, 1563, 1568, 1572, 1576, 1581, 1586, + 1591, 1596, 1600, 1604, 1610, 1615, 1620, 1625, + 1629, 1633, 1638, 1643, 1648, 1652, 1656, 1661, + 1666, 1671, 1676, 1680, 1684, 1689, 1694, 1699, + 1704, 1708, 1712, 1717, 1722, 1727, 1731, 1735, + 1740, 1745, 1750, 1755, 1760, 1764, 1768, 1774, + 1780, 1785, 1790, 1794, 1798, 1803, 1808, 1812, + 1816, 1821, 1826, 1831, 1835, 1839, 1844, 1849, + 1854, 1859, 1863, 1867, 1872, 1877, 1882, 1887, + 1891, 1895, 1900, 1905, 1910, 1915, 1919, 1923, + 1928, 1933, 1938, 1943, 1947, 1951, 1956, 1961, + 1966, 1971, 1975, 1979, 1985, 1990, 1995, 2000, + 2004, 2008, 2013, 2018, 2023, 2027, 2031, 2036, + 2041, 2046, 2051, 2055, 2059, 2064, 2069, 2074, + 2079, 2083, 2087, 2092, 2097, 2102, 2106, 2110, + 2115, 2120, 2125, 2130, 2135, 2139, 2143, 2147, + 2151, 2154, 2156, 2159, 2164, 2166, 2168, 2170, + 2172, 2174, 2176, 2198, 2200, 2202, 2204, 2207, + 2212, 2217, 2228, 2230, 2236, 2239, 2246, 2248, + 2251, 2255, 2263, 2266, 2269, 2272, 2276, 2320, + 2322, 2324, 2327, 2330, 2332, 2334, 2335, 2341, + 2343, 2345, 2347, 2349, 2353, 2355, 2357, 2362, + 2366, 2370, 2372, 2375, 2378, 2380, 2384, 2408, + 2412, 2418, 2428, 2431, 2433, 2436, 2442, 2446, + 2449, 2452, 2454, 2457, 2468, 2475, 2477, 2483, + 2488, 2497, 2500, 2510, 2515, 2517, 2519, 2524, + 2526, 2530, 2531, 2539, 2549, 2551, 2553, 2555, + 2559, 2595, 2597, 2599, 2601, 2603, 2605, 2606, + 2608, 2610, 2615, 2619, 2623, 2627, 2630, 2637, + 2639, 2641, 2643, 2645, 2652, 2654, 2656, 2658, + 2660, 2662, 0 + }; + + static const signed char _regex_trans_cond_spaces[] = { + 0, -1, 0, -1, 0, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 0, -1, + 0, -1, 0, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 0, -1, 0, -1, + 0, -1, 0, -1, 0, -1, 0, -1, + -1, -1, 1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 0, + 0, 0, 0, 0, -1, -1, 0, -1, + 0, -1, 0, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 0, + 0, 0, 0, 0, -1, 0, -1, 0, + -1, 0, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + 2, 2, -1, -1, -1, -1, -1, 0, + 0, 0, 0, 0, -1, 0, -1, 0, + -1, 0, -1, -1, -1, -1, 0, 0, + 0, 0, 0, -1, 0, -1, 0, -1, + 0, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 0 + }; + + static const short _regex_trans_offsets[] = { + 0, 2, 3, 5, 6, 8, 9, 10, + 11, 12, 13, 14, 15, 16, 17, 18, + 19, 20, 21, 22, 23, 24, 25, 26, + 27, 28, 29, 30, 31, 32, 33, 34, + 35, 36, 37, 38, 39, 40, 41, 42, + 43, 44, 45, 46, 47, 48, 49, 50, + 51, 52, 53, 54, 55, 56, 57, 58, + 59, 60, 61, 62, 63, 64, 65, 66, + 67, 68, 69, 70, 71, 72, 73, 74, + 75, 76, 77, 78, 79, 80, 81, 82, + 83, 84, 85, 86, 87, 88, 89, 90, + 91, 92, 93, 94, 95, 96, 97, 98, + 99, 100, 101, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, 119, 120, 121, 122, + 123, 124, 125, 126, 127, 128, 129, 130, + 131, 132, 133, 134, 135, 136, 137, 138, + 139, 140, 141, 142, 143, 144, 145, 146, + 147, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, 161, 162, + 163, 164, 165, 166, 167, 168, 169, 170, + 171, 172, 173, 174, 175, 176, 177, 178, + 179, 180, 181, 182, 183, 184, 185, 186, + 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, 198, 199, 200, 201, 202, + 203, 204, 205, 206, 207, 208, 209, 210, + 211, 212, 213, 214, 215, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, + 227, 228, 229, 230, 231, 232, 233, 234, + 235, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, + 259, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 277, 278, 279, 280, 281, 282, + 283, 284, 285, 286, 287, 288, 289, 290, + 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, 321, 322, + 323, 324, 325, 326, 327, 328, 329, 330, + 331, 332, 333, 334, 335, 336, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, + 443, 444, 445, 446, 447, 448, 449, 450, + 451, 452, 453, 454, 455, 456, 457, 458, + 459, 460, 461, 462, 463, 464, 465, 466, + 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 501, 502, 503, 504, 505, 506, + 507, 508, 509, 510, 511, 512, 513, 514, + 515, 516, 517, 518, 519, 520, 521, 522, + 523, 524, 525, 526, 527, 528, 529, 530, + 531, 532, 533, 534, 535, 536, 537, 538, + 539, 540, 541, 542, 543, 544, 545, 546, + 547, 548, 549, 550, 551, 552, 553, 554, + 555, 556, 557, 558, 559, 560, 561, 562, + 563, 564, 565, 566, 567, 568, 569, 570, + 571, 572, 573, 574, 575, 576, 577, 578, + 579, 580, 581, 582, 583, 584, 585, 586, + 587, 588, 589, 590, 591, 592, 593, 594, + 595, 596, 597, 598, 599, 600, 601, 602, + 603, 604, 605, 606, 607, 608, 609, 610, + 611, 612, 613, 614, 615, 616, 617, 618, + 619, 620, 621, 622, 623, 624, 625, 626, + 627, 628, 629, 630, 631, 632, 633, 634, + 635, 636, 637, 638, 639, 640, 641, 642, + 643, 644, 645, 646, 647, 648, 649, 650, + 651, 652, 653, 654, 655, 656, 657, 658, + 659, 660, 661, 662, 663, 664, 665, 666, + 667, 668, 669, 670, 671, 672, 673, 674, + 675, 676, 677, 678, 679, 680, 681, 682, + 683, 684, 685, 686, 687, 688, 689, 690, + 691, 692, 693, 694, 695, 696, 697, 698, + 699, 700, 701, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 713, 714, + 715, 716, 717, 718, 719, 720, 721, 722, + 723, 724, 725, 726, 727, 728, 729, 730, + 731, 732, 733, 734, 735, 736, 737, 738, + 739, 740, 741, 742, 743, 744, 745, 746, + 747, 748, 749, 750, 751, 752, 753, 754, + 755, 756, 757, 758, 759, 760, 761, 762, + 763, 764, 765, 766, 767, 768, 769, 770, + 771, 772, 773, 774, 775, 776, 777, 778, + 779, 780, 781, 782, 783, 784, 785, 786, + 787, 788, 789, 790, 791, 792, 793, 794, + 795, 796, 797, 798, 799, 800, 801, 802, + 803, 804, 805, 806, 807, 808, 809, 810, + 811, 812, 813, 814, 815, 816, 817, 818, + 819, 820, 821, 822, 823, 824, 825, 826, + 827, 828, 829, 830, 831, 832, 833, 834, + 835, 836, 837, 838, 839, 840, 841, 842, + 843, 844, 845, 846, 847, 848, 849, 850, + 851, 852, 853, 854, 855, 856, 857, 858, + 859, 860, 861, 862, 863, 864, 865, 866, + 867, 868, 869, 870, 871, 872, 873, 874, + 875, 876, 877, 878, 879, 880, 881, 882, + 883, 884, 885, 886, 887, 888, 889, 890, + 891, 892, 893, 894, 895, 896, 897, 898, + 899, 900, 901, 902, 903, 904, 905, 906, + 907, 908, 909, 910, 911, 912, 913, 914, + 915, 916, 917, 918, 919, 920, 921, 922, + 923, 924, 925, 926, 927, 928, 929, 930, + 931, 932, 933, 934, 935, 936, 937, 938, + 939, 940, 941, 942, 943, 944, 945, 946, + 947, 948, 949, 950, 951, 952, 953, 954, + 955, 956, 957, 958, 959, 960, 961, 962, + 963, 964, 965, 966, 967, 968, 969, 970, + 971, 972, 973, 974, 975, 976, 977, 978, + 979, 980, 981, 982, 983, 984, 985, 986, + 987, 988, 989, 990, 991, 992, 993, 994, + 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, + 1011, 1012, 1013, 1014, 1015, 1016, 1017, 1018, + 1019, 1020, 1021, 1022, 1023, 1024, 1025, 1026, + 1027, 1028, 1029, 1030, 1031, 1032, 1033, 1034, + 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, + 1043, 1044, 1045, 1046, 1047, 1048, 1049, 1050, + 1051, 1052, 1053, 1054, 1055, 1056, 1057, 1058, + 1059, 1060, 1061, 1062, 1063, 1064, 1065, 1066, + 1067, 1068, 1069, 1070, 1071, 1072, 1073, 1074, + 1075, 1076, 1077, 1078, 1079, 1080, 1081, 1082, + 1083, 1084, 1085, 1086, 1087, 1088, 1089, 1090, + 1091, 1092, 1093, 1094, 1095, 1096, 1097, 1098, + 1099, 1100, 1101, 1102, 1103, 1104, 1105, 1106, + 1107, 1108, 1109, 1110, 1111, 1112, 1113, 1114, + 1115, 1116, 1117, 1118, 1119, 1120, 1121, 1122, + 1123, 1124, 1125, 1126, 1127, 1128, 1129, 1130, + 1131, 1132, 1133, 1134, 1135, 1136, 1137, 1138, + 1139, 1140, 1141, 1142, 1143, 1144, 1145, 1146, + 1147, 1148, 1149, 1150, 1151, 1152, 1153, 1154, + 1155, 1156, 1157, 1158, 1159, 1160, 1161, 1162, + 1163, 1164, 1165, 1166, 1167, 1168, 1169, 1170, + 1171, 1172, 1173, 1174, 1175, 1176, 1177, 1178, + 1179, 1180, 1181, 1182, 1183, 1184, 1185, 1186, + 1187, 1188, 1189, 1190, 1191, 1192, 1193, 1194, + 1195, 1196, 1197, 1198, 1199, 1200, 1201, 1202, + 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, + 1211, 1212, 1213, 1214, 1215, 1216, 1217, 1218, + 1219, 1220, 1221, 1222, 1223, 1224, 1225, 1226, + 1227, 1228, 1229, 1230, 1231, 1232, 1233, 1234, + 1235, 1236, 1237, 1238, 1239, 1240, 1241, 1242, + 1243, 1244, 1245, 1246, 1247, 1248, 1249, 1250, + 1251, 1252, 1253, 1254, 1255, 1256, 1257, 1258, + 1259, 1260, 1261, 1262, 1263, 1264, 1265, 1266, + 1267, 1268, 1269, 1270, 1271, 1272, 1273, 1274, + 1275, 1276, 1277, 1278, 1279, 1280, 1281, 1282, + 1283, 1284, 1285, 1286, 1287, 1288, 1289, 1290, + 1291, 1292, 1293, 1294, 1295, 1296, 1297, 1298, + 1299, 1300, 1301, 1302, 1303, 1304, 1305, 1306, + 1307, 1308, 1309, 1310, 1311, 1312, 1313, 1314, + 1315, 1316, 1317, 1318, 1319, 1320, 1321, 1322, + 1323, 1324, 1325, 1326, 1327, 1328, 1329, 1330, + 1331, 1332, 1333, 1334, 1335, 1336, 1337, 1339, + 1340, 1342, 1343, 1345, 1346, 1347, 1348, 1349, + 1350, 1351, 1352, 1353, 1354, 1355, 1356, 1357, + 1358, 1359, 1360, 1361, 1362, 1363, 1364, 1365, + 1366, 1367, 1368, 1369, 1370, 1371, 1372, 1373, + 1374, 1375, 1376, 1377, 1378, 1379, 1380, 1381, + 1382, 1383, 1384, 1385, 1386, 1387, 1388, 1389, + 1390, 1391, 1392, 1393, 1394, 1395, 1396, 1397, + 1398, 1399, 1400, 1401, 1402, 1403, 1404, 1405, + 1406, 1407, 1408, 1409, 1410, 1411, 1412, 1413, + 1414, 1415, 1416, 1417, 1418, 1419, 1420, 1421, + 1422, 1423, 1424, 1425, 1426, 1427, 1428, 1429, + 1430, 1431, 1432, 1433, 1434, 1435, 1436, 1437, + 1438, 1439, 1440, 1441, 1442, 1443, 1444, 1445, + 1446, 1447, 1448, 1449, 1450, 1451, 1452, 1453, + 1454, 1455, 1456, 1457, 1458, 1459, 1460, 1461, + 1462, 1463, 1464, 1465, 1466, 1467, 1468, 1469, + 1470, 1471, 1472, 1473, 1474, 1475, 1476, 1477, + 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1485, + 1486, 1487, 1488, 1489, 1490, 1491, 1492, 1493, + 1494, 1495, 1496, 1497, 1498, 1499, 1500, 1501, + 1502, 1503, 1504, 1505, 1506, 1507, 1508, 1509, + 1510, 1511, 1512, 1513, 1514, 1515, 1516, 1517, + 1518, 1519, 1520, 1521, 1522, 1523, 1524, 1525, + 1526, 1527, 1528, 1529, 1530, 1531, 1532, 1533, + 1534, 1535, 1536, 1537, 1538, 1539, 1540, 1541, + 1542, 1543, 1544, 1545, 1546, 1547, 1548, 1549, + 1550, 1551, 1552, 1553, 1554, 1555, 1556, 1557, + 1558, 1559, 1560, 1561, 1562, 1563, 1564, 1565, + 1566, 1567, 1568, 1569, 1570, 1571, 1572, 1573, + 1574, 1575, 1576, 1577, 1578, 1579, 1580, 1581, + 1582, 1583, 1584, 1585, 1586, 1587, 1588, 1589, + 1590, 1591, 1592, 1593, 1594, 1595, 1596, 1597, + 1598, 1599, 1600, 1601, 1602, 1603, 1604, 1605, + 1606, 1607, 1608, 1609, 1610, 1611, 1612, 1613, + 1614, 1615, 1616, 1617, 1618, 1619, 1620, 1621, + 1622, 1623, 1624, 1625, 1626, 1627, 1628, 1629, + 1630, 1631, 1632, 1633, 1634, 1635, 1636, 1637, + 1638, 1639, 1640, 1641, 1642, 1643, 1644, 1645, + 1646, 1647, 1648, 1649, 1650, 1651, 1652, 1653, + 1654, 1655, 1656, 1657, 1658, 1659, 1660, 1661, + 1662, 1663, 1664, 1665, 1666, 1667, 1668, 1669, + 1670, 1671, 1672, 1673, 1674, 1675, 1676, 1677, + 1678, 1679, 1680, 1681, 1682, 1683, 1684, 1685, + 1686, 1687, 1688, 1689, 1690, 1691, 1692, 1693, + 1694, 1695, 1696, 1697, 1698, 1699, 1700, 1701, + 1702, 1703, 1704, 1705, 1706, 1707, 1708, 1709, + 1710, 1711, 1712, 1713, 1714, 1715, 1716, 1717, + 1718, 1719, 1720, 1721, 1722, 1723, 1724, 1725, + 1726, 1727, 1728, 1729, 1730, 1731, 1732, 1733, + 1734, 1735, 1736, 1737, 1738, 1739, 1740, 1741, + 1742, 1743, 1744, 1745, 1746, 1747, 1748, 1749, + 1750, 1751, 1752, 1753, 1754, 1755, 1756, 1757, + 1758, 1759, 1760, 1761, 1762, 1763, 1764, 1765, + 1766, 1767, 1768, 1769, 1770, 1771, 1772, 1773, + 1774, 1775, 1776, 1777, 1778, 1779, 1780, 1781, + 1782, 1783, 1784, 1785, 1786, 1787, 1788, 1789, + 1790, 1791, 1792, 1793, 1794, 1795, 1796, 1797, + 1798, 1799, 1800, 1801, 1802, 1803, 1804, 1805, + 1806, 1807, 1808, 1809, 1810, 1811, 1812, 1813, + 1814, 1815, 1816, 1817, 1818, 1819, 1820, 1821, + 1822, 1823, 1824, 1825, 1826, 1827, 1828, 1829, + 1830, 1831, 1832, 1833, 1834, 1835, 1836, 1837, + 1838, 1839, 1840, 1841, 1842, 1843, 1844, 1845, + 1846, 1847, 1848, 1849, 1850, 1851, 1852, 1853, + 1854, 1855, 1856, 1857, 1858, 1859, 1860, 1861, + 1862, 1863, 1864, 1865, 1866, 1867, 1868, 1869, + 1870, 1871, 1872, 1873, 1874, 1875, 1876, 1877, + 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1885, + 1886, 1887, 1888, 1889, 1890, 1891, 1892, 1893, + 1894, 1895, 1896, 1897, 1898, 1899, 1900, 1901, + 1902, 1903, 1904, 1905, 1906, 1907, 1908, 1909, + 1910, 1911, 1912, 1913, 1914, 1915, 1916, 1917, + 1918, 1919, 1920, 1921, 1922, 1923, 1924, 1925, + 1926, 1927, 1928, 1929, 1930, 1931, 1932, 1933, + 1934, 1935, 1936, 1937, 1938, 1939, 1940, 1941, + 1942, 1943, 1944, 1945, 1946, 1947, 1948, 1949, + 1950, 1951, 1952, 1953, 1954, 1955, 1956, 1957, + 1958, 1959, 1960, 1961, 1962, 1963, 1964, 1965, + 1966, 1967, 1968, 1969, 1970, 1971, 1972, 1973, + 1974, 1975, 1976, 1977, 1978, 1979, 1980, 1981, + 1982, 1983, 1984, 1985, 1986, 1987, 1988, 1989, + 1990, 1991, 1992, 1993, 1994, 1995, 1996, 1997, + 1998, 1999, 2000, 2001, 2002, 2003, 2004, 2005, + 2006, 2007, 2008, 2009, 2010, 2011, 2012, 2013, + 2014, 2015, 2016, 2017, 2018, 2019, 2020, 2021, + 2022, 2023, 2024, 2025, 2026, 2027, 2028, 2029, + 2030, 2031, 2032, 2033, 2034, 2035, 2036, 2037, + 2038, 2039, 2040, 2041, 2042, 2043, 2044, 2045, + 2046, 2047, 2048, 2049, 2050, 2051, 2052, 2053, + 2054, 2055, 2056, 2057, 2058, 2059, 2060, 2061, + 2062, 2063, 2064, 2065, 2066, 2067, 2068, 2069, + 2070, 2071, 2072, 2073, 2074, 2075, 2076, 2077, + 2078, 2079, 2080, 2081, 2082, 2083, 2084, 2085, + 2086, 2087, 2088, 2089, 2090, 2091, 2092, 2093, + 2094, 2095, 2096, 2097, 2098, 2099, 2100, 2101, + 2102, 2103, 2104, 2105, 2106, 2107, 2108, 2109, + 2110, 2111, 2112, 2113, 2114, 2115, 2116, 2117, + 2118, 2119, 2120, 2121, 2122, 2123, 2124, 2125, + 2126, 2127, 2128, 2129, 2130, 2131, 2132, 2133, + 2134, 2135, 2136, 2137, 2138, 2139, 2140, 2141, + 2142, 2143, 2144, 2145, 2146, 2147, 2148, 2149, + 2150, 2151, 2152, 2153, 2154, 2155, 2156, 2157, + 2158, 2159, 2160, 2161, 2162, 2163, 2164, 2165, + 2166, 2167, 2168, 2169, 2170, 2172, 2173, 2175, + 2176, 2178, 2179, 2181, 2182, 2184, 2185, 2187, + 2188, 2189, 2190, 2192, 2193, 2194, 2195, 2196, + 2197, 2198, 2199, 2200, 2201, 2202, 2203, 2204, + 2206, 2208, 2210, 2212, 2214, 2215, 2216, 2218, + 2219, 2221, 2222, 2224, 2225, 2226, 2227, 2228, + 2229, 2230, 2231, 2232, 2233, 2234, 2235, 2236, + 2237, 2238, 2239, 2240, 2241, 2242, 2243, 2244, + 2245, 2246, 2247, 2248, 2249, 2250, 2251, 2252, + 2253, 2254, 2255, 2256, 2257, 2258, 2259, 2260, + 2261, 2262, 2263, 2264, 2265, 2266, 2267, 2268, + 2269, 2270, 2271, 2272, 2273, 2274, 2275, 2276, + 2277, 2278, 2279, 2280, 2281, 2282, 2283, 2284, + 2285, 2286, 2287, 2288, 2289, 2290, 2291, 2292, + 2293, 2294, 2295, 2296, 2297, 2298, 2299, 2300, + 2301, 2302, 2303, 2304, 2305, 2306, 2307, 2308, + 2309, 2310, 2311, 2312, 2313, 2314, 2315, 2316, + 2317, 2318, 2319, 2320, 2321, 2322, 2323, 2324, + 2325, 2326, 2327, 2328, 2329, 2330, 2331, 2332, + 2333, 2334, 2335, 2336, 2337, 2338, 2339, 2340, + 2341, 2342, 2343, 2344, 2345, 2346, 2347, 2348, + 2349, 2350, 2351, 2352, 2353, 2354, 2355, 2356, + 2357, 2358, 2359, 2360, 2361, 2362, 2363, 2364, + 2365, 2366, 2367, 2368, 2369, 2370, 2371, 2372, + 2373, 2374, 2375, 2376, 2377, 2378, 2379, 2380, + 2381, 2382, 2383, 2384, 2385, 2386, 2387, 2388, + 2389, 2390, 2391, 2392, 2393, 2394, 2395, 2396, + 2397, 2398, 2399, 2400, 2401, 2402, 2403, 2404, + 2405, 2406, 2407, 2408, 2409, 2410, 2411, 2412, + 2413, 2414, 2415, 2416, 2417, 2418, 2419, 2420, + 2421, 2422, 2423, 2424, 2425, 2426, 2427, 2428, + 2429, 2430, 2431, 2432, 2433, 2434, 2435, 2436, + 2437, 2438, 2439, 2440, 2441, 2442, 2443, 2444, + 2445, 2446, 2447, 2448, 2449, 2450, 2451, 2452, + 2453, 2454, 2455, 2456, 2457, 2458, 2459, 2460, + 2461, 2462, 2463, 2464, 2465, 2466, 2467, 2468, + 2469, 2470, 2471, 2472, 2473, 2474, 2475, 2476, + 2477, 2478, 2479, 2480, 2481, 2482, 2483, 2484, + 2485, 2486, 2487, 2488, 2489, 2490, 2491, 2492, + 2493, 2494, 2495, 2496, 2497, 2498, 2499, 2500, + 2501, 2502, 2503, 2504, 2505, 2506, 2507, 2508, + 2509, 2510, 2511, 2512, 2513, 2514, 2515, 2516, + 2517, 2518, 2519, 2520, 2521, 2522, 2523, 2524, + 2525, 2526, 2527, 2528, 2529, 2530, 2531, 2532, + 2533, 2534, 2535, 2536, 2537, 2538, 2539, 2540, + 2541, 2542, 2543, 2544, 2545, 2546, 2547, 2548, + 2549, 2550, 2551, 2552, 2553, 2554, 2555, 2556, + 2557, 2558, 2559, 2560, 2561, 2562, 2563, 2564, + 2566, 2568, 2570, 2572, 2574, 2575, 2577, 2578, + 2580, 2581, 2583, 2584, 2585, 2586, 2587, 2588, + 2589, 2590, 2591, 2592, 2593, 2594, 2595, 2596, + 2597, 2598, 2599, 2600, 2601, 2602, 2603, 2604, + 2605, 2606, 2607, 2608, 2609, 2610, 2611, 2612, + 2613, 2614, 2615, 2616, 2617, 2618, 2619, 2620, + 2621, 2622, 2623, 2624, 2625, 2626, 2627, 2628, + 2629, 2630, 2631, 2632, 2633, 2634, 2635, 2636, + 2637, 2638, 2639, 2640, 2641, 2642, 2643, 2644, + 2645, 2646, 2647, 2648, 2649, 2650, 2651, 2652, + 2653, 2655, 2657, 2658, 2659, 2660, 2661, 2662, + 2664, 2666, 2668, 2670, 2672, 2673, 2675, 2676, + 2678, 2679, 2681, 2682, 2683, 2684, 2685, 2687, + 2689, 2691, 2693, 2695, 2696, 2698, 2699, 2701, + 2702, 2704, 2705, 2706, 2707, 2708, 2709, 2710, + 2711, 2712, 2713, 2714, 2715, 2716, 2717, 2718, + 2719, 2720, 2721, 2722, 2723, 2724, 2725, 2726, + 2727, 2728, 2729, 2730, 2731, 2732, 2733, 2734, + 2735, 2736, 2737, 2738, 2739, 2740, 2741, 2742, + 2743, 2744, 2745, 2746, 2747, 2748, 2749, 2750, + 2751, 2752, 2753, 2754, 2755, 2756, 2757, 2758, + 2759, 2760, 2761, 2762, 2763, 2764, 2765, 2766, + 2767, 2768, 2769, 2770, 2771, 2772, 2773, 2774, + 2775, 2776, 2777, 2778, 2779, 2780, 2781, 2782, + 2783, 2784, 2785, 2786, 2787, 2788, 2789, 2790, + 2791, 2792, 2793, 2794, 2795, 2796, 2797, 2798, + 2799, 2800, 2801, 2802, 2803, 2804, 2805, 2806, + 2807, 2808, 2809, 2810, 2811, 2812, 2813, 2814, + 2815, 2816, 2817, 2818, 2819, 2820, 2821, 2822, + 2823, 2824, 2825, 2826, 2827, 2828, 2829, 2830, + 2831, 2832, 2833, 2834, 2835, 2836, 2837, 2838, + 2839, 2840, 2841, 2842, 2843, 2844, 2845, 2846, + 2847, 2848, 2849, 2850, 2851, 2852, 2853, 2854, + 2855, 2856, 2857, 2858, 2859, 2860, 2861, 2862, + 2863, 2864, 2865, 2866, 2867, 2868, 2869, 2870, + 2871, 2872, 2873, 2874, 2875, 2876, 2877, 2878, + 2879, 2880, 2881, 2882, 2883, 2884, 2885, 2886, + 2887, 2888, 2889, 2890, 2891, 2892, 2893, 2894, + 2895, 2896, 2897, 2898, 2899, 2900, 2901, 2902, + 2903, 2904, 2905, 2906, 2907, 2908, 2909, 2910, + 2911, 2912, 2913, 2914, 2915, 2916, 2917, 2918, + 2919, 2920, 2921, 2922, 2923, 2924, 2925, 2926, + 2927, 2928, 2929, 2930, 2931, 2932, 2933, 2934, + 2935, 2936, 2937, 2938, 2939, 2940, 2941, 2942, + 2943, 2944, 2945, 2946, 2947, 2948, 2949, 2950, + 2951, 2952, 2953, 2954, 2955, 2956, 2957, 2958, + 2959, 2960, 2961, 2962, 2963, 2964, 2965, 2966, + 2967, 2968, 2969, 2970, 2971, 2972, 2973, 2974, + 2975, 2976, 2977, 2978, 2979, 2980, 2981, 2982, + 2983, 2984, 2985, 2986, 2987, 2988, 2989, 2990, + 2991, 2992, 2993, 2994, 2995, 2996, 2997, 2998, + 2999, 3000, 3001, 3002, 3003, 3004, 3005, 3006, + 3007, 3008, 3009, 3010, 3011, 3012, 3013, 3014, + 3015, 3016, 3017, 3018, 3019, 3020, 3021, 3022, + 3023, 3024, 3025, 3026, 3027, 3028, 3029, 3030, + 3031, 3032, 3033, 3034, 3035, 3036, 3037, 3038, + 3039, 3040, 3041, 3042, 3043, 3044, 3045, 3046, + 3047, 3048, 3049, 3050, 3051, 3052, 3053, 3054, + 3055, 3056, 3057, 3058, 3059, 3060, 3061, 3062, + 3063, 3064, 3065, 3066, 3067, 3068, 3069, 3070, + 3071, 3072, 3073, 3074, 3075, 3076, 3077, 3078, + 3079, 3080, 3081, 3082, 3083, 3084, 3085, 3086, + 3087, 3088, 3089, 3090, 3091, 3092, 3093, 3094, + 3095, 3096, 3097, 3098, 3099, 3100, 3101, 3102, + 3103, 3104, 3105, 3106, 3107, 3108, 3109, 3110, + 3111, 3112, 3113, 3114, 3115, 3116, 3117, 3118, + 3119, 3120, 3121, 3122, 3123, 3124, 3125, 3126, + 3127, 3128, 3129, 3130, 3131, 3132, 3133, 3134, + 3135, 3136, 3137, 3138, 3139, 3140, 3141, 3142, + 3143, 3144, 3145, 3146, 3147, 3148, 3149, 3150, + 3151, 3152, 3153, 3154, 3155, 3156, 3157, 3158, + 3159, 3160, 3161, 3162, 3163, 3164, 3165, 3166, + 3167, 3168, 3169, 3170, 3171, 3172, 3173, 3174, + 3175, 3176, 3177, 3178, 3179, 3180, 3181, 3182, + 3183, 3184, 3185, 3186, 3187, 3188, 3189, 3190, + 3191, 3192, 3193, 3194, 3195, 3196, 3197, 3198, + 3199, 3200, 3201, 3202, 3203, 3204, 3205, 3206, + 3207, 3208, 3209, 3210, 3211, 3212, 3213, 3214, + 3215, 3216, 3217, 3218, 3219, 3220, 3221, 3222, + 3223, 3224, 3225, 3226, 3227, 3228, 3229, 3230, + 3231, 3232, 3233, 3234, 3235, 3236, 3237, 3238, + 3239, 3240, 3241, 3242, 3243, 3244, 3245, 3246, + 3247, 3248, 3249, 3250, 3251, 3252, 3253, 3254, + 3255, 3256, 3257, 3258, 3259, 3260, 3261, 3262, + 3263, 3264, 3265, 3266, 3267, 3268, 3269, 3270, + 3271, 3272, 3273, 3274, 3275, 3276, 3277, 3278, + 3279, 3280, 3281, 3282, 3283, 3284, 3285, 3286, + 3287, 3288, 3289, 3290, 3291, 3292, 3293, 3294, + 3295, 3296, 3297, 3298, 3299, 3300, 3301, 3302, + 3303, 3304, 3305, 3306, 3307, 3308, 3309, 3310, + 3311, 3312, 3313, 3314, 3315, 3316, 3317, 3318, + 3319, 3320, 3321, 3322, 3323, 3324, 3325, 3326, + 3327, 3328, 3329, 3330, 3331, 3332, 3333, 3334, + 3335, 3336, 3337, 3338, 3339, 3340, 3341, 3342, + 3343, 3344, 3345, 3346, 3347, 3348, 3349, 3350, + 3351, 3352, 3353, 3354, 3355, 3356, 3357, 3358, + 3359, 3360, 3361, 3362, 3363, 3364, 3365, 3366, + 3367, 3368, 3369, 3370, 3371, 3372, 3373, 3374, + 3375, 3376, 3377, 3378, 3379, 3380, 3381, 3382, + 3383, 3384, 3385, 3386, 3387, 3388, 3389, 3390, + 3391, 3392, 3393, 3394, 3395, 3396, 3397, 3398, + 3399, 3400, 3401, 3402, 3403, 3404, 3405, 3406, + 3407, 3408, 3409, 3410, 3411, 3412, 3413, 3414, + 3415, 3416, 3417, 3418, 3419, 3420, 3421, 3422, + 3423, 3424, 3425, 3426, 3427, 3428, 3429, 3430, + 3431, 3432, 3433, 3434, 3435, 3436, 3437, 3438, + 3439, 3440, 3441, 3442, 3443, 3444, 3445, 3446, + 3447, 3448, 3449, 3450, 3451, 3452, 3453, 3454, + 3455, 3456, 3457, 3458, 3459, 3460, 3461, 3462, + 3463, 3464, 3465, 3466, 3467, 3468, 3469, 3470, + 3471, 3472, 3473, 3474, 3475, 3476, 3477, 3478, + 3479, 3480, 3481, 3482, 3483, 3484, 3485, 3486, + 3487, 3488, 3489, 3490, 3491, 3492, 3493, 3494, + 3495, 3496, 3497, 3498, 3499, 3500, 3501, 3502, + 3503, 3504, 3505, 3506, 3507, 3508, 3509, 3510, + 3511, 3512, 3513, 3514, 3515, 3516, 3517, 3518, + 3519, 3520, 3521, 3522, 3523, 3524, 3525, 3526, + 3527, 3528, 3529, 3530, 3531, 3532, 3533, 3534, + 3535, 3536, 3537, 3538, 3539, 3540, 3541, 3542, + 3543, 3544, 3545, 3546, 3547, 3548, 3549, 3550, + 3551, 3552, 3553, 3554, 3555, 3556, 3557, 3558, + 3559, 3560, 0 + }; + + static const signed char _regex_trans_lengths[] = { + 2, 1, 2, 1, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, + 2, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 2, 1, 2, 1, + 2, 1, 2, 1, 2, 1, 2, 1, + 1, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 2, + 2, 2, 2, 2, 1, 1, 2, 1, + 2, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 2, + 2, 2, 2, 2, 1, 2, 1, 2, + 1, 2, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 2, 2, 1, 1, 1, 1, 1, 2, + 2, 2, 2, 2, 1, 2, 1, 2, + 1, 2, 1, 1, 1, 1, 2, 2, + 2, 2, 2, 1, 2, 1, 2, 1, + 2, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 0 + }; + + static const signed char _regex_cond_keys[] = { + 0, 1, 0, 0, 1, 0, 0, 1, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1, 0, 0, 1, 0, 0, + 1, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 0, 0, 1, 0, + 0, 1, 0, 0, 1, 0, 0, 1, + 0, 0, 1, 0, 0, 0, 0, 1, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1, 0, 1, + 0, 1, 0, 1, 0, 1, 0, 0, + 0, 1, 0, 0, 1, 0, 0, 1, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1, 0, 1, + 0, 1, 0, 1, 0, 1, 0, 0, + 1, 0, 0, 1, 0, 0, 1, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1, 0, + 1, 0, 0, 0, 0, 0, 0, 1, + 0, 1, 0, 1, 0, 1, 0, 1, + 0, 0, 1, 0, 0, 1, 0, 0, + 1, 0, 0, 0, 0, 0, 1, 0, + 1, 0, 1, 0, 1, 0, 1, 0, + 0, 1, 0, 0, 1, 0, 0, 1, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0 + }; + + static const short _regex_cond_targs[] = { + 746, 746, 746, 746, 3, 746, 746, 746, + 746, 746, 746, 746, 746, 751, 752, 753, + 746, 754, 755, 746, 757, 746, 746, 758, + 759, 760, 761, 761, 761, 761, 746, 756, + 746, 746, 6, 6, 6, 6, 746, 746, + 7, 7, 7, 7, 746, 9, 9, 9, + 9, 746, 10, 9, 9, 9, 9, 746, + 746, 746, 746, 11, 11, 11, 11, 746, + 746, 11, 12, 11, 11, 746, 746, 11, + 12, 11, 11, 746, 15, 15, 15, 15, + 746, 10, 15, 15, 15, 15, 746, 746, + 17, 746, 746, 746, 746, 746, 19, 746, + 11, 21, 11, 11, 746, 20, 20, 20, + 20, 746, 746, 20, 20, 20, 20, 746, + 746, 11, 21, 11, 11, 746, 746, 22, + 746, 746, 746, 23, 23, 23, 23, 746, + 746, 24, 24, 24, 24, 746, 746, 25, + 746, 24, 24, 24, 24, 746, 28, 28, + 28, 28, 746, 746, 28, 28, 28, 28, + 746, 6, 6, 6, 6, 746, 23, 23, + 23, 23, 746, 746, 30, 746, 31, 31, + 31, 31, 746, 33, 34, 746, 32, 33, + 34, 746, 32, 33, 34, 32, 36, 37, + 746, 35, 36, 37, 746, 35, 36, 37, + 35, 39, 40, 746, 38, 39, 40, 746, + 38, 39, 40, 38, 746, 41, 775, 746, + 746, 43, 45, 52, 49, 52, 52, 746, + 46, 746, 746, 47, 746, 746, 48, 746, + 746, 746, 52, 746, 50, 52, 52, 746, + 52, 746, 51, 52, 52, 746, 52, 746, + 52, 52, 52, 746, 52, 746, 52, 52, + 52, 746, 54, 54, 54, 54, 746, 746, + 54, 54, 54, 54, 746, 56, 56, 56, + 56, 746, 746, 56, 56, 56, 56, 746, + 58, 58, 58, 58, 746, 58, 746, 58, + 58, 58, 746, 60, 746, 746, 60, 746, + 746, 61, 61, 61, 746, 63, 786, 62, + 746, 786, 64, 746, 786, 64, 746, 787, + 65, 787, 67, 65, 787, 65, 787, 69, + 65, 787, 70, 65, 787, 65, 790, 790, + 73, 76, 790, 74, 790, 75, 790, 790, + 790, 77, 790, 78, 790, 79, 790, 80, + 790, 790, 790, 82, 790, 83, 790, 84, + 790, 85, 790, 790, 790, 87, 92, 94, + 790, 88, 790, 89, 790, 90, 790, 91, + 790, 790, 790, 93, 790, 790, 790, 95, + 790, 790, 790, 97, 790, 98, 790, 99, + 790, 100, 790, 790, 790, 102, 790, 103, + 790, 104, 790, 105, 790, 106, 790, 790, + 790, 108, 790, 109, 111, 790, 110, 790, + 790, 790, 112, 790, 113, 790, 790, 790, + 115, 120, 790, 116, 790, 117, 790, 118, + 790, 119, 790, 790, 790, 121, 790, 790, + 790, 123, 139, 790, 124, 790, 125, 790, + 126, 790, 127, 790, 128, 790, 129, 790, + 130, 790, 131, 790, 132, 790, 133, 790, + 134, 790, 135, 790, 136, 790, 137, 790, + 138, 790, 790, 790, 140, 790, 141, 790, + 790, 790, 143, 144, 790, 790, 790, 145, + 790, 146, 790, 147, 790, 148, 790, 790, + 790, 150, 790, 151, 790, 790, 790, 153, + 790, 154, 790, 790, 790, 156, 790, 157, + 790, 158, 790, 159, 790, 160, 790, 161, + 790, 790, 790, 163, 167, 790, 164, 790, + 165, 790, 166, 790, 790, 790, 168, 790, + 169, 790, 170, 790, 171, 790, 790, 790, + 173, 177, 790, 174, 790, 175, 790, 176, + 790, 790, 790, 178, 790, 179, 790, 180, + 790, 181, 790, 182, 790, 183, 790, 790, + 790, 185, 790, 186, 790, 187, 790, 188, + 790, 189, 790, 190, 790, 191, 790, 192, + 790, 193, 790, 194, 790, 195, 790, 196, + 790, 197, 790, 198, 790, 199, 790, 200, + 790, 201, 790, 790, 790, 203, 790, 204, + 790, 205, 790, 206, 790, 207, 790, 790, + 790, 209, 790, 210, 790, 211, 790, 212, + 790, 213, 790, 790, 790, 215, 790, 216, + 790, 217, 790, 218, 790, 219, 790, 220, + 790, 221, 790, 790, 790, 223, 790, 224, + 790, 225, 790, 790, 790, 227, 790, 228, + 790, 790, 790, 230, 235, 790, 231, 790, + 232, 790, 233, 790, 234, 790, 790, 790, + 236, 790, 237, 790, 238, 790, 239, 790, + 790, 790, 799, 790, 242, 790, 790, 790, + 244, 790, 245, 790, 790, 790, 247, 790, + 248, 790, 249, 790, 790, 790, 251, 790, + 252, 790, 253, 790, 254, 790, 255, 790, + 790, 790, 257, 790, 258, 790, 259, 790, + 260, 790, 261, 790, 262, 790, 263, 790, + 264, 790, 265, 790, 266, 790, 267, 790, + 268, 790, 269, 790, 790, 790, 271, 277, + 790, 272, 790, 273, 790, 274, 790, 275, + 790, 276, 790, 790, 790, 278, 790, 279, + 790, 280, 790, 281, 790, 282, 790, 283, + 790, 284, 790, 285, 790, 286, 790, 287, + 790, 288, 790, 289, 790, 290, 790, 291, + 295, 790, 292, 790, 293, 790, 294, 790, + 790, 790, 296, 790, 297, 790, 298, 790, + 299, 790, 790, 790, 301, 790, 302, 790, + 303, 790, 304, 790, 305, 790, 790, 790, + 307, 310, 314, 319, 790, 308, 790, 309, + 790, 790, 790, 311, 790, 312, 790, 313, + 790, 790, 790, 315, 790, 316, 790, 317, + 790, 318, 790, 790, 790, 320, 790, 321, + 790, 322, 790, 323, 790, 790, 790, 325, + 332, 790, 326, 790, 327, 790, 328, 790, + 329, 790, 330, 790, 331, 790, 790, 790, + 333, 790, 790, 790, 790, 335, 790, 336, + 790, 790, 790, 338, 790, 339, 790, 340, + 790, 790, 790, 342, 344, 349, 790, 343, + 790, 790, 790, 345, 790, 346, 790, 347, + 790, 348, 790, 790, 790, 790, 790, 351, + 354, 790, 352, 790, 353, 790, 790, 790, + 355, 790, 356, 790, 790, 790, 358, 364, + 790, 359, 790, 360, 790, 361, 790, 362, + 790, 363, 790, 790, 790, 365, 790, 366, + 790, 367, 790, 790, 790, 369, 790, 370, + 790, 371, 790, 372, 790, 373, 790, 374, + 790, 375, 790, 376, 790, 790, 790, 378, + 790, 379, 790, 380, 790, 381, 790, 382, + 790, 383, 790, 790, 790, 385, 790, 386, + 790, 387, 790, 388, 790, 790, 790, 390, + 790, 391, 790, 392, 790, 393, 790, 394, + 790, 395, 790, 396, 790, 397, 790, 790, + 790, 790, 790, 400, 790, 401, 790, 790, + 790, 403, 408, 790, 404, 790, 405, 790, + 406, 790, 407, 790, 790, 790, 409, 790, + 410, 415, 421, 433, 790, 411, 790, 412, + 790, 413, 790, 414, 790, 790, 790, 416, + 790, 417, 790, 418, 790, 419, 790, 420, + 790, 790, 790, 422, 790, 423, 790, 424, + 790, 425, 790, 426, 790, 427, 790, 428, + 790, 429, 790, 430, 790, 431, 790, 432, + 790, 790, 790, 434, 790, 435, 790, 436, + 790, 437, 790, 790, 790, 439, 790, 440, + 790, 790, 790, 442, 790, 443, 790, 444, + 790, 445, 790, 790, 790, 447, 452, 790, + 448, 790, 449, 790, 450, 790, 451, 790, + 790, 790, 453, 790, 454, 790, 455, 790, + 456, 790, 457, 790, 458, 790, 790, 790, + 460, 790, 461, 790, 462, 790, 790, 790, + 464, 790, 465, 790, 790, 790, 467, 473, + 790, 468, 790, 469, 790, 470, 790, 471, + 790, 472, 790, 790, 790, 474, 790, 475, + 790, 476, 790, 477, 790, 478, 790, 479, + 790, 790, 790, 481, 790, 482, 790, 483, + 790, 484, 790, 790, 790, 486, 790, 487, + 790, 488, 790, 489, 790, 790, 790, 491, + 790, 492, 790, 493, 790, 494, 790, 495, + 790, 496, 790, 790, 790, 498, 507, 790, + 499, 790, 500, 790, 501, 790, 502, 790, + 503, 790, 504, 790, 505, 790, 506, 790, + 790, 790, 508, 790, 509, 790, 790, 790, + 511, 519, 528, 790, 512, 515, 790, 513, + 790, 514, 790, 790, 790, 516, 790, 517, + 790, 518, 790, 790, 790, 520, 790, 521, + 522, 525, 790, 790, 790, 523, 790, 524, + 790, 790, 790, 526, 790, 527, 790, 790, + 790, 529, 790, 790, 790, 531, 790, 532, + 790, 533, 790, 790, 790, 535, 790, 536, + 790, 790, 537, 790, 790, 790, 539, 543, + 790, 540, 790, 541, 790, 542, 790, 790, + 790, 544, 790, 545, 790, 546, 790, 547, + 790, 790, 790, 549, 790, 550, 790, 551, + 790, 552, 790, 553, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 560, 0, 562, 561, 817, 0, 0, + 561, 819, 819, 819, 819, 565, 819, 819, + 819, 819, 567, 568, 819, 566, 567, 568, + 819, 566, 567, 568, 566, 571, 572, 819, + 573, 654, 669, 675, 681, 687, 693, 699, + 710, 716, 722, 727, 570, 571, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 570, + 571, 572, 819, 574, 589, 595, 601, 607, + 613, 619, 630, 636, 642, 647, 570, 571, + 572, 819, 575, 584, 570, 571, 572, 819, + 576, 580, 570, 571, 572, 819, 577, 570, + 571, 572, 819, 578, 570, 579, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 581, 570, 571, 572, 819, 582, 570, 583, + 572, 819, 570, 571, 572, 819, 570, 571, + 572, 819, 585, 570, 571, 572, 819, 586, + 570, 571, 572, 819, 587, 570, 588, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 590, 570, 571, 572, 819, 591, 570, + 571, 572, 819, 592, 570, 571, 572, 819, + 593, 570, 594, 572, 819, 570, 571, 572, + 819, 570, 571, 572, 819, 596, 570, 571, + 572, 819, 597, 570, 571, 572, 819, 598, + 570, 571, 572, 819, 599, 570, 600, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 602, 570, 571, 572, 819, 603, 570, + 571, 572, 819, 604, 570, 571, 572, 819, + 605, 570, 606, 572, 819, 570, 571, 572, + 819, 570, 571, 572, 819, 608, 570, 571, + 572, 819, 609, 570, 571, 572, 819, 610, + 570, 571, 572, 819, 611, 570, 612, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 614, 570, 571, 572, 819, 615, 570, + 571, 572, 819, 616, 570, 571, 572, 819, + 617, 570, 618, 572, 819, 570, 571, 572, + 819, 570, 571, 572, 819, 620, 625, 570, + 571, 572, 819, 621, 570, 571, 572, 819, + 622, 570, 571, 572, 819, 623, 570, 624, + 572, 819, 570, 571, 572, 819, 570, 571, + 572, 819, 626, 570, 571, 572, 819, 627, + 570, 571, 572, 819, 628, 570, 629, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 631, 570, 571, 572, 819, 632, 570, + 571, 572, 819, 633, 570, 571, 572, 819, + 634, 570, 635, 572, 819, 570, 571, 572, + 819, 570, 571, 572, 819, 637, 570, 571, + 572, 819, 638, 570, 571, 572, 819, 639, + 570, 571, 572, 819, 640, 570, 641, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 643, 570, 571, 572, 819, 644, 570, + 571, 572, 819, 645, 570, 646, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 648, 570, 571, 572, 819, 649, 570, 571, + 572, 819, 650, 570, 571, 572, 819, 651, + 570, 571, 572, 819, 652, 570, 653, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 655, 664, 570, 571, 572, 819, 656, + 660, 570, 571, 572, 819, 657, 570, 571, + 572, 819, 658, 570, 659, 572, 819, 570, + 571, 572, 819, 570, 571, 572, 819, 661, + 570, 571, 572, 819, 662, 570, 663, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 665, 570, 571, 572, 819, 666, 570, + 571, 572, 819, 667, 570, 668, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 670, 570, 571, 572, 819, 671, 570, 571, + 572, 819, 672, 570, 571, 572, 819, 673, + 570, 674, 572, 819, 570, 571, 572, 819, + 570, 571, 572, 819, 676, 570, 571, 572, + 819, 677, 570, 571, 572, 819, 678, 570, + 571, 572, 819, 679, 570, 680, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 682, 570, 571, 572, 819, 683, 570, 571, + 572, 819, 684, 570, 571, 572, 819, 685, + 570, 686, 572, 819, 570, 571, 572, 819, + 570, 571, 572, 819, 688, 570, 571, 572, + 819, 689, 570, 571, 572, 819, 690, 570, + 571, 572, 819, 691, 570, 692, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 694, 570, 571, 572, 819, 695, 570, 571, + 572, 819, 696, 570, 571, 572, 819, 697, + 570, 698, 572, 819, 570, 571, 572, 819, + 570, 571, 572, 819, 700, 705, 570, 571, + 572, 819, 701, 570, 571, 572, 819, 702, + 570, 571, 572, 819, 703, 570, 704, 572, + 819, 570, 571, 572, 819, 570, 571, 572, + 819, 706, 570, 571, 572, 819, 707, 570, + 571, 572, 819, 708, 570, 709, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 711, 570, 571, 572, 819, 712, 570, 571, + 572, 819, 713, 570, 571, 572, 819, 714, + 570, 715, 572, 819, 570, 571, 572, 819, + 570, 571, 572, 819, 717, 570, 571, 572, + 819, 718, 570, 571, 572, 819, 719, 570, + 571, 572, 819, 720, 570, 721, 572, 819, + 570, 571, 572, 819, 570, 571, 572, 819, + 723, 570, 571, 572, 819, 724, 570, 571, + 572, 819, 725, 570, 726, 572, 819, 570, + 571, 572, 819, 570, 571, 572, 819, 728, + 570, 571, 572, 819, 729, 570, 571, 572, + 819, 730, 570, 571, 572, 819, 731, 570, + 571, 572, 819, 732, 570, 733, 572, 819, + 570, 571, 572, 819, 570, 735, 736, 819, + 734, 735, 736, 819, 734, 735, 736, 734, + 738, 819, 819, 738, 819, 819, 739, 739, + 739, 819, 838, 838, 838, 838, 742, 838, + 838, 838, 838, 843, 843, 843, 843, 745, + 843, 843, 843, 843, 746, 746, 746, 746, + 746, 750, 746, 762, 763, 746, 764, 765, + 766, 746, 785, 746, 746, 746, 746, 747, + 746, 748, 746, 749, 746, 746, 746, 746, + 746, 746, 746, 746, 1, 746, 746, 2, + 746, 4, 5, 746, 6, 6, 6, 6, + 746, 7, 7, 7, 7, 746, 8, 11, + 14, 16, 18, 11, 12, 13, 11, 11, + 746, 22, 746, 23, 23, 23, 23, 22, + 746, 746, 22, 746, 746, 746, 24, 24, + 24, 24, 746, 746, 746, 746, 25, 746, + 26, 27, 29, 746, 746, 30, 746, 31, + 31, 31, 31, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 32, 35, 38, + 746, 767, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 772, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 773, + 746, 746, 746, 774, 746, 779, 746, 746, + 780, 781, 746, 746, 746, 746, 746, 746, + 782, 746, 769, 771, 746, 768, 746, 746, + 746, 770, 771, 746, 771, 771, 746, 771, + 746, 746, 746, 746, 41, 42, 43, 44, + 777, 746, 776, 746, 746, 746, 778, 746, + 746, 746, 53, 55, 57, 746, 59, 746, + 746, 746, 784, 783, 783, 783, 746, 746, + 746, 746, 746, 61, 61, 61, 746, 62, + 746, 746, 746, 746, 787, 789, 788, 787, + 65, 787, 66, 68, 65, 791, 792, 793, + 795, 796, 797, 798, 800, 801, 802, 803, + 804, 806, 807, 808, 809, 810, 811, 812, + 813, 814, 815, 816, 790, 71, 72, 81, + 790, 86, 96, 101, 107, 114, 790, 122, + 790, 790, 142, 790, 794, 790, 155, 162, + 790, 149, 152, 790, 172, 790, 184, 202, + 790, 208, 214, 222, 226, 229, 790, 240, + 246, 250, 790, 241, 243, 790, 256, 270, + 790, 300, 790, 306, 324, 790, 790, 334, + 337, 341, 790, 790, 790, 790, 790, 350, + 790, 357, 790, 805, 790, 377, 384, 790, + 368, 790, 790, 389, 398, 790, 790, 790, + 399, 402, 438, 441, 790, 790, 790, 790, + 790, 446, 790, 790, 790, 790, 459, 463, + 790, 466, 790, 480, 485, 790, 790, 790, + 490, 497, 790, 510, 530, 534, 538, 790, + 548, 790, 554, 790, 555, 556, 557, 558, + 790, 790, 790, 790, 790, 790, 790, 0, + 818, 818, 818, 818, 818, 818, 818, 818, + 819, 823, 824, 819, 819, 819, 819, 820, + 819, 821, 819, 822, 819, 819, 819, 819, + 819, 819, 819, 563, 819, 819, 564, 819, + 566, 569, 734, 819, 825, 819, 819, 819, + 819, 819, 829, 819, 819, 819, 819, 819, + 819, 819, 830, 819, 819, 819, 819, 819, + 819, 819, 831, 832, 819, 819, 819, 819, + 819, 819, 833, 827, 819, 819, 819, 819, + 826, 819, 819, 819, 828, 819, 819, 819, + 819, 819, 819, 737, 819, 819, 819, 835, + 834, 834, 834, 819, 819, 819, 819, 819, + 739, 739, 739, 819, 837, 836, 836, 836, + 836, 836, 836, 836, 836, 842, 838, 838, + 838, 839, 838, 840, 838, 841, 838, 838, + 838, 838, 838, 838, 838, 740, 838, 838, + 741, 838, 838, 838, 847, 843, 843, 843, + 844, 843, 845, 843, 846, 843, 843, 843, + 843, 843, 843, 843, 743, 843, 843, 744, + 843, 843, 843, 848, 848, 849, 849, 0, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 787, 787, 787, 787, 787, 787, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 559, 560, + 561, 562, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 838, 838, 838, 843, 843, + 843, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 746, 746, 746, 746, 746, 746, + 746, 746, 787, 787, 787, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 790, 790, 790, + 817, 818, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 819, 819, 819, 819, 819, + 819, 819, 819, 836, 836, 838, 838, 838, + 838, 838, 843, 843, 843, 843, 843, 848, + 849, 0 + }; + + static const short _regex_cond_actions[] = { + 843, 769, 843, 843, 0, 843, 843, 771, + 843, 827, 631, 731, 723, 45, 903, 903, + 897, 45, 912, 900, 903, 729, 741, 0, + 45, 45, 923, 923, 923, 923, 737, 45, + 765, 747, 0, 0, 0, 0, 841, 743, + 3, 3, 3, 3, 841, 3, 3, 3, + 3, 839, 0, 3, 3, 3, 3, 839, + 761, 839, 761, 3, 3, 3, 3, 839, + 759, 3, 870, 3, 3, 839, 761, 3, + 870, 3, 3, 839, 3, 3, 3, 3, + 839, 0, 3, 3, 3, 3, 839, 751, + 0, 749, 839, 755, 753, 839, 0, 757, + 3, 3, 3, 3, 839, 0, 0, 0, + 0, 839, 757, 0, 0, 0, 0, 839, + 757, 3, 3, 3, 3, 839, 745, 0, + 841, 725, 727, 27, 27, 27, 27, 841, + 743, 3, 3, 3, 3, 841, 763, 0, + 841, 3, 3, 3, 3, 841, 3, 3, + 3, 3, 841, 673, 3, 3, 3, 3, + 841, 0, 0, 0, 0, 841, 27, 27, + 27, 27, 841, 725, 0, 727, 25, 25, + 25, 25, 841, 0, 0, 829, 0, 0, + 0, 603, 0, 0, 0, 0, 0, 0, + 829, 0, 0, 0, 601, 0, 0, 0, + 0, 0, 0, 829, 0, 0, 0, 603, + 0, 0, 0, 0, 675, 0, 17, 831, + 675, 0, 7, 3, 870, 3, 3, 831, + 17, 831, 663, 17, 831, 663, 17, 831, + 663, 831, 3, 661, 870, 3, 3, 831, + 3, 661, 870, 3, 3, 831, 3, 661, + 3, 3, 3, 831, 3, 665, 3, 3, + 3, 831, 3, 3, 3, 3, 837, 671, + 3, 3, 3, 3, 837, 3, 3, 3, + 3, 837, 669, 3, 3, 3, 3, 837, + 3, 3, 3, 3, 837, 3, 667, 3, + 3, 3, 837, 0, 833, 677, 0, 833, + 679, 0, 0, 0, 835, 0, 29, 11, + 845, 31, 13, 845, 0, 13, 845, 53, + 0, 53, 0, 0, 51, 0, 53, 0, + 0, 49, 0, 0, 47, 0, 315, 359, + 0, 0, 359, 0, 359, 0, 359, 127, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 129, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 131, 359, 0, 0, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 133, 359, 0, 359, 135, 359, 0, + 359, 137, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 139, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 141, + 359, 0, 359, 0, 0, 359, 0, 359, + 143, 359, 0, 359, 0, 359, 145, 359, + 0, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 147, 359, 0, 359, 149, + 359, 0, 0, 341, 0, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 0, 341, + 0, 341, 151, 341, 0, 341, 0, 341, + 153, 341, 0, 0, 341, 155, 341, 0, + 341, 0, 341, 0, 341, 0, 341, 157, + 341, 0, 343, 0, 343, 159, 343, 0, + 343, 0, 343, 161, 343, 0, 341, 0, + 341, 0, 341, 0, 341, 0, 341, 0, + 341, 163, 341, 0, 0, 341, 0, 341, + 0, 341, 0, 341, 165, 341, 0, 341, + 0, 341, 0, 341, 0, 341, 167, 341, + 0, 0, 359, 0, 359, 0, 359, 0, + 359, 169, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 171, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 173, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 175, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 177, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 179, 359, 0, 359, 0, + 359, 0, 359, 181, 359, 0, 359, 0, + 359, 183, 359, 0, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 185, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 187, 359, 45, 359, 0, 357, 189, 357, + 0, 357, 0, 357, 191, 357, 0, 359, + 0, 359, 0, 359, 193, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 195, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 197, 359, 0, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 199, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 0, 359, 0, 359, 0, 359, 0, 359, + 201, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 203, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 205, 359, + 0, 0, 0, 0, 359, 0, 359, 0, + 359, 207, 359, 0, 359, 0, 359, 0, + 359, 209, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 211, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 213, 359, 0, + 0, 359, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 0, 359, 215, 359, + 0, 359, 217, 359, 219, 0, 345, 0, + 345, 221, 345, 0, 345, 0, 345, 0, + 345, 223, 345, 0, 0, 0, 345, 0, + 345, 225, 345, 0, 345, 0, 345, 0, + 345, 0, 345, 227, 345, 229, 345, 0, + 0, 345, 0, 345, 0, 345, 231, 345, + 0, 345, 0, 345, 233, 345, 0, 0, + 347, 0, 347, 0, 347, 0, 347, 0, + 347, 0, 347, 235, 347, 0, 347, 0, + 347, 0, 347, 237, 347, 0, 349, 0, + 349, 0, 349, 0, 349, 0, 349, 0, + 349, 0, 349, 0, 349, 239, 349, 0, + 347, 0, 347, 0, 347, 0, 347, 0, + 347, 0, 347, 241, 347, 0, 347, 0, + 347, 0, 347, 0, 347, 243, 347, 0, + 351, 0, 351, 0, 351, 0, 351, 0, + 351, 0, 351, 0, 351, 0, 351, 245, + 351, 247, 351, 0, 359, 0, 359, 249, + 359, 0, 0, 359, 0, 359, 0, 359, + 0, 359, 0, 359, 251, 359, 0, 359, + 0, 0, 0, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 253, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 255, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 257, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 259, 359, 0, 359, 0, + 359, 261, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 263, 359, 0, 0, 353, + 0, 353, 0, 353, 0, 353, 0, 353, + 265, 353, 0, 353, 0, 353, 0, 353, + 0, 353, 0, 353, 0, 353, 267, 353, + 0, 359, 0, 359, 0, 359, 269, 359, + 0, 359, 0, 359, 271, 359, 0, 0, + 355, 0, 355, 0, 355, 0, 355, 0, + 355, 0, 355, 273, 355, 0, 355, 0, + 355, 0, 355, 0, 355, 0, 355, 0, + 355, 275, 355, 0, 355, 0, 355, 0, + 355, 0, 355, 277, 355, 0, 355, 0, + 355, 0, 355, 0, 355, 279, 355, 0, + 355, 0, 355, 0, 355, 0, 355, 0, + 355, 0, 355, 281, 355, 0, 0, 355, + 0, 355, 0, 355, 0, 355, 0, 355, + 0, 355, 0, 355, 0, 355, 0, 355, + 283, 355, 0, 355, 0, 355, 285, 355, + 0, 0, 0, 359, 0, 0, 359, 0, + 359, 0, 359, 287, 359, 0, 359, 0, + 359, 0, 359, 289, 359, 0, 359, 0, + 0, 0, 359, 291, 359, 0, 359, 0, + 359, 293, 359, 0, 359, 0, 359, 295, + 359, 0, 359, 297, 359, 0, 359, 0, + 359, 0, 359, 299, 359, 0, 359, 0, + 303, 359, 0, 359, 301, 359, 0, 0, + 359, 0, 359, 0, 359, 0, 359, 305, + 359, 0, 359, 0, 359, 0, 359, 0, + 359, 307, 359, 0, 359, 0, 359, 0, + 359, 0, 359, 0, 359, 309, 359, 311, + 359, 119, 359, 121, 359, 123, 359, 125, + 359, 0, 39, 33, 35, 37, 39, 39, + 35, 537, 503, 537, 537, 0, 537, 537, + 505, 537, 0, 0, 539, 0, 0, 0, + 377, 0, 0, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 539, + 0, 0, 0, 435, 0, 0, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 381, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 385, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 389, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 393, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 397, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 401, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 405, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 409, 0, 0, 0, 539, 0, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 413, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 417, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 421, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 425, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 429, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 433, 0, 0, 0, + 539, 0, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 379, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 383, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 387, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 391, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 395, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 399, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 403, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 407, + 0, 0, 0, 539, 0, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 411, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 415, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 419, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 423, 0, 0, 0, 539, + 0, 0, 0, 0, 539, 0, 0, 0, + 0, 539, 0, 0, 0, 0, 539, 0, + 0, 0, 427, 0, 0, 0, 539, 0, + 0, 0, 0, 539, 0, 0, 0, 0, + 539, 0, 0, 0, 0, 539, 0, 0, + 0, 0, 539, 0, 0, 0, 0, 539, + 0, 0, 0, 431, 0, 0, 0, 539, + 0, 0, 0, 377, 0, 0, 0, 0, + 0, 533, 471, 0, 533, 475, 0, 0, + 0, 535, 569, 557, 569, 569, 0, 569, + 569, 559, 569, 587, 575, 587, 587, 0, + 587, 587, 577, 587, 633, 775, 777, 629, + 637, 45, 597, 0, 0, 609, 0, 45, + 0, 635, 909, 599, 777, 773, 777, 0, + 777, 45, 777, 45, 777, 773, 775, 777, + 823, 767, 823, 823, 0, 823, 823, 0, + 823, 0, 0, 779, 0, 0, 0, 0, + 821, 3, 3, 3, 3, 821, 1, 3, + 1, 0, 3, 3, 855, 855, 3, 3, + 819, 0, 821, 27, 27, 27, 27, 0, + 821, 745, 0, 821, 735, 733, 3, 3, + 3, 3, 821, 739, 821, 763, 0, 821, + 1, 1, 0, 821, 725, 0, 727, 25, + 25, 25, 25, 821, 615, 613, 783, 619, + 617, 785, 623, 621, 787, 0, 0, 0, + 781, 5, 639, 647, 611, 695, 607, 717, + 699, 715, 683, 683, 0, 605, 713, 691, + 683, 703, 687, 719, 641, 657, 645, 0, + 693, 659, 655, 906, 697, 45, 683, 651, + 45, 0, 653, 689, 649, 683, 701, 685, + 7, 643, 852, 915, 721, 15, 791, 867, + 791, 858, 919, 795, 927, 919, 793, 919, + 847, 709, 711, 681, 0, 7, 0, 849, + 17, 801, 17, 799, 876, 799, 17, 797, + 873, 797, 1, 1, 1, 815, 0, 803, + 705, 707, 45, 19, 23, 21, 805, 882, + 894, 888, 805, 0, 0, 0, 807, 11, + 825, 627, 625, 789, 55, 45, 45, 53, + 0, 53, 0, 0, 0, 45, 45, 45, + 45, 45, 45, 45, 45, 45, 45, 45, + 45, 45, 45, 45, 45, 45, 45, 45, + 45, 45, 0, 0, 317, 0, 0, 0, + 339, 0, 0, 0, 0, 0, 339, 0, + 61, 63, 0, 65, 45, 67, 0, 0, + 319, 0, 0, 321, 0, 339, 0, 0, + 339, 0, 0, 0, 0, 0, 339, 0, + 0, 0, 339, 0, 0, 337, 0, 0, + 339, 0, 339, 0, 0, 339, 79, 0, + 0, 0, 69, 71, 73, 75, 77, 0, + 323, 0, 81, 45, 83, 0, 0, 325, + 0, 327, 85, 0, 0, 87, 89, 329, + 0, 0, 0, 0, 339, 91, 93, 95, + 97, 0, 99, 101, 103, 331, 0, 0, + 339, 0, 105, 0, 0, 107, 109, 111, + 0, 0, 333, 0, 0, 0, 0, 339, + 0, 339, 0, 339, 0, 0, 0, 0, + 339, 313, 339, 113, 115, 117, 335, 39, + 361, 363, 365, 367, 369, 371, 373, 375, + 491, 45, 0, 511, 509, 507, 509, 0, + 509, 45, 509, 45, 509, 507, 509, 529, + 501, 529, 529, 0, 529, 529, 0, 529, + 0, 0, 0, 531, 5, 489, 439, 457, + 493, 493, 0, 437, 485, 493, 461, 481, + 451, 441, 0, 487, 453, 449, 495, 455, + 493, 445, 45, 0, 447, 483, 443, 493, + 459, 479, 7, 9, 473, 497, 497, 499, + 15, 517, 861, 517, 15, 519, 864, 519, + 467, 469, 477, 0, 521, 463, 465, 45, + 19, 23, 21, 523, 879, 891, 885, 523, + 0, 0, 0, 525, 0, 549, 543, 549, + 541, 549, 547, 545, 551, 0, 563, 561, + 563, 0, 563, 45, 563, 45, 563, 561, + 563, 565, 555, 565, 565, 0, 565, 565, + 0, 565, 553, 567, 0, 581, 579, 581, + 0, 581, 45, 581, 45, 581, 579, 581, + 583, 573, 583, 583, 0, 583, 583, 0, + 583, 571, 585, 589, 591, 593, 595, 0, + 843, 843, 843, 827, 827, 841, 841, 839, + 839, 839, 839, 839, 839, 839, 839, 839, + 839, 839, 839, 839, 839, 841, 841, 841, + 841, 841, 841, 841, 841, 841, 841, 829, + 829, 829, 829, 829, 829, 829, 829, 829, + 831, 831, 831, 831, 831, 831, 831, 831, + 831, 831, 831, 831, 837, 837, 837, 837, + 837, 837, 833, 833, 835, 845, 845, 845, + 59, 59, 59, 59, 59, 59, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 343, 343, 343, 343, + 343, 343, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 341, 341, 341, 341, 341, + 341, 341, 341, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 357, 357, 357, 357, 357, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 345, 345, 345, + 345, 345, 345, 345, 345, 345, 345, 345, + 345, 345, 345, 345, 345, 345, 345, 345, + 345, 345, 345, 345, 347, 347, 347, 347, + 347, 347, 347, 347, 347, 347, 347, 349, + 349, 349, 349, 349, 349, 349, 349, 349, + 347, 347, 347, 347, 347, 347, 347, 347, + 347, 347, 347, 347, 351, 351, 351, 351, + 351, 351, 351, 351, 351, 351, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 353, 353, 353, + 353, 353, 353, 353, 353, 353, 353, 353, + 353, 353, 359, 359, 359, 359, 359, 359, + 359, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 355, 355, 355, + 355, 355, 355, 355, 355, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 359, 359, + 359, 359, 359, 359, 359, 359, 39, 39, + 39, 39, 537, 537, 537, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 539, 539, 539, 539, 539, 539, 539, 539, + 533, 533, 535, 569, 569, 569, 587, 587, + 587, 0, 823, 823, 823, 779, 821, 821, + 819, 821, 821, 821, 821, 821, 821, 821, + 821, 783, 785, 787, 781, 817, 791, 791, + 795, 793, 847, 811, 809, 801, 799, 799, + 797, 797, 815, 803, 813, 805, 805, 807, + 825, 789, 0, 57, 57, 0, 339, 339, + 319, 321, 339, 339, 339, 339, 337, 339, + 339, 339, 323, 325, 327, 329, 339, 331, + 339, 333, 339, 339, 339, 339, 339, 335, + 0, 0, 0, 529, 529, 529, 531, 531, + 517, 517, 519, 519, 513, 527, 521, 515, + 523, 523, 525, 0, 551, 0, 565, 565, + 565, 567, 0, 583, 583, 583, 585, 0, + 0, 0 + }; + + static const signed char _regex_to_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 41, + 0, 41, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 41, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 41, 0, 0, 41, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 41, 41, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 41, 0, 41, 0, + 0, 0, 0, 41, 0, 0, 0, 0, + 41, 41, 0 + }; + + static const signed char _regex_from_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 43, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 43, 0, 0, 43, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 43, 43, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 43, 0, 43, 0, + 0, 0, 0, 43, 0, 0, 0, 0, + 43, 43, 0 + }; + + static const short _regex_eof_trans[] = { + 2665, 2666, 2667, 2668, 2669, 2670, 2671, 2672, + 2673, 2674, 2675, 2676, 2677, 2678, 2679, 2680, + 2681, 2682, 2683, 2684, 2685, 2686, 2687, 2688, + 2689, 2690, 2691, 2692, 2693, 2694, 2695, 2696, + 2697, 2698, 2699, 2700, 2701, 2702, 2703, 2704, + 2705, 2706, 2707, 2708, 2709, 2710, 2711, 2712, + 2713, 2714, 2715, 2716, 2717, 2718, 2719, 2720, + 2721, 2722, 2723, 2724, 2725, 2726, 2727, 2728, + 2729, 2730, 2731, 2732, 2733, 2734, 2735, 2736, + 2737, 2738, 2739, 2740, 2741, 2742, 2743, 2744, + 2745, 2746, 2747, 2748, 2749, 2750, 2751, 2752, + 2753, 2754, 2755, 2756, 2757, 2758, 2759, 2760, + 2761, 2762, 2763, 2764, 2765, 2766, 2767, 2768, + 2769, 2770, 2771, 2772, 2773, 2774, 2775, 2776, + 2777, 2778, 2779, 2780, 2781, 2782, 2783, 2784, + 2785, 2786, 2787, 2788, 2789, 2790, 2791, 2792, + 2793, 2794, 2795, 2796, 2797, 2798, 2799, 2800, + 2801, 2802, 2803, 2804, 2805, 2806, 2807, 2808, + 2809, 2810, 2811, 2812, 2813, 2814, 2815, 2816, + 2817, 2818, 2819, 2820, 2821, 2822, 2823, 2824, + 2825, 2826, 2827, 2828, 2829, 2830, 2831, 2832, + 2833, 2834, 2835, 2836, 2837, 2838, 2839, 2840, + 2841, 2842, 2843, 2844, 2845, 2846, 2847, 2848, + 2849, 2850, 2851, 2852, 2853, 2854, 2855, 2856, + 2857, 2858, 2859, 2860, 2861, 2862, 2863, 2864, + 2865, 2866, 2867, 2868, 2869, 2870, 2871, 2872, + 2873, 2874, 2875, 2876, 2877, 2878, 2879, 2880, + 2881, 2882, 2883, 2884, 2885, 2886, 2887, 2888, + 2889, 2890, 2891, 2892, 2893, 2894, 2895, 2896, + 2897, 2898, 2899, 2900, 2901, 2902, 2903, 2904, + 2905, 2906, 2907, 2908, 2909, 2910, 2911, 2912, + 2913, 2914, 2915, 2916, 2917, 2918, 2919, 2920, + 2921, 2922, 2923, 2924, 2925, 2926, 2927, 2928, + 2929, 2930, 2931, 2932, 2933, 2934, 2935, 2936, + 2937, 2938, 2939, 2940, 2941, 2942, 2943, 2944, + 2945, 2946, 2947, 2948, 2949, 2950, 2951, 2952, + 2953, 2954, 2955, 2956, 2957, 2958, 2959, 2960, + 2961, 2962, 2963, 2964, 2965, 2966, 2967, 2968, + 2969, 2970, 2971, 2972, 2973, 2974, 2975, 2976, + 2977, 2978, 2979, 2980, 2981, 2982, 2983, 2984, + 2985, 2986, 2987, 2988, 2989, 2990, 2991, 2992, + 2993, 2994, 2995, 2996, 2997, 2998, 2999, 3000, + 3001, 3002, 3003, 3004, 3005, 3006, 3007, 3008, + 3009, 3010, 3011, 3012, 3013, 3014, 3015, 3016, + 3017, 3018, 3019, 3020, 3021, 3022, 3023, 3024, + 3025, 3026, 3027, 3028, 3029, 3030, 3031, 3032, + 3033, 3034, 3035, 3036, 3037, 3038, 3039, 3040, + 3041, 3042, 3043, 3044, 3045, 3046, 3047, 3048, + 3049, 3050, 3051, 3052, 3053, 3054, 3055, 3056, + 3057, 3058, 3059, 3060, 3061, 3062, 3063, 3064, + 3065, 3066, 3067, 3068, 3069, 3070, 3071, 3072, + 3073, 3074, 3075, 3076, 3077, 3078, 3079, 3080, + 3081, 3082, 3083, 3084, 3085, 3086, 3087, 3088, + 3089, 3090, 3091, 3092, 3093, 3094, 3095, 3096, + 3097, 3098, 3099, 3100, 3101, 3102, 3103, 3104, + 3105, 3106, 3107, 3108, 3109, 3110, 3111, 3112, + 3113, 3114, 3115, 3116, 3117, 3118, 3119, 3120, + 3121, 3122, 3123, 3124, 3125, 3126, 3127, 3128, + 3129, 3130, 3131, 3132, 3133, 3134, 3135, 3136, + 3137, 3138, 3139, 3140, 3141, 3142, 3143, 3144, + 3145, 3146, 3147, 3148, 3149, 3150, 3151, 3152, + 3153, 3154, 3155, 3156, 3157, 3158, 3159, 3160, + 3161, 3162, 3163, 3164, 3165, 3166, 3167, 3168, + 3169, 3170, 3171, 3172, 3173, 3174, 3175, 3176, + 3177, 3178, 3179, 3180, 3181, 3182, 3183, 3184, + 3185, 3186, 3187, 3188, 3189, 3190, 3191, 3192, + 3193, 3194, 3195, 3196, 3197, 3198, 3199, 3200, + 3201, 3202, 3203, 3204, 3205, 3206, 3207, 3208, + 3209, 3210, 3211, 3212, 3213, 3214, 3215, 3216, + 3217, 3218, 3219, 3220, 3221, 3222, 3223, 3224, + 3225, 3226, 3227, 3228, 3229, 3230, 3231, 3232, + 3233, 3234, 3235, 3236, 3237, 3238, 3239, 3240, + 3241, 3242, 3243, 3244, 3245, 3246, 3247, 3248, + 3249, 3250, 3251, 3252, 3253, 3254, 3255, 3256, + 3257, 3258, 3259, 3260, 3261, 3262, 3263, 3264, + 3265, 3266, 3267, 3268, 3269, 3270, 3271, 3272, + 3273, 3274, 3275, 3276, 3277, 3278, 3279, 3280, + 3281, 3282, 3283, 3284, 3285, 3286, 3287, 3288, + 3289, 3290, 3291, 3292, 3293, 3294, 3295, 3296, + 3297, 3298, 3299, 3300, 3301, 3302, 3303, 3304, + 3305, 3306, 3307, 3308, 3309, 3310, 3311, 3312, + 3313, 3314, 3315, 3316, 3317, 3318, 3319, 3320, + 3321, 3322, 3323, 3324, 3325, 3326, 3327, 3328, + 3329, 3330, 3331, 3332, 3333, 3334, 3335, 3336, + 3337, 3338, 3339, 3340, 3341, 3342, 3343, 3344, + 3345, 3346, 3347, 3348, 3349, 3350, 3351, 3352, + 3353, 3354, 3355, 3356, 3357, 3358, 3359, 3360, + 3361, 3362, 3363, 3364, 3365, 3366, 3367, 3368, + 3369, 3370, 3371, 3372, 3373, 3374, 3375, 3376, + 3377, 3378, 3379, 3380, 3381, 3382, 3383, 3384, + 3385, 3386, 3387, 3388, 3389, 3390, 3391, 3392, + 3393, 3394, 3395, 3396, 3397, 3398, 3399, 3400, + 3401, 3402, 3403, 3404, 3405, 3406, 3407, 3408, + 3409, 3410, 3411, 3412, 3413, 3414, 3415, 3416, + 3417, 3418, 3419, 3420, 3421, 3422, 3423, 3424, + 3425, 3426, 3427, 3428, 3429, 3430, 3431, 3432, + 3433, 3434, 3435, 3436, 3437, 3438, 3439, 3440, + 3441, 3442, 3443, 3444, 3445, 3446, 3447, 3448, + 3449, 3450, 3451, 3452, 3453, 3454, 3455, 3456, + 3457, 3458, 3459, 3460, 3461, 3462, 3463, 3464, + 3465, 3466, 3467, 3468, 3469, 3470, 3471, 3472, + 3473, 3474, 3475, 3476, 3477, 3478, 3479, 3480, + 3481, 3482, 3483, 3484, 3485, 3486, 3487, 3488, + 3489, 3490, 3491, 3492, 3493, 3494, 3495, 3496, + 3497, 3498, 3499, 3500, 3501, 3502, 3503, 3504, + 3505, 3506, 3507, 3508, 3509, 3510, 3511, 3512, + 3513, 3514, 0 + }; + + static const int regex_start = 746; + static const int regex_error = 0; + + static const int regex_en_readVerb = 787; + static const int regex_en_readUCP = 790; + static const int regex_en_readBracedUCP = 559; + static const int regex_en_readUCPSingle = 818; + static const int regex_en_charClassGuts = 819; + static const int regex_en_readClass = 836; + static const int regex_en_readQuotedLiteral = 838; + static const int regex_en_readQuotedClass = 843; + static const int regex_en_readComment = 848; + static const int regex_en_readNewlineTerminatedComment = 849; + static const int regex_en_main = 746; + + +#line 1911 "Parser.rl" + + + /** \brief Main parser call, returns root Component or nullptr. */ + unique_ptr parse(const char *ptr, ParseMode &globalMode) { + assert(ptr); + + const char *p = ptr; + const char *pe = ptr + strlen(ptr); + + // First, read the control verbs, set any global mode flags and move the + // ptr forward. + p = read_control_verbs(p, pe, 0, globalMode); + + const char *eof = pe; + int cs; + UNUSED int act; + int top; + vector stack; + const char *ts, *te; + unichar accumulator = 0; + unichar octAccumulator = 0; /* required as we are also accumulating for + * back ref when looking for octals */ + unsigned repeatN = 0; + unsigned repeatM = 0; + string label; + + ParseMode mode = globalMode; + ParseMode newMode; + + bool negated = false; + bool inComment = false; + + // Stack of sequences and flags used to store state when we enter + // sub-sequences. + vector sequences; + + // Index of the next capturing group. Note that zero is reserved for the + // root sequence. + unsigned groupIndex = 1; + + // Set storing group names that are currently in use. + flat_set groupNames; + + // Root sequence. + unique_ptr rootSeq = std::make_unique(); + rootSeq->setCaptureIndex(0); + + // Current sequence being appended to + ComponentSequence *currentSeq = rootSeq.get(); + + // The current character class being appended to. This is used as the + // accumulator for both character class and UCP properties. + unique_ptr currentCls; + + // True if the machine is currently inside a character class, i.e. square + // brackets [..]. + bool inCharClass = false; + + // True if the machine is inside a character class but it has not processed + // any "real" elements yet, i.e. it's still processing meta-characters like + // '^'. + bool inCharClassEarly = false; + + // Location at which the current character class began. + const char *currentClsBegin = p; + + // We throw exceptions on various parsing failures beyond this point: we + // use a try/catch block here to clean up our allocated memory before we + // re-throw the exception to the caller. + try { + // Embed the Ragel machine here + +#line 4187 "Parser.cpp" + { + cs = (int)regex_start; + top = 0; + ts = 0; + te = 0; + act = 0; + } + +#line 1982 "Parser.rl" + + +#line 4196 "Parser.cpp" + { + int _cpc; + int _klen; + const signed char * _ckeys; + unsigned int _trans = 0; + unsigned int _cond = 0; + const char * _keys; + const short * _acts; + unsigned int _nacts; + _resume: {} + if ( p == pe && p != eof ) + goto _out; + _acts = ( _regex_actions + (_regex_from_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 24: { + { +#line 1 "NONE" + {ts = p;}} + +#line 4218 "Parser.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + if ( p == eof ) { + if ( _regex_eof_trans[cs] > 0 ) { + _trans = (unsigned int)_regex_eof_trans[cs] - 1; + } + } + else { + _keys = ( _regex_trans_keys + (_regex_key_offsets[cs])); + _trans = (unsigned int)_regex_index_offsets[cs]; + + _klen = (int)_regex_single_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + _klen - 1; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _keys += _klen; + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + ((_upper-_lower) >> 1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 1; + else if ( ( (*( p))) > (*( _mid)) ) + _lower = _mid + 1; + else { + _trans += (unsigned int)(_mid - _keys); + goto _match; + } + } + } + + _klen = (int)_regex_range_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + (_klen<<1) - 2; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + (((_upper-_lower) >> 1) & ~1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 2; + else if ( ( (*( p))) > (*( _mid + 1)) ) + _lower = _mid + 2; + else { + _trans += (unsigned int)((_mid - _keys)>>1); + break; + } + } + } + + _match: {} + } + _ckeys = ( _regex_cond_keys + (_regex_trans_offsets[_trans])); + _klen = (int)_regex_trans_lengths[_trans]; + _cond = (unsigned int)_regex_trans_offsets[_trans]; + + _cpc = 0; + switch ( _regex_trans_cond_spaces[_trans] ) { + + case 0: { + if ( ( mode.utf8 ) +#line 4293 "Parser.cpp" + ) _cpc += 1; + break; + } + case 1: { + if ( ( mode.ignore_space ) +#line 4298 "Parser.cpp" + ) _cpc += 1; + break; + } + case 2: { + if ( ( inCharClassEarly ) +#line 4303 "Parser.cpp" + ) _cpc += 1; + break; + } + } + { + const signed char *_lower = _ckeys; + const signed char *_upper = _ckeys + _klen - 1; + const signed char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _cond = 3561; + + break; + } + + _mid = _lower + ((_upper-_lower) >> 1); + if ( _cpc < (int)(*( _mid)) ) + _upper = _mid - 1; + else if ( _cpc > (int)(*( _mid)) ) + _lower = _mid + 1; + else { + _cond += (unsigned int)(_mid - _ckeys); + + break; + } + } + } + cs = (int)_regex_cond_targs[_cond]; + + if ( _regex_cond_actions[_cond] != 0 ) { + + _acts = ( _regex_actions + (_regex_cond_actions[_cond])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) + { + case 0: { + { +#line 285 "Parser.rl" + label.clear();} + +#line 4345 "Parser.cpp" + + break; + } + case 1: { + { +#line 286 "Parser.rl" + label.push_back((( (*( p)))));} + +#line 4353 "Parser.cpp" + + break; + } + case 2: { + { +#line 287 "Parser.rl" + octAccumulator = 0;} + +#line 4361 "Parser.cpp" + + break; + } + case 3: { + { +#line 288 "Parser.rl" + accumulator = 0;} + +#line 4369 "Parser.cpp" + + break; + } + case 4: { + { +#line 289 "Parser.rl" + + octAccumulator = 0; + pushOct(&octAccumulator, (( (*( p))))); + } + +#line 4380 "Parser.cpp" + + break; + } + case 5: { + { +#line 293 "Parser.rl" + + accumulator = 0; + pushDec(&accumulator, (( (*( p))))); + } + +#line 4391 "Parser.cpp" + + break; + } + case 6: { + { +#line 297 "Parser.rl" + repeatN = 0; repeatM = 0; } + +#line 4399 "Parser.cpp" + + break; + } + case 7: { + { +#line 298 "Parser.rl" + pushDec(&repeatN, (( (*( p))))); } + +#line 4407 "Parser.cpp" + + break; + } + case 8: { + { +#line 299 "Parser.rl" + pushDec(&repeatM, (( (*( p))))); } + +#line 4415 "Parser.cpp" + + break; + } + case 9: { + { +#line 300 "Parser.rl" + pushOct(&octAccumulator, (( (*( p))))); } + +#line 4423 "Parser.cpp" + + break; + } + case 10: { + { +#line 301 "Parser.rl" + pushDec(&accumulator, (( (*( p))))); } + +#line 4431 "Parser.cpp" + + break; + } + case 11: { + { +#line 302 "Parser.rl" + + accumulator *= 16; + accumulator += (( (*( p)))) - '0'; + } + +#line 4442 "Parser.cpp" + + break; + } + case 12: { + { +#line 306 "Parser.rl" + + accumulator *= 16; + accumulator += 10 + (( (*( p)))) - 'a'; + } + +#line 4453 "Parser.cpp" + + break; + } + case 13: { + { +#line 310 "Parser.rl" + + accumulator *= 16; + accumulator += 10 + (( (*( p)))) - 'A'; + } + +#line 4464 "Parser.cpp" + + break; + } + case 14: { + { +#line 430 "Parser.rl" + + newMode = mode; + } + +#line 4474 "Parser.cpp" + + break; + } + case 15: { + { +#line 437 "Parser.rl" + + switch ((( (*( p))))) { + case 'i': + newMode.caseless = true; + break; + case 'm': + newMode.multiline = true; + break; + case 's': + newMode.dotall = true; + break; + case 'x': + newMode.ignore_space = true; + break; + default: + assert(0); // this action only called for [imsx] + break; + } + } + +#line 4500 "Parser.cpp" + + break; + } + case 16: { + { +#line 456 "Parser.rl" + + switch ((( (*( p))))) { + case 'i': + newMode.caseless = false; + break; + case 'm': + newMode.multiline = false; + break; + case 's': + newMode.dotall = false; + break; + case 'x': + newMode.ignore_space = false; + break; + default: + assert(0); // this action only called for [imsx] + break; + } + } + +#line 4526 "Parser.cpp" + + break; + } + case 17: { + { +#line 510 "Parser.rl" + repeatM = repeatN;} + +#line 4534 "Parser.cpp" + + break; + } + case 18: { + { +#line 510 "Parser.rl" + repeatM = ComponentRepeat::NoLimit;} + +#line 4542 "Parser.cpp" + + break; + } + case 19: { + { +#line 722 "Parser.rl" + negated = !negated; } + +#line 4550 "Parser.cpp" + + break; + } + case 20: { + { +#line 723 "Parser.rl" + {p = p - 1; } {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 790;goto _again;}} + +#line 4566 "Parser.cpp" + + break; + } + case 21: { + { +#line 724 "Parser.rl" + if (!inCharClass) { // not inside [..] + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + +#line 4579 "Parser.cpp" + + break; + } + case 22: { + { +#line 730 "Parser.rl" + throw LocatedParseError("Malformed property"); } + +#line 4587 "Parser.cpp" + + break; + } + case 25: { + { +#line 1 "NONE" + {te = p+1;}} + +#line 4595 "Parser.cpp" + + break; + } + case 26: { + { +#line 550 "Parser.rl" + {te = p+1;{ +#line 550 "Parser.rl" + + throw LocatedParseError("(*UTF8) must be at start of " + "expression, encountered"); + } + }} + +#line 4609 "Parser.cpp" + + break; + } + case 27: { + { +#line 554 "Parser.rl" + {te = p+1;{ +#line 554 "Parser.rl" + + throw LocatedParseError("(*UTF) must be at start of " + "expression, encountered"); + } + }} + +#line 4623 "Parser.cpp" + + break; + } + case 28: { + { +#line 558 "Parser.rl" + {te = p+1;{ +#line 558 "Parser.rl" + + throw LocatedParseError("(*UCP) must be at start of " + "expression, encountered"); + } + }} + +#line 4637 "Parser.cpp" + + break; + } + case 29: { + { +#line 564 "Parser.rl" + {te = p+1;{ +#line 564 "Parser.rl" + + ParseMode temp_mode; + assert(ts - 2 >= ptr); // parser needs the '(*' at the start too. + read_control_verbs(ts - 2, te, (ts - 2 - ptr), temp_mode); + assert(0); // Should have thrown a parse error. + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4654 "Parser.cpp" + + break; + } + case 30: { + { +#line 571 "Parser.rl" + {te = p+1;{ +#line 571 "Parser.rl" + + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4667 "Parser.cpp" + + break; + } + case 31: { + { +#line 571 "Parser.rl" + {te = p;p = p - 1;{ +#line 571 "Parser.rl" + + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4680 "Parser.cpp" + + break; + } + case 32: { + { +#line 571 "Parser.rl" + {p = ((te))-1; + { +#line 571 "Parser.rl" + + throw LocatedParseError("Unknown control verb"); + } + }} + +#line 4694 "Parser.cpp" + + break; + } + case 33: { + { +#line 581 "Parser.rl" + {te = p+1;{ +#line 581 "Parser.rl" + currentCls->add(CLASS_UCP_CC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4705 "Parser.cpp" + + break; + } + case 34: { + { +#line 582 "Parser.rl" + {te = p+1;{ +#line 582 "Parser.rl" + currentCls->add(CLASS_UCP_CF, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4716 "Parser.cpp" + + break; + } + case 35: { + { +#line 583 "Parser.rl" + {te = p+1;{ +#line 583 "Parser.rl" + currentCls->add(CLASS_UCP_CN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4727 "Parser.cpp" + + break; + } + case 36: { + { +#line 585 "Parser.rl" + {te = p+1;{ +#line 585 "Parser.rl" + currentCls->add(CLASS_UCP_CS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4738 "Parser.cpp" + + break; + } + case 37: { + { +#line 587 "Parser.rl" + {te = p+1;{ +#line 587 "Parser.rl" + currentCls->add(CLASS_UCP_LL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4749 "Parser.cpp" + + break; + } + case 38: { + { +#line 588 "Parser.rl" + {te = p+1;{ +#line 588 "Parser.rl" + currentCls->add(CLASS_UCP_LM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4760 "Parser.cpp" + + break; + } + case 39: { + { +#line 589 "Parser.rl" + {te = p+1;{ +#line 589 "Parser.rl" + currentCls->add(CLASS_UCP_LO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4771 "Parser.cpp" + + break; + } + case 40: { + { +#line 590 "Parser.rl" + {te = p+1;{ +#line 590 "Parser.rl" + currentCls->add(CLASS_UCP_LT, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4782 "Parser.cpp" + + break; + } + case 41: { + { +#line 591 "Parser.rl" + {te = p+1;{ +#line 591 "Parser.rl" + currentCls->add(CLASS_UCP_LU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4793 "Parser.cpp" + + break; + } + case 42: { + { +#line 592 "Parser.rl" + {te = p+1;{ +#line 592 "Parser.rl" + currentCls->add(CLASS_UCP_L_AND, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4804 "Parser.cpp" + + break; + } + case 43: { + { +#line 594 "Parser.rl" + {te = p+1;{ +#line 594 "Parser.rl" + currentCls->add(CLASS_UCP_MC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4815 "Parser.cpp" + + break; + } + case 44: { + { +#line 596 "Parser.rl" + {te = p+1;{ +#line 596 "Parser.rl" + currentCls->add(CLASS_UCP_MN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4826 "Parser.cpp" + + break; + } + case 45: { + { +#line 598 "Parser.rl" + {te = p+1;{ +#line 598 "Parser.rl" + currentCls->add(CLASS_UCP_ND, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4837 "Parser.cpp" + + break; + } + case 46: { + { +#line 599 "Parser.rl" + {te = p+1;{ +#line 599 "Parser.rl" + currentCls->add(CLASS_UCP_NL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4848 "Parser.cpp" + + break; + } + case 47: { + { +#line 600 "Parser.rl" + {te = p+1;{ +#line 600 "Parser.rl" + currentCls->add(CLASS_UCP_NO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4859 "Parser.cpp" + + break; + } + case 48: { + { +#line 602 "Parser.rl" + {te = p+1;{ +#line 602 "Parser.rl" + currentCls->add(CLASS_UCP_PC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4870 "Parser.cpp" + + break; + } + case 49: { + { +#line 603 "Parser.rl" + {te = p+1;{ +#line 603 "Parser.rl" + currentCls->add(CLASS_UCP_PD, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4881 "Parser.cpp" + + break; + } + case 50: { + { +#line 604 "Parser.rl" + {te = p+1;{ +#line 604 "Parser.rl" + currentCls->add(CLASS_UCP_PE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4892 "Parser.cpp" + + break; + } + case 51: { + { +#line 605 "Parser.rl" + {te = p+1;{ +#line 605 "Parser.rl" + currentCls->add(CLASS_UCP_PF, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4903 "Parser.cpp" + + break; + } + case 52: { + { +#line 606 "Parser.rl" + {te = p+1;{ +#line 606 "Parser.rl" + currentCls->add(CLASS_UCP_PI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4914 "Parser.cpp" + + break; + } + case 53: { + { +#line 607 "Parser.rl" + {te = p+1;{ +#line 607 "Parser.rl" + currentCls->add(CLASS_UCP_PO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4925 "Parser.cpp" + + break; + } + case 54: { + { +#line 608 "Parser.rl" + {te = p+1;{ +#line 608 "Parser.rl" + currentCls->add(CLASS_UCP_PS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4936 "Parser.cpp" + + break; + } + case 55: { + { +#line 610 "Parser.rl" + {te = p+1;{ +#line 610 "Parser.rl" + currentCls->add(CLASS_UCP_SC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4947 "Parser.cpp" + + break; + } + case 56: { + { +#line 611 "Parser.rl" + {te = p+1;{ +#line 611 "Parser.rl" + currentCls->add(CLASS_UCP_SK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4958 "Parser.cpp" + + break; + } + case 57: { + { +#line 612 "Parser.rl" + {te = p+1;{ +#line 612 "Parser.rl" + currentCls->add(CLASS_UCP_SM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4969 "Parser.cpp" + + break; + } + case 58: { + { +#line 613 "Parser.rl" + {te = p+1;{ +#line 613 "Parser.rl" + currentCls->add(CLASS_UCP_SO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4980 "Parser.cpp" + + break; + } + case 59: { + { +#line 615 "Parser.rl" + {te = p+1;{ +#line 615 "Parser.rl" + currentCls->add(CLASS_UCP_ZL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 4991 "Parser.cpp" + + break; + } + case 60: { + { +#line 616 "Parser.rl" + {te = p+1;{ +#line 616 "Parser.rl" + currentCls->add(CLASS_UCP_ZP, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5002 "Parser.cpp" + + break; + } + case 61: { + { +#line 617 "Parser.rl" + {te = p+1;{ +#line 617 "Parser.rl" + currentCls->add(CLASS_UCP_ZS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5013 "Parser.cpp" + + break; + } + case 62: { + { +#line 618 "Parser.rl" + {te = p+1;{ +#line 618 "Parser.rl" + currentCls->add(CLASS_UCP_XAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5024 "Parser.cpp" + + break; + } + case 63: { + { +#line 619 "Parser.rl" + {te = p+1;{ +#line 619 "Parser.rl" + currentCls->add(CLASS_UCP_XPS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5035 "Parser.cpp" + + break; + } + case 64: { + { +#line 620 "Parser.rl" + {te = p+1;{ +#line 620 "Parser.rl" + currentCls->add(CLASS_UCP_XSP, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5046 "Parser.cpp" + + break; + } + case 65: { + { +#line 621 "Parser.rl" + {te = p+1;{ +#line 621 "Parser.rl" + currentCls->add(CLASS_UCP_XWD, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5057 "Parser.cpp" + + break; + } + case 66: { + { +#line 622 "Parser.rl" + {te = p+1;{ +#line 622 "Parser.rl" + currentCls->add(CLASS_SCRIPT_ARABIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5068 "Parser.cpp" + + break; + } + case 67: { + { +#line 623 "Parser.rl" + {te = p+1;{ +#line 623 "Parser.rl" + currentCls->add(CLASS_SCRIPT_ARMENIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5079 "Parser.cpp" + + break; + } + case 68: { + { +#line 624 "Parser.rl" + {te = p+1;{ +#line 624 "Parser.rl" + currentCls->add(CLASS_SCRIPT_AVESTAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5090 "Parser.cpp" + + break; + } + case 69: { + { +#line 625 "Parser.rl" + {te = p+1;{ +#line 625 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BALINESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5101 "Parser.cpp" + + break; + } + case 70: { + { +#line 626 "Parser.rl" + {te = p+1;{ +#line 626 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BAMUM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5112 "Parser.cpp" + + break; + } + case 71: { + { +#line 627 "Parser.rl" + {te = p+1;{ +#line 627 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BATAK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5123 "Parser.cpp" + + break; + } + case 72: { + { +#line 628 "Parser.rl" + {te = p+1;{ +#line 628 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BENGALI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5134 "Parser.cpp" + + break; + } + case 73: { + { +#line 629 "Parser.rl" + {te = p+1;{ +#line 629 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BOPOMOFO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5145 "Parser.cpp" + + break; + } + case 74: { + { +#line 630 "Parser.rl" + {te = p+1;{ +#line 630 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BRAHMI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5156 "Parser.cpp" + + break; + } + case 75: { + { +#line 631 "Parser.rl" + {te = p+1;{ +#line 631 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BRAILLE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5167 "Parser.cpp" + + break; + } + case 76: { + { +#line 632 "Parser.rl" + {te = p+1;{ +#line 632 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BUGINESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5178 "Parser.cpp" + + break; + } + case 77: { + { +#line 633 "Parser.rl" + {te = p+1;{ +#line 633 "Parser.rl" + currentCls->add(CLASS_SCRIPT_BUHID, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5189 "Parser.cpp" + + break; + } + case 78: { + { +#line 634 "Parser.rl" + {te = p+1;{ +#line 634 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CANADIAN_ABORIGINAL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5200 "Parser.cpp" + + break; + } + case 79: { + { +#line 635 "Parser.rl" + {te = p+1;{ +#line 635 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CARIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5211 "Parser.cpp" + + break; + } + case 80: { + { +#line 636 "Parser.rl" + {te = p+1;{ +#line 636 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CHAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5222 "Parser.cpp" + + break; + } + case 81: { + { +#line 637 "Parser.rl" + {te = p+1;{ +#line 637 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CHEROKEE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5233 "Parser.cpp" + + break; + } + case 82: { + { +#line 638 "Parser.rl" + {te = p+1;{ +#line 638 "Parser.rl" + currentCls->add(CLASS_SCRIPT_COMMON, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5244 "Parser.cpp" + + break; + } + case 83: { + { +#line 639 "Parser.rl" + {te = p+1;{ +#line 639 "Parser.rl" + currentCls->add(CLASS_SCRIPT_COPTIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5255 "Parser.cpp" + + break; + } + case 84: { + { +#line 640 "Parser.rl" + {te = p+1;{ +#line 640 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CUNEIFORM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5266 "Parser.cpp" + + break; + } + case 85: { + { +#line 641 "Parser.rl" + {te = p+1;{ +#line 641 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CYPRIOT, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5277 "Parser.cpp" + + break; + } + case 86: { + { +#line 642 "Parser.rl" + {te = p+1;{ +#line 642 "Parser.rl" + currentCls->add(CLASS_SCRIPT_CYRILLIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5288 "Parser.cpp" + + break; + } + case 87: { + { +#line 643 "Parser.rl" + {te = p+1;{ +#line 643 "Parser.rl" + currentCls->add(CLASS_SCRIPT_DESERET, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5299 "Parser.cpp" + + break; + } + case 88: { + { +#line 644 "Parser.rl" + {te = p+1;{ +#line 644 "Parser.rl" + currentCls->add(CLASS_SCRIPT_DEVANAGARI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5310 "Parser.cpp" + + break; + } + case 89: { + { +#line 645 "Parser.rl" + {te = p+1;{ +#line 645 "Parser.rl" + currentCls->add(CLASS_SCRIPT_EGYPTIAN_HIEROGLYPHS, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5321 "Parser.cpp" + + break; + } + case 90: { + { +#line 646 "Parser.rl" + {te = p+1;{ +#line 646 "Parser.rl" + currentCls->add(CLASS_SCRIPT_ETHIOPIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5332 "Parser.cpp" + + break; + } + case 91: { + { +#line 647 "Parser.rl" + {te = p+1;{ +#line 647 "Parser.rl" + currentCls->add(CLASS_SCRIPT_GEORGIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5343 "Parser.cpp" + + break; + } + case 92: { + { +#line 648 "Parser.rl" + {te = p+1;{ +#line 648 "Parser.rl" + currentCls->add(CLASS_SCRIPT_GLAGOLITIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5354 "Parser.cpp" + + break; + } + case 93: { + { +#line 649 "Parser.rl" + {te = p+1;{ +#line 649 "Parser.rl" + currentCls->add(CLASS_SCRIPT_GOTHIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5365 "Parser.cpp" + + break; + } + case 94: { + { +#line 650 "Parser.rl" + {te = p+1;{ +#line 650 "Parser.rl" + currentCls->add(CLASS_SCRIPT_GREEK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5376 "Parser.cpp" + + break; + } + case 95: { + { +#line 651 "Parser.rl" + {te = p+1;{ +#line 651 "Parser.rl" + currentCls->add(CLASS_SCRIPT_GUJARATI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5387 "Parser.cpp" + + break; + } + case 96: { + { +#line 652 "Parser.rl" + {te = p+1;{ +#line 652 "Parser.rl" + currentCls->add(CLASS_SCRIPT_GURMUKHI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5398 "Parser.cpp" + + break; + } + case 97: { + { +#line 654 "Parser.rl" + {te = p+1;{ +#line 654 "Parser.rl" + currentCls->add(CLASS_SCRIPT_HANGUL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5409 "Parser.cpp" + + break; + } + case 98: { + { +#line 655 "Parser.rl" + {te = p+1;{ +#line 655 "Parser.rl" + currentCls->add(CLASS_SCRIPT_HANUNOO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5420 "Parser.cpp" + + break; + } + case 99: { + { +#line 656 "Parser.rl" + {te = p+1;{ +#line 656 "Parser.rl" + currentCls->add(CLASS_SCRIPT_HEBREW, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5431 "Parser.cpp" + + break; + } + case 100: { + { +#line 657 "Parser.rl" + {te = p+1;{ +#line 657 "Parser.rl" + currentCls->add(CLASS_SCRIPT_HIRAGANA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5442 "Parser.cpp" + + break; + } + case 101: { + { +#line 658 "Parser.rl" + {te = p+1;{ +#line 658 "Parser.rl" + currentCls->add(CLASS_SCRIPT_IMPERIAL_ARAMAIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5453 "Parser.cpp" + + break; + } + case 102: { + { +#line 659 "Parser.rl" + {te = p+1;{ +#line 659 "Parser.rl" + currentCls->add(CLASS_SCRIPT_INHERITED, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5464 "Parser.cpp" + + break; + } + case 103: { + { +#line 660 "Parser.rl" + {te = p+1;{ +#line 660 "Parser.rl" + currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PAHLAVI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5475 "Parser.cpp" + + break; + } + case 104: { + { +#line 661 "Parser.rl" + {te = p+1;{ +#line 661 "Parser.rl" + currentCls->add(CLASS_SCRIPT_INSCRIPTIONAL_PARTHIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5486 "Parser.cpp" + + break; + } + case 105: { + { +#line 662 "Parser.rl" + {te = p+1;{ +#line 662 "Parser.rl" + currentCls->add(CLASS_SCRIPT_JAVANESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5497 "Parser.cpp" + + break; + } + case 106: { + { +#line 663 "Parser.rl" + {te = p+1;{ +#line 663 "Parser.rl" + currentCls->add(CLASS_SCRIPT_KAITHI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5508 "Parser.cpp" + + break; + } + case 107: { + { +#line 664 "Parser.rl" + {te = p+1;{ +#line 664 "Parser.rl" + currentCls->add(CLASS_SCRIPT_KANNADA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5519 "Parser.cpp" + + break; + } + case 108: { + { +#line 665 "Parser.rl" + {te = p+1;{ +#line 665 "Parser.rl" + currentCls->add(CLASS_SCRIPT_KATAKANA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5530 "Parser.cpp" + + break; + } + case 109: { + { +#line 666 "Parser.rl" + {te = p+1;{ +#line 666 "Parser.rl" + currentCls->add(CLASS_SCRIPT_KAYAH_LI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5541 "Parser.cpp" + + break; + } + case 110: { + { +#line 667 "Parser.rl" + {te = p+1;{ +#line 667 "Parser.rl" + currentCls->add(CLASS_SCRIPT_KHAROSHTHI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5552 "Parser.cpp" + + break; + } + case 111: { + { +#line 668 "Parser.rl" + {te = p+1;{ +#line 668 "Parser.rl" + currentCls->add(CLASS_SCRIPT_KHMER, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5563 "Parser.cpp" + + break; + } + case 112: { + { +#line 669 "Parser.rl" + {te = p+1;{ +#line 669 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LAO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5574 "Parser.cpp" + + break; + } + case 113: { + { +#line 670 "Parser.rl" + {te = p+1;{ +#line 670 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LATIN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5585 "Parser.cpp" + + break; + } + case 114: { + { +#line 671 "Parser.rl" + {te = p+1;{ +#line 671 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LEPCHA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5596 "Parser.cpp" + + break; + } + case 115: { + { +#line 672 "Parser.rl" + {te = p+1;{ +#line 672 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LIMBU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5607 "Parser.cpp" + + break; + } + case 116: { + { +#line 673 "Parser.rl" + {te = p+1;{ +#line 673 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LINEAR_B, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5618 "Parser.cpp" + + break; + } + case 117: { + { +#line 674 "Parser.rl" + {te = p+1;{ +#line 674 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LISU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5629 "Parser.cpp" + + break; + } + case 118: { + { +#line 675 "Parser.rl" + {te = p+1;{ +#line 675 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LYCIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5640 "Parser.cpp" + + break; + } + case 119: { + { +#line 676 "Parser.rl" + {te = p+1;{ +#line 676 "Parser.rl" + currentCls->add(CLASS_SCRIPT_LYDIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5651 "Parser.cpp" + + break; + } + case 120: { + { +#line 677 "Parser.rl" + {te = p+1;{ +#line 677 "Parser.rl" + currentCls->add(CLASS_SCRIPT_MALAYALAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5662 "Parser.cpp" + + break; + } + case 121: { + { +#line 678 "Parser.rl" + {te = p+1;{ +#line 678 "Parser.rl" + currentCls->add(CLASS_SCRIPT_MANDAIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5673 "Parser.cpp" + + break; + } + case 122: { + { +#line 679 "Parser.rl" + {te = p+1;{ +#line 679 "Parser.rl" + currentCls->add(CLASS_SCRIPT_MEETEI_MAYEK, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5684 "Parser.cpp" + + break; + } + case 123: { + { +#line 680 "Parser.rl" + {te = p+1;{ +#line 680 "Parser.rl" + currentCls->add(CLASS_SCRIPT_MONGOLIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5695 "Parser.cpp" + + break; + } + case 124: { + { +#line 681 "Parser.rl" + {te = p+1;{ +#line 681 "Parser.rl" + currentCls->add(CLASS_SCRIPT_MYANMAR, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5706 "Parser.cpp" + + break; + } + case 125: { + { +#line 682 "Parser.rl" + {te = p+1;{ +#line 682 "Parser.rl" + currentCls->add(CLASS_SCRIPT_NEW_TAI_LUE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5717 "Parser.cpp" + + break; + } + case 126: { + { +#line 683 "Parser.rl" + {te = p+1;{ +#line 683 "Parser.rl" + currentCls->add(CLASS_SCRIPT_NKO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5728 "Parser.cpp" + + break; + } + case 127: { + { +#line 684 "Parser.rl" + {te = p+1;{ +#line 684 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OGHAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5739 "Parser.cpp" + + break; + } + case 128: { + { +#line 685 "Parser.rl" + {te = p+1;{ +#line 685 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OL_CHIKI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5750 "Parser.cpp" + + break; + } + case 129: { + { +#line 686 "Parser.rl" + {te = p+1;{ +#line 686 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_ITALIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5761 "Parser.cpp" + + break; + } + case 130: { + { +#line 687 "Parser.rl" + {te = p+1;{ +#line 687 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_PERSIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5772 "Parser.cpp" + + break; + } + case 131: { + { +#line 688 "Parser.rl" + {te = p+1;{ +#line 688 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_SOUTH_ARABIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5783 "Parser.cpp" + + break; + } + case 132: { + { +#line 689 "Parser.rl" + {te = p+1;{ +#line 689 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OLD_TURKIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5794 "Parser.cpp" + + break; + } + case 133: { + { +#line 690 "Parser.rl" + {te = p+1;{ +#line 690 "Parser.rl" + currentCls->add(CLASS_SCRIPT_ORIYA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5805 "Parser.cpp" + + break; + } + case 134: { + { +#line 691 "Parser.rl" + {te = p+1;{ +#line 691 "Parser.rl" + currentCls->add(CLASS_SCRIPT_OSMANYA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5816 "Parser.cpp" + + break; + } + case 135: { + { +#line 692 "Parser.rl" + {te = p+1;{ +#line 692 "Parser.rl" + currentCls->add(CLASS_SCRIPT_PHAGS_PA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5827 "Parser.cpp" + + break; + } + case 136: { + { +#line 693 "Parser.rl" + {te = p+1;{ +#line 693 "Parser.rl" + currentCls->add(CLASS_SCRIPT_PHOENICIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5838 "Parser.cpp" + + break; + } + case 137: { + { +#line 694 "Parser.rl" + {te = p+1;{ +#line 694 "Parser.rl" + currentCls->add(CLASS_SCRIPT_REJANG, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5849 "Parser.cpp" + + break; + } + case 138: { + { +#line 695 "Parser.rl" + {te = p+1;{ +#line 695 "Parser.rl" + currentCls->add(CLASS_SCRIPT_RUNIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5860 "Parser.cpp" + + break; + } + case 139: { + { +#line 696 "Parser.rl" + {te = p+1;{ +#line 696 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SAMARITAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5871 "Parser.cpp" + + break; + } + case 140: { + { +#line 697 "Parser.rl" + {te = p+1;{ +#line 697 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SAURASHTRA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5882 "Parser.cpp" + + break; + } + case 141: { + { +#line 698 "Parser.rl" + {te = p+1;{ +#line 698 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SHAVIAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5893 "Parser.cpp" + + break; + } + case 142: { + { +#line 699 "Parser.rl" + {te = p+1;{ +#line 699 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SINHALA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5904 "Parser.cpp" + + break; + } + case 143: { + { +#line 700 "Parser.rl" + {te = p+1;{ +#line 700 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SUNDANESE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5915 "Parser.cpp" + + break; + } + case 144: { + { +#line 701 "Parser.rl" + {te = p+1;{ +#line 701 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SYLOTI_NAGRI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5926 "Parser.cpp" + + break; + } + case 145: { + { +#line 702 "Parser.rl" + {te = p+1;{ +#line 702 "Parser.rl" + currentCls->add(CLASS_SCRIPT_SYRIAC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5937 "Parser.cpp" + + break; + } + case 146: { + { +#line 703 "Parser.rl" + {te = p+1;{ +#line 703 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TAGALOG, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5948 "Parser.cpp" + + break; + } + case 147: { + { +#line 704 "Parser.rl" + {te = p+1;{ +#line 704 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TAGBANWA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5959 "Parser.cpp" + + break; + } + case 148: { + { +#line 705 "Parser.rl" + {te = p+1;{ +#line 705 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TAI_LE, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5970 "Parser.cpp" + + break; + } + case 149: { + { +#line 706 "Parser.rl" + {te = p+1;{ +#line 706 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TAI_THAM, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5981 "Parser.cpp" + + break; + } + case 150: { + { +#line 707 "Parser.rl" + {te = p+1;{ +#line 707 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TAI_VIET, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 5992 "Parser.cpp" + + break; + } + case 151: { + { +#line 708 "Parser.rl" + {te = p+1;{ +#line 708 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TAMIL, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6003 "Parser.cpp" + + break; + } + case 152: { + { +#line 709 "Parser.rl" + {te = p+1;{ +#line 709 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TELUGU, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6014 "Parser.cpp" + + break; + } + case 153: { + { +#line 710 "Parser.rl" + {te = p+1;{ +#line 710 "Parser.rl" + currentCls->add(CLASS_SCRIPT_THAANA, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6025 "Parser.cpp" + + break; + } + case 154: { + { +#line 711 "Parser.rl" + {te = p+1;{ +#line 711 "Parser.rl" + currentCls->add(CLASS_SCRIPT_THAI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6036 "Parser.cpp" + + break; + } + case 155: { + { +#line 712 "Parser.rl" + {te = p+1;{ +#line 712 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TIBETAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6047 "Parser.cpp" + + break; + } + case 156: { + { +#line 713 "Parser.rl" + {te = p+1;{ +#line 713 "Parser.rl" + currentCls->add(CLASS_SCRIPT_TIFINAGH, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6058 "Parser.cpp" + + break; + } + case 157: { + { +#line 714 "Parser.rl" + {te = p+1;{ +#line 714 "Parser.rl" + currentCls->add(CLASS_SCRIPT_UGARITIC, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6069 "Parser.cpp" + + break; + } + case 158: { + { +#line 715 "Parser.rl" + {te = p+1;{ +#line 715 "Parser.rl" + currentCls->add(CLASS_SCRIPT_VAI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6080 "Parser.cpp" + + break; + } + case 159: { + { +#line 716 "Parser.rl" + {te = p+1;{ +#line 716 "Parser.rl" + currentCls->add(CLASS_SCRIPT_YI, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6091 "Parser.cpp" + + break; + } + case 160: { + { +#line 717 "Parser.rl" + {te = p+1;{ +#line 717 "Parser.rl" + currentCls->add(CLASS_UCP_ANY, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6102 "Parser.cpp" + + break; + } + case 161: { + { +#line 718 "Parser.rl" + {te = p+1;{ +#line 718 "Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6113 "Parser.cpp" + + break; + } + case 162: { + { +#line 580 "Parser.rl" + {te = p;p = p - 1;{ +#line 580 "Parser.rl" + currentCls->add(CLASS_UCP_C, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6124 "Parser.cpp" + + break; + } + case 163: { + { +#line 584 "Parser.rl" + {te = p;p = p - 1;{ +#line 584 "Parser.rl" + currentCls->add(CLASS_UCP_CO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6135 "Parser.cpp" + + break; + } + case 164: { + { +#line 586 "Parser.rl" + {te = p;p = p - 1;{ +#line 586 "Parser.rl" + currentCls->add(CLASS_UCP_L, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6146 "Parser.cpp" + + break; + } + case 165: { + { +#line 593 "Parser.rl" + {te = p;p = p - 1;{ +#line 593 "Parser.rl" + currentCls->add(CLASS_UCP_M, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6157 "Parser.cpp" + + break; + } + case 166: { + { +#line 595 "Parser.rl" + {te = p;p = p - 1;{ +#line 595 "Parser.rl" + currentCls->add(CLASS_UCP_ME, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6168 "Parser.cpp" + + break; + } + case 167: { + { +#line 597 "Parser.rl" + {te = p;p = p - 1;{ +#line 597 "Parser.rl" + currentCls->add(CLASS_UCP_N, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6179 "Parser.cpp" + + break; + } + case 168: { + { +#line 601 "Parser.rl" + {te = p;p = p - 1;{ +#line 601 "Parser.rl" + currentCls->add(CLASS_UCP_P, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6190 "Parser.cpp" + + break; + } + case 169: { + { +#line 609 "Parser.rl" + {te = p;p = p - 1;{ +#line 609 "Parser.rl" + currentCls->add(CLASS_UCP_S, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6201 "Parser.cpp" + + break; + } + case 170: { + { +#line 614 "Parser.rl" + {te = p;p = p - 1;{ +#line 614 "Parser.rl" + currentCls->add(CLASS_UCP_Z, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6212 "Parser.cpp" + + break; + } + case 171: { + { +#line 653 "Parser.rl" + {te = p;p = p - 1;{ +#line 653 "Parser.rl" + currentCls->add(CLASS_SCRIPT_HAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6223 "Parser.cpp" + + break; + } + case 172: { + { +#line 718 "Parser.rl" + {te = p;p = p - 1;{ +#line 718 "Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6234 "Parser.cpp" + + break; + } + case 173: { + { +#line 580 "Parser.rl" + {p = ((te))-1; + { +#line 580 "Parser.rl" + currentCls->add(CLASS_UCP_C, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6246 "Parser.cpp" + + break; + } + case 174: { + { +#line 584 "Parser.rl" + {p = ((te))-1; + { +#line 584 "Parser.rl" + currentCls->add(CLASS_UCP_CO, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6258 "Parser.cpp" + + break; + } + case 175: { + { +#line 586 "Parser.rl" + {p = ((te))-1; + { +#line 586 "Parser.rl" + currentCls->add(CLASS_UCP_L, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6270 "Parser.cpp" + + break; + } + case 176: { + { +#line 593 "Parser.rl" + {p = ((te))-1; + { +#line 593 "Parser.rl" + currentCls->add(CLASS_UCP_M, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6282 "Parser.cpp" + + break; + } + case 177: { + { +#line 595 "Parser.rl" + {p = ((te))-1; + { +#line 595 "Parser.rl" + currentCls->add(CLASS_UCP_ME, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6294 "Parser.cpp" + + break; + } + case 178: { + { +#line 597 "Parser.rl" + {p = ((te))-1; + { +#line 597 "Parser.rl" + currentCls->add(CLASS_UCP_N, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6306 "Parser.cpp" + + break; + } + case 179: { + { +#line 601 "Parser.rl" + {p = ((te))-1; + { +#line 601 "Parser.rl" + currentCls->add(CLASS_UCP_P, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6318 "Parser.cpp" + + break; + } + case 180: { + { +#line 609 "Parser.rl" + {p = ((te))-1; + { +#line 609 "Parser.rl" + currentCls->add(CLASS_UCP_S, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6330 "Parser.cpp" + + break; + } + case 181: { + { +#line 653 "Parser.rl" + {p = ((te))-1; + { +#line 653 "Parser.rl" + currentCls->add(CLASS_SCRIPT_HAN, negated); {top -= 1;cs = stack[top];goto _again;} } + }} + +#line 6342 "Parser.cpp" + + break; + } + case 182: { + { +#line 718 "Parser.rl" + {p = ((te))-1; + { +#line 718 "Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6354 "Parser.cpp" + + break; + } + case 183: { + { +#line 733 "Parser.rl" + {te = p+1;{ +#line 733 "Parser.rl" + + currentCls->add(CLASS_UCP_C, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6372 "Parser.cpp" + + break; + } + case 184: { + { +#line 741 "Parser.rl" + {te = p+1;{ +#line 741 "Parser.rl" + + currentCls->add(CLASS_UCP_L, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6390 "Parser.cpp" + + break; + } + case 185: { + { +#line 749 "Parser.rl" + {te = p+1;{ +#line 749 "Parser.rl" + + currentCls->add(CLASS_UCP_M, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6408 "Parser.cpp" + + break; + } + case 186: { + { +#line 757 "Parser.rl" + {te = p+1;{ +#line 757 "Parser.rl" + + currentCls->add(CLASS_UCP_N, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6426 "Parser.cpp" + + break; + } + case 187: { + { +#line 765 "Parser.rl" + {te = p+1;{ +#line 765 "Parser.rl" + + currentCls->add(CLASS_UCP_P, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6444 "Parser.cpp" + + break; + } + case 188: { + { +#line 773 "Parser.rl" + {te = p+1;{ +#line 773 "Parser.rl" + + currentCls->add(CLASS_UCP_S, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6462 "Parser.cpp" + + break; + } + case 189: { + { +#line 781 "Parser.rl" + {te = p+1;{ +#line 781 "Parser.rl" + + currentCls->add(CLASS_UCP_Z, negated); + if (!inCharClass) { + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + } + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 6480 "Parser.cpp" + + break; + } + case 190: { + { +#line 790 "Parser.rl" + {te = p+1;{ +#line 790 "Parser.rl" + throw LocatedParseError("Unknown property"); } + }} + +#line 6491 "Parser.cpp" + + break; + } + case 191: { + { +#line 796 "Parser.rl" + {te = p+1;{ +#line 796 "Parser.rl" + + throw LocatedParseError("Unsupported POSIX collating " + "element"); + } + }} + +#line 6505 "Parser.cpp" + + break; + } + case 192: { + { +#line 803 "Parser.rl" + {te = p+1;{ +#line 803 "Parser.rl" + + currentCls->add(CLASS_ALNUM, false); + } + }} + +#line 6518 "Parser.cpp" + + break; + } + case 193: { + { +#line 806 "Parser.rl" + {te = p+1;{ +#line 806 "Parser.rl" + + currentCls->add(CLASS_ALNUM, true); + } + }} + +#line 6531 "Parser.cpp" + + break; + } + case 194: { + { +#line 809 "Parser.rl" + {te = p+1;{ +#line 809 "Parser.rl" + + currentCls->add(CLASS_ALPHA, false); + } + }} + +#line 6544 "Parser.cpp" + + break; + } + case 195: { + { +#line 812 "Parser.rl" + {te = p+1;{ +#line 812 "Parser.rl" + + currentCls->add(CLASS_ALPHA, true); + } + }} + +#line 6557 "Parser.cpp" + + break; + } + case 196: { + { +#line 815 "Parser.rl" + {te = p+1;{ +#line 815 "Parser.rl" + + currentCls->add(CLASS_ASCII, false); + } + }} + +#line 6570 "Parser.cpp" + + break; + } + case 197: { + { +#line 818 "Parser.rl" + {te = p+1;{ +#line 818 "Parser.rl" + + currentCls->add(CLASS_ASCII, true); + } + }} + +#line 6583 "Parser.cpp" + + break; + } + case 198: { + { +#line 821 "Parser.rl" + {te = p+1;{ +#line 821 "Parser.rl" + + currentCls->add(CLASS_BLANK, false); + } + }} + +#line 6596 "Parser.cpp" + + break; + } + case 199: { + { +#line 824 "Parser.rl" + {te = p+1;{ +#line 824 "Parser.rl" + + currentCls->add(CLASS_BLANK, true); + } + }} + +#line 6609 "Parser.cpp" + + break; + } + case 200: { + { +#line 827 "Parser.rl" + {te = p+1;{ +#line 827 "Parser.rl" + + currentCls->add(CLASS_CNTRL, false); + } + }} + +#line 6622 "Parser.cpp" + + break; + } + case 201: { + { +#line 830 "Parser.rl" + {te = p+1;{ +#line 830 "Parser.rl" + + currentCls->add(CLASS_CNTRL, true); + } + }} + +#line 6635 "Parser.cpp" + + break; + } + case 202: { + { +#line 833 "Parser.rl" + {te = p+1;{ +#line 833 "Parser.rl" + + currentCls->add(CLASS_DIGIT, false); + } + }} + +#line 6648 "Parser.cpp" + + break; + } + case 203: { + { +#line 836 "Parser.rl" + {te = p+1;{ +#line 836 "Parser.rl" + + currentCls->add(CLASS_DIGIT, true); + } + }} + +#line 6661 "Parser.cpp" + + break; + } + case 204: { + { +#line 839 "Parser.rl" + {te = p+1;{ +#line 839 "Parser.rl" + + currentCls->add(CLASS_GRAPH, false); + } + }} + +#line 6674 "Parser.cpp" + + break; + } + case 205: { + { +#line 842 "Parser.rl" + {te = p+1;{ +#line 842 "Parser.rl" + + currentCls->add(CLASS_GRAPH, true); + } + }} + +#line 6687 "Parser.cpp" + + break; + } + case 206: { + { +#line 845 "Parser.rl" + {te = p+1;{ +#line 845 "Parser.rl" + + currentCls->add(CLASS_LOWER, false); + } + }} + +#line 6700 "Parser.cpp" + + break; + } + case 207: { + { +#line 848 "Parser.rl" + {te = p+1;{ +#line 848 "Parser.rl" + + currentCls->add(CLASS_LOWER, true); + } + }} + +#line 6713 "Parser.cpp" + + break; + } + case 208: { + { +#line 851 "Parser.rl" + {te = p+1;{ +#line 851 "Parser.rl" + + currentCls->add(CLASS_PRINT, false); + } + }} + +#line 6726 "Parser.cpp" + + break; + } + case 209: { + { +#line 854 "Parser.rl" + {te = p+1;{ +#line 854 "Parser.rl" + + currentCls->add(CLASS_PRINT, true); + } + }} + +#line 6739 "Parser.cpp" + + break; + } + case 210: { + { +#line 857 "Parser.rl" + {te = p+1;{ +#line 857 "Parser.rl" + + currentCls->add(CLASS_PUNCT, false); + } + }} + +#line 6752 "Parser.cpp" + + break; + } + case 211: { + { +#line 860 "Parser.rl" + {te = p+1;{ +#line 860 "Parser.rl" + + currentCls->add(CLASS_PUNCT, true); + } + }} + +#line 6765 "Parser.cpp" + + break; + } + case 212: { + { +#line 864 "Parser.rl" + {te = p+1;{ +#line 864 "Parser.rl" + + currentCls->add(CLASS_SPACE, false); + } + }} + +#line 6778 "Parser.cpp" + + break; + } + case 213: { + { +#line 867 "Parser.rl" + {te = p+1;{ +#line 867 "Parser.rl" + + currentCls->add(CLASS_SPACE, true); + } + }} + +#line 6791 "Parser.cpp" + + break; + } + case 214: { + { +#line 870 "Parser.rl" + {te = p+1;{ +#line 870 "Parser.rl" + + currentCls->add(CLASS_UPPER, false); + } + }} + +#line 6804 "Parser.cpp" + + break; + } + case 215: { + { +#line 873 "Parser.rl" + {te = p+1;{ +#line 873 "Parser.rl" + + currentCls->add(CLASS_UPPER, true); + } + }} + +#line 6817 "Parser.cpp" + + break; + } + case 216: { + { +#line 876 "Parser.rl" + {te = p+1;{ +#line 876 "Parser.rl" + + currentCls->add(CLASS_WORD, false); + } + }} + +#line 6830 "Parser.cpp" + + break; + } + case 217: { + { +#line 879 "Parser.rl" + {te = p+1;{ +#line 879 "Parser.rl" + + currentCls->add(CLASS_WORD, true); + } + }} + +#line 6843 "Parser.cpp" + + break; + } + case 218: { + { +#line 882 "Parser.rl" + {te = p+1;{ +#line 882 "Parser.rl" + + currentCls->add(CLASS_XDIGIT, false); + } + }} + +#line 6856 "Parser.cpp" + + break; + } + case 219: { + { +#line 885 "Parser.rl" + {te = p+1;{ +#line 885 "Parser.rl" + + currentCls->add(CLASS_XDIGIT, true); + } + }} + +#line 6869 "Parser.cpp" + + break; + } + case 220: { + { +#line 890 "Parser.rl" + {te = p+1;{ +#line 890 "Parser.rl" + + throw LocatedParseError("Invalid POSIX named class"); + } + }} + +#line 6882 "Parser.cpp" + + break; + } + case 221: { + { +#line 893 "Parser.rl" + {te = p+1;{ +#line 893 "Parser.rl" + + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 843;goto _again;}} + }} + +#line 6902 "Parser.cpp" + + break; + } + case 222: { + { +#line 896 "Parser.rl" + {te = p+1;{ +#line 896 "Parser.rl" + /*noop*/} + }} + +#line 6913 "Parser.cpp" + + break; + } + case 223: { + { +#line 898 "Parser.rl" + {te = p+1;{ +#line 898 "Parser.rl" + + currentCls->add('\x08'); + } + }} + +#line 6926 "Parser.cpp" + + break; + } + case 224: { + { +#line 902 "Parser.rl" + {te = p+1;{ +#line 902 "Parser.rl" + + currentCls->add('\x09'); + } + }} + +#line 6939 "Parser.cpp" + + break; + } + case 225: { + { +#line 906 "Parser.rl" + {te = p+1;{ +#line 906 "Parser.rl" + + currentCls->add('\x0a'); + } + }} + +#line 6952 "Parser.cpp" + + break; + } + case 226: { + { +#line 910 "Parser.rl" + {te = p+1;{ +#line 910 "Parser.rl" + + currentCls->add('\x0d'); + } + }} + +#line 6965 "Parser.cpp" + + break; + } + case 227: { + { +#line 914 "Parser.rl" + {te = p+1;{ +#line 914 "Parser.rl" + + currentCls->add('\x0c'); + } + }} + +#line 6978 "Parser.cpp" + + break; + } + case 228: { + { +#line 918 "Parser.rl" + {te = p+1;{ +#line 918 "Parser.rl" + + currentCls->add('\x07'); + } + }} + +#line 6991 "Parser.cpp" + + break; + } + case 229: { + { +#line 922 "Parser.rl" + {te = p+1;{ +#line 922 "Parser.rl" + + currentCls->add('\x1b'); + } + }} + +#line 7004 "Parser.cpp" + + break; + } + case 230: { + { +#line 926 "Parser.rl" + {te = p+1;{ +#line 926 "Parser.rl" + + currentCls->add(CLASS_HORZ, false); + } + }} + +#line 7017 "Parser.cpp" + + break; + } + case 231: { + { +#line 930 "Parser.rl" + {te = p+1;{ +#line 930 "Parser.rl" + + currentCls->add(CLASS_HORZ, true); + } + }} + +#line 7030 "Parser.cpp" + + break; + } + case 232: { + { +#line 934 "Parser.rl" + {te = p+1;{ +#line 934 "Parser.rl" + + currentCls->add(CLASS_VERT, false); + } + }} + +#line 7043 "Parser.cpp" + + break; + } + case 233: { + { +#line 938 "Parser.rl" + {te = p+1;{ +#line 938 "Parser.rl" + + currentCls->add(CLASS_VERT, true); + } + }} + +#line 7056 "Parser.cpp" + + break; + } + case 234: { + { +#line 942 "Parser.rl" + {te = p+1;{ +#line 942 "Parser.rl" + + negated = false; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 7078 "Parser.cpp" + + break; + } + case 235: { + { +#line 948 "Parser.rl" + {te = p+1;{ +#line 948 "Parser.rl" + + negated = false; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 7100 "Parser.cpp" + + break; + } + case 236: { + { +#line 954 "Parser.rl" + {te = p+1;{ +#line 954 "Parser.rl" + + negated = true; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 7122 "Parser.cpp" + + break; + } + case 237: { + { +#line 960 "Parser.rl" + {te = p+1;{ +#line 960 "Parser.rl" + + negated = true; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 7144 "Parser.cpp" + + break; + } + case 238: { + { +#line 970 "Parser.rl" + {te = p+1;{ +#line 970 "Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7157 "Parser.cpp" + + break; + } + case 239: { + { +#line 973 "Parser.rl" + {te = p+1;{ +#line 973 "Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7170 "Parser.cpp" + + break; + } + case 240: { + { +#line 977 "Parser.rl" + {te = p+1;{ +#line 977 "Parser.rl" + + string oct(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(oct, nullptr, 8); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if ((!mode.utf8 && val > 255) || val > MAX_UNICODE) { + throw LocatedParseError("Value in \\o{...} sequence is too large"); + } + currentCls->add((unichar)val); + } + }} + +#line 7193 "Parser.cpp" + + break; + } + case 241: { + { +#line 997 "Parser.rl" + {te = p+1;{ +#line 997 "Parser.rl" + + currentCls->add(accumulator); + } + }} + +#line 7206 "Parser.cpp" + + break; + } + case 242: { + { +#line 1001 "Parser.rl" + {te = p+1;{ +#line 1001 "Parser.rl" + + // whatever we found here + currentCls->add(*(ts + 1)); + + } + }} + +#line 7221 "Parser.cpp" + + break; + } + case 243: { + { +#line 1007 "Parser.rl" + {te = p+1;{ +#line 1007 "Parser.rl" + + string hex(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(hex, nullptr, 16); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if (val > MAX_UNICODE) { + throw LocatedParseError("Value in \\x{...} sequence is too large"); + } + currentCls->add((unichar)val); + } + }} + +#line 7244 "Parser.cpp" + + break; + } + case 244: { + { +#line 1025 "Parser.rl" + {te = p+1;{ +#line 1025 "Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + currentCls->add(decodeCtrl(ts[2])); + } + } + }} + +#line 7263 "Parser.cpp" + + break; + } + case 245: { + { +#line 1035 "Parser.rl" + {te = p+1;{ +#line 1035 "Parser.rl" + + currentCls->add(CLASS_WORD, false); + } + }} + +#line 7276 "Parser.cpp" + + break; + } + case 246: { + { +#line 1039 "Parser.rl" + {te = p+1;{ +#line 1039 "Parser.rl" + + currentCls->add(CLASS_WORD, true); + } + }} + +#line 7289 "Parser.cpp" + + break; + } + case 247: { + { +#line 1043 "Parser.rl" + {te = p+1;{ +#line 1043 "Parser.rl" + + currentCls->add(CLASS_SPACE, false); + } + }} + +#line 7302 "Parser.cpp" + + break; + } + case 248: { + { +#line 1047 "Parser.rl" + {te = p+1;{ +#line 1047 "Parser.rl" + + currentCls->add(CLASS_SPACE, true); + } + }} + +#line 7315 "Parser.cpp" + + break; + } + case 249: { + { +#line 1051 "Parser.rl" + {te = p+1;{ +#line 1051 "Parser.rl" + + currentCls->add(CLASS_DIGIT, false); + } + }} + +#line 7328 "Parser.cpp" + + break; + } + case 250: { + { +#line 1055 "Parser.rl" + {te = p+1;{ +#line 1055 "Parser.rl" + + currentCls->add(CLASS_DIGIT, true); + } + }} + +#line 7341 "Parser.cpp" + + break; + } + case 251: { + { +#line 1058 "Parser.rl" + {te = p+1;{ +#line 1058 "Parser.rl" + + currentCls->addDash(); + } + }} + +#line 7354 "Parser.cpp" + + break; + } + case 252: { + { +#line 276 "Parser.rl" + {te = p+1;{ +#line 276 "Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported in a character class."; + throw ParseError(str.str()); + } + }} + +#line 7370 "Parser.cpp" + + break; + } + case 253: { + { +#line 276 "Parser.rl" + {te = p+1;{ +#line 276 "Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported in a character class."; + throw ParseError(str.str()); + } + }} + +#line 7386 "Parser.cpp" + + break; + } + case 254: { + { +#line 276 "Parser.rl" + {te = p+1;{ +#line 276 "Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported in a character class."; + throw ParseError(str.str()); + } + }} + +#line 7402 "Parser.cpp" + + break; + } + case 255: { + { +#line 1075 "Parser.rl" + {te = p+1;{ +#line 1075 "Parser.rl" + + // add the literal char + currentCls->add(*(ts + 1)); + } + }} + +#line 7416 "Parser.cpp" + + break; + } + case 256: { + { +#line 1081 "Parser.rl" + {te = p+1;{ +#line 1081 "Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint2c(ts)); + } + }} + +#line 7430 "Parser.cpp" + + break; + } + case 257: { + { +#line 1086 "Parser.rl" + {te = p+1;{ +#line 1086 "Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint3c(ts)); + } + }} + +#line 7444 "Parser.cpp" + + break; + } + case 258: { + { +#line 1091 "Parser.rl" + {te = p+1;{ +#line 1091 "Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint4c(ts)); + } + }} + +#line 7458 "Parser.cpp" + + break; + } + case 259: { + { +#line 1096 "Parser.rl" + {te = p+1;{ +#line 1096 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7472 "Parser.cpp" + + break; + } + case 260: { + { +#line 1102 "Parser.rl" + {te = p+1;{ +#line 1102 "Parser.rl" + + currentCls->add((u8)*ts); + } + }} + +#line 7485 "Parser.cpp" + + break; + } + case 261: { + { +#line 1106 "Parser.rl" + {te = p+1;{ +#line 1106 "Parser.rl" + + currentCls->finalize(); + currentSeq->addComponent(move(currentCls)); + inCharClass = false; + {cs = 746;goto _again;}} + }} + +#line 7500 "Parser.cpp" + + break; + } + case 262: { + { +#line 966 "Parser.rl" + {te = p;p = p - 1;{ +#line 966 "Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 7511 "Parser.cpp" + + break; + } + case 263: { + { +#line 967 "Parser.rl" + {te = p;p = p - 1;{ +#line 967 "Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 7522 "Parser.cpp" + + break; + } + case 264: { + { +#line 970 "Parser.rl" + {te = p;p = p - 1;{ +#line 970 "Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7535 "Parser.cpp" + + break; + } + case 265: { + { +#line 973 "Parser.rl" + {te = p;p = p - 1;{ +#line 973 "Parser.rl" + + currentCls->add(octAccumulator); + } + }} + +#line 7548 "Parser.cpp" + + break; + } + case 266: { + { +#line 992 "Parser.rl" + {te = p;p = p - 1;{ +#line 992 "Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 7561 "Parser.cpp" + + break; + } + case 267: { + { +#line 997 "Parser.rl" + {te = p;p = p - 1;{ +#line 997 "Parser.rl" + + currentCls->add(accumulator); + } + }} + +#line 7574 "Parser.cpp" + + break; + } + case 268: { + { +#line 1021 "Parser.rl" + {te = p;p = p - 1;{ +#line 1021 "Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 7587 "Parser.cpp" + + break; + } + case 269: { + { +#line 1025 "Parser.rl" + {te = p;p = p - 1;{ +#line 1025 "Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + currentCls->add(decodeCtrl(ts[2])); + } + } + }} + +#line 7606 "Parser.cpp" + + break; + } + case 270: { + { +#line 1096 "Parser.rl" + {te = p;p = p - 1;{ +#line 1096 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7620 "Parser.cpp" + + break; + } + case 271: { + { +#line 1102 "Parser.rl" + {te = p;p = p - 1;{ +#line 1102 "Parser.rl" + + currentCls->add((u8)*ts); + } + }} + +#line 7633 "Parser.cpp" + + break; + } + case 272: { + { +#line 992 "Parser.rl" + {p = ((te))-1; + { +#line 992 "Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 7647 "Parser.cpp" + + break; + } + case 273: { + { +#line 1021 "Parser.rl" + {p = ((te))-1; + { +#line 1021 "Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 7661 "Parser.cpp" + + break; + } + case 274: { + { +#line 1096 "Parser.rl" + {p = ((te))-1; + { +#line 1096 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7676 "Parser.cpp" + + break; + } + case 275: { + { +#line 1102 "Parser.rl" + {p = ((te))-1; + { +#line 1102 "Parser.rl" + + currentCls->add((u8)*ts); + } + }} + +#line 7690 "Parser.cpp" + + break; + } + case 276: { + { +#line 1120 "Parser.rl" + {te = p+1;{ +#line 1120 "Parser.rl" + + if (currentCls->isNegated()) { + // Already seen a caret; the second one is not a meta-character. + inCharClassEarly = false; + {p = p - 1; } {cs = 819;goto _again;}} else { + currentCls->negate(); + // Note: we cannot switch off inCharClassEarly here, as /[^]]/ + // needs to use the right square bracket path below. + } + } + }} + +#line 7710 "Parser.cpp" + + break; + } + case 277: { + { +#line 1133 "Parser.rl" + {te = p+1;{ +#line 1133 "Parser.rl" + + currentCls->add(']'); + inCharClassEarly = false; + } + }} + +#line 7724 "Parser.cpp" + + break; + } + case 278: { + { +#line 1138 "Parser.rl" + {te = p+1;{ +#line 1138 "Parser.rl" + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 843;goto _again;}} + }} + +#line 7743 "Parser.cpp" + + break; + } + case 279: { + { +#line 1139 "Parser.rl" + {te = p+1;{ +#line 1139 "Parser.rl" + /*noop*/} + }} + +#line 7754 "Parser.cpp" + + break; + } + case 280: { + { +#line 1142 "Parser.rl" + {te = p+1;{ +#line 1142 "Parser.rl" + + inCharClassEarly = false; + {p = p - 1; } + {cs = 819;goto _again;}} + }} + +#line 7768 "Parser.cpp" + + break; + } + case 281: { + { +#line 1142 "Parser.rl" + {te = p;p = p - 1;{ +#line 1142 "Parser.rl" + + inCharClassEarly = false; + {p = p - 1; } + {cs = 819;goto _again;}} + }} + +#line 7782 "Parser.cpp" + + break; + } + case 282: { + { +#line 1154 "Parser.rl" + {te = p+1;{ +#line 1154 "Parser.rl" + + {cs = 746;goto _again;}} + }} + +#line 7794 "Parser.cpp" + + break; + } + case 283: { + { +#line 1159 "Parser.rl" + {te = p+1;{ +#line 1159 "Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint2c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 7812 "Parser.cpp" + + break; + } + case 284: { + { +#line 1168 "Parser.rl" + {te = p+1;{ +#line 1168 "Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint3c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 7830 "Parser.cpp" + + break; + } + case 285: { + { +#line 1177 "Parser.rl" + {te = p+1;{ +#line 1177 "Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint4c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 7848 "Parser.cpp" + + break; + } + case 286: { + { +#line 1186 "Parser.rl" + {te = p+1;{ +#line 1186 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7862 "Parser.cpp" + + break; + } + case 287: { + { +#line 1192 "Parser.rl" + {te = p+1;{ +#line 1192 "Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 7875 "Parser.cpp" + + break; + } + case 288: { + { +#line 1186 "Parser.rl" + {te = p;p = p - 1;{ +#line 1186 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7889 "Parser.cpp" + + break; + } + case 289: { + { +#line 1192 "Parser.rl" + {te = p;p = p - 1;{ +#line 1192 "Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 7902 "Parser.cpp" + + break; + } + case 290: { + { +#line 1186 "Parser.rl" + {p = ((te))-1; + { +#line 1186 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7917 "Parser.cpp" + + break; + } + case 291: { + { +#line 1202 "Parser.rl" + {te = p+1;{ +#line 1202 "Parser.rl" + + {top -= 1;cs = stack[top];goto _again;} + } + }} + +#line 7930 "Parser.cpp" + + break; + } + case 292: { + { +#line 1207 "Parser.rl" + {te = p+1;{ +#line 1207 "Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint2c(ts)); + inCharClassEarly = false; + } + }} + +#line 7945 "Parser.cpp" + + break; + } + case 293: { + { +#line 1213 "Parser.rl" + {te = p+1;{ +#line 1213 "Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint3c(ts)); + inCharClassEarly = false; + } + }} + +#line 7960 "Parser.cpp" + + break; + } + case 294: { + { +#line 1219 "Parser.rl" + {te = p+1;{ +#line 1219 "Parser.rl" + + assert(mode.utf8); + currentCls->add(readUtf8CodePoint4c(ts)); + inCharClassEarly = false; + } + }} + +#line 7975 "Parser.cpp" + + break; + } + case 295: { + { +#line 1225 "Parser.rl" + {te = p+1;{ +#line 1225 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 7989 "Parser.cpp" + + break; + } + case 296: { + { +#line 1231 "Parser.rl" + {te = p+1;{ +#line 1231 "Parser.rl" + + currentCls->add(*ts); + inCharClassEarly = false; + } + }} + +#line 8003 "Parser.cpp" + + break; + } + case 297: { + { +#line 1225 "Parser.rl" + {te = p;p = p - 1;{ +#line 1225 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 8017 "Parser.cpp" + + break; + } + case 298: { + { +#line 1231 "Parser.rl" + {te = p;p = p - 1;{ +#line 1231 "Parser.rl" + + currentCls->add(*ts); + inCharClassEarly = false; + } + }} + +#line 8031 "Parser.cpp" + + break; + } + case 299: { + { +#line 1225 "Parser.rl" + {p = ((te))-1; + { +#line 1225 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 8046 "Parser.cpp" + + break; + } + case 300: { + { +#line 1243 "Parser.rl" + {te = p+1;{ +#line 1243 "Parser.rl" + inComment = false; {cs = 746;goto _again;}} + }} + +#line 8057 "Parser.cpp" + + break; + } + case 301: { + { +#line 1 "-" + {te = p+1;}} + +#line 8065 "Parser.cpp" + + break; + } + case 302: { + { +#line 1255 "Parser.rl" + {te = p+1;{ +#line 1255 "Parser.rl" + inComment = false; {cs = 746;goto _again;}} + }} + +#line 8076 "Parser.cpp" + + break; + } + case 303: { + { +#line 1 "-" + {te = p+1;}} + +#line 8084 "Parser.cpp" + + break; + } + case 304: { + { +#line 1491 "Parser.rl" + {act = 288;}} + +#line 8092 "Parser.cpp" + + break; + } + case 305: { + { +#line 1508 "Parser.rl" + {act = 290;}} + +#line 8100 "Parser.cpp" + + break; + } + case 306: { + { +#line 1737 "Parser.rl" + {act = 330;}} + +#line 8108 "Parser.cpp" + + break; + } + case 307: { + { +#line 362 "Parser.rl" + {te = p+1;{ +#line 362 "Parser.rl" + + if (sequences.empty()) { + throw LocatedParseError("Unmatched parentheses"); + } + currentSeq->finalize(); + POP_SEQUENCE; + } + }} + +#line 8125 "Parser.cpp" + + break; + } + case 308: { + { +#line 1274 "Parser.rl" + {te = p+1;{ +#line 1274 "Parser.rl" + + currentSeq->addAlternation(); + } + }} + +#line 8138 "Parser.cpp" + + break; + } + case 309: { + { +#line 1279 "Parser.rl" + {te = p+1;{ +#line 1279 "Parser.rl" + + throw LocatedParseError("POSIX named classes are only " + "supported inside a class"); + } + }} + +#line 8152 "Parser.cpp" + + break; + } + case 310: { + { +#line 1286 "Parser.rl" + {te = p+1;{ +#line 1286 "Parser.rl" + + throw LocatedParseError("Unsupported POSIX collating " + "element"); + } + }} + +#line 8166 "Parser.cpp" + + break; + } + case 311: { + { +#line 1293 "Parser.rl" + {te = p+1;{ +#line 1293 "Parser.rl" + + {cs = 838;goto _again;}} + }} + +#line 8178 "Parser.cpp" + + break; + } + case 312: { + { +#line 1297 "Parser.rl" + {te = p+1;{ +#line 1297 "Parser.rl" + /* noop */ } + }} + +#line 8189 "Parser.cpp" + + break; + } + case 313: { + { +#line 1299 "Parser.rl" + {te = p+1;{ +#line 1299 "Parser.rl" + + currentSeq->addComponent(generateComponent(CLASS_ANY, false, mode)); + } + }} + +#line 8202 "Parser.cpp" + + break; + } + case 314: { + { +#line 1303 "Parser.rl" + {te = p+1;{ +#line 1303 "Parser.rl" + + if (mode.utf8) { + throw LocatedParseError("\\C is unsupported in UTF8"); + } + currentSeq->addComponent(std::make_unique()); + } + }} + +#line 8218 "Parser.cpp" + + break; + } + case 315: { + { +#line 1317 "Parser.rl" + {te = p+1;{ +#line 1317 "Parser.rl" + + if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8234 "Parser.cpp" + + break; + } + case 316: { + { +#line 1324 "Parser.rl" + {te = p+1;{ +#line 1324 "Parser.rl" + + if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8250 "Parser.cpp" + + break; + } + case 317: { + { +#line 1338 "Parser.rl" + {te = p+1;{ +#line 1338 "Parser.rl" + + if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8266 "Parser.cpp" + + break; + } + case 318: { + { +#line 1345 "Parser.rl" + {te = p+1;{ +#line 1345 "Parser.rl" + + if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8282 "Parser.cpp" + + break; + } + case 319: { + { +#line 1359 "Parser.rl" + {te = p+1;{ +#line 1359 "Parser.rl" + + if (!currentSeq->addRepeat( + 0, 1, ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8298 "Parser.cpp" + + break; + } + case 320: { + { +#line 1366 "Parser.rl" + {te = p+1;{ +#line 1366 "Parser.rl" + + if (!currentSeq->addRepeat( + 0, 1, ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8314 "Parser.cpp" + + break; + } + case 321: { + { +#line 1383 "Parser.rl" + {te = p+1;{ +#line 1383 "Parser.rl" + + if (repeatN > repeatM || repeatM == 0) { + throwInvalidRepeat(); + } else if (!currentSeq->addRepeat( + repeatN, repeatM, + ComponentRepeat::REPEAT_NONGREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 8333 "Parser.cpp" + + break; + } + case 322: { + { +#line 1393 "Parser.rl" + {te = p+1;{ +#line 1393 "Parser.rl" + + if (repeatN > repeatM || repeatM == 0) { + throwInvalidRepeat(); + } else if (!currentSeq->addRepeat( + repeatN, repeatM, + ComponentRepeat::REPEAT_POSSESSIVE)) { + throwInvalidRepeat(); + } + } + }} + +#line 8352 "Parser.cpp" + + break; + } + case 323: { + { +#line 322 "Parser.rl" + {te = p+1;{ +#line 322 "Parser.rl" + + inComment = true; + {cs = 849;goto _again;}} + }} + +#line 8365 "Parser.cpp" + + break; + } + case 324: { + { +#line 1410 "Parser.rl" + {te = p+1;{ +#line 1410 "Parser.rl" + {p = p - 1; } {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 787;goto _again;}} + }} + +#line 8384 "Parser.cpp" + + break; + } + case 325: { + { +#line 1414 "Parser.rl" + {te = p+1;{ +#line 1414 "Parser.rl" + assert(0); {p += 1; goto _out; } } + }} + +#line 8395 "Parser.cpp" + + break; + } + case 326: { + { +#line 1421 "Parser.rl" + {te = p+1;{ +#line 1421 "Parser.rl" + + auto bound = mode.multiline ? ComponentBoundary::BEGIN_LINE + : ComponentBoundary::BEGIN_STRING; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8410 "Parser.cpp" + + break; + } + case 327: { + { +#line 1428 "Parser.rl" + {te = p+1;{ +#line 1428 "Parser.rl" + + auto bound = mode.multiline ? ComponentBoundary::END_LINE + : ComponentBoundary::END_STRING_OPTIONAL_LF; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8425 "Parser.cpp" + + break; + } + case 328: { + { +#line 1434 "Parser.rl" + {te = p+1;{ +#line 1434 "Parser.rl" + + auto bound = ComponentBoundary::BEGIN_STRING; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8439 "Parser.cpp" + + break; + } + case 329: { + { +#line 1439 "Parser.rl" + {te = p+1;{ +#line 1439 "Parser.rl" + + auto bound = ComponentBoundary::END_STRING_OPTIONAL_LF; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8453 "Parser.cpp" + + break; + } + case 330: { + { +#line 1444 "Parser.rl" + {te = p+1;{ +#line 1444 "Parser.rl" + + auto bound = ComponentBoundary::END_STRING; + currentSeq->addComponent(std::make_unique(bound)); + } + }} + +#line 8467 "Parser.cpp" + + break; + } + case 331: { + { +#line 1449 "Parser.rl" + {te = p+1;{ +#line 1449 "Parser.rl" + + currentSeq->addComponent( + std::make_unique(ts - ptr, false, mode)); + } + }} + +#line 8481 "Parser.cpp" + + break; + } + case 332: { + { +#line 1454 "Parser.rl" + {te = p+1;{ +#line 1454 "Parser.rl" + + currentSeq->addComponent( + std::make_unique(ts - ptr, true, mode)); + } + }} + +#line 8495 "Parser.cpp" + + break; + } + case 333: { + { +#line 1464 "Parser.rl" + {te = p+1;{ +#line 1464 "Parser.rl" + + addLiteral(currentSeq, '\x09', mode); + } + }} + +#line 8508 "Parser.cpp" + + break; + } + case 334: { + { +#line 1468 "Parser.rl" + {te = p+1;{ +#line 1468 "Parser.rl" + + addLiteral(currentSeq, '\x0a', mode); + } + }} + +#line 8521 "Parser.cpp" + + break; + } + case 335: { + { +#line 1472 "Parser.rl" + {te = p+1;{ +#line 1472 "Parser.rl" + + addLiteral(currentSeq, '\x0d', mode); + } + }} + +#line 8534 "Parser.cpp" + + break; + } + case 336: { + { +#line 1476 "Parser.rl" + {te = p+1;{ +#line 1476 "Parser.rl" + + addLiteral(currentSeq, '\x0c', mode); + } + }} + +#line 8547 "Parser.cpp" + + break; + } + case 337: { + { +#line 1480 "Parser.rl" + {te = p+1;{ +#line 1480 "Parser.rl" + + addLiteral(currentSeq, '\x07', mode); + } + }} + +#line 8560 "Parser.cpp" + + break; + } + case 338: { + { +#line 1484 "Parser.rl" + {te = p+1;{ +#line 1484 "Parser.rl" + + addLiteral(currentSeq, '\x1b', mode); + } + }} + +#line 8573 "Parser.cpp" + + break; + } + case 339: { + { +#line 1488 "Parser.rl" + {te = p+1;{ +#line 1488 "Parser.rl" + + addLiteral(currentSeq, octAccumulator, mode); + } + }} + +#line 8586 "Parser.cpp" + + break; + } + case 340: { + { +#line 479 "Parser.rl" + {te = p+1;{ +#line 479 "Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 8602 "Parser.cpp" + + break; + } + case 341: { + { +#line 486 "Parser.rl" + {te = p+1;{ +#line 486 "Parser.rl" + + // Accumulator is a negative offset. + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + if (accumulator >= groupIndex) { + throw LocatedParseError("Invalid reference"); + } + unsigned idx = groupIndex - accumulator; + currentSeq->addComponent(std::make_unique(idx)); + } + }} + +#line 8623 "Parser.cpp" + + break; + } + case 342: { + { +#line 479 "Parser.rl" + {te = p+1;{ +#line 479 "Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 8639 "Parser.cpp" + + break; + } + case 343: { + { +#line 486 "Parser.rl" + {te = p+1;{ +#line 486 "Parser.rl" + + // Accumulator is a negative offset. + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + if (accumulator >= groupIndex) { + throw LocatedParseError("Invalid reference"); + } + unsigned idx = groupIndex - accumulator; + currentSeq->addComponent(std::make_unique(idx)); + } + }} + +#line 8660 "Parser.cpp" + + break; + } + case 344: { + { +#line 498 "Parser.rl" + {te = p+1;{ +#line 498 "Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8673 "Parser.cpp" + + break; + } + case 345: { + { +#line 498 "Parser.rl" + {te = p+1;{ +#line 498 "Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8686 "Parser.cpp" + + break; + } + case 346: { + { +#line 498 "Parser.rl" + {te = p+1;{ +#line 498 "Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8699 "Parser.cpp" + + break; + } + case 347: { + { +#line 498 "Parser.rl" + {te = p+1;{ +#line 498 "Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8712 "Parser.cpp" + + break; + } + case 348: { + { +#line 498 "Parser.rl" + {te = p+1;{ +#line 498 "Parser.rl" + + currentSeq->addComponent(std::make_unique(label)); + } + }} + +#line 8725 "Parser.cpp" + + break; + } + case 349: { + { +#line 1549 "Parser.rl" + {te = p+1;{ +#line 1549 "Parser.rl" + + ostringstream str; + str << "Onigiruma subroutine call at index " << ts - ptr << + " not supported."; + throw ParseError(str.str()); + } + }} + +#line 8741 "Parser.cpp" + + break; + } + case 350: { + { +#line 1560 "Parser.rl" + {te = p+1;{ +#line 1560 "Parser.rl" + + string oct(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(oct, nullptr, 8); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if ((!mode.utf8 && val > 255) || val > MAX_UNICODE) { + throw LocatedParseError("Value in \\o{...} sequence is too large"); + } + addEscapedOctal(currentSeq, (unichar)val, mode); + } + }} + +#line 8764 "Parser.cpp" + + break; + } + case 351: { + { +#line 1578 "Parser.rl" + {te = p+1;{ +#line 1578 "Parser.rl" + + addEscapedHex(currentSeq, accumulator, mode); + } + }} + +#line 8777 "Parser.cpp" + + break; + } + case 352: { + { +#line 1582 "Parser.rl" + {te = p+1;{ +#line 1582 "Parser.rl" + + string hex(ts + 3, te - ts - 4); + unsigned long val; + try { + val = stoul(hex, nullptr, 16); + } catch (const std::out_of_range &) { + val = MAX_UNICODE + 1; + } + if (val > MAX_UNICODE) { + throw LocatedParseError("Value in \\x{...} sequence is too large"); + } + addEscapedHex(currentSeq, (unichar)val, mode); + } + }} + +#line 8800 "Parser.cpp" + + break; + } + case 353: { + { +#line 1600 "Parser.rl" + {te = p+1;{ +#line 1600 "Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + addLiteral(currentSeq, decodeCtrl(ts[2]), mode); + } + } + }} + +#line 8819 "Parser.cpp" + + break; + } + case 354: { + { +#line 1610 "Parser.rl" + {te = p+1;{ +#line 1610 "Parser.rl" + + ostringstream str; + str << "'\\" << *(ts + 1) << "' at index " << ts - ptr + << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 8835 "Parser.cpp" + + break; + } + case 355: { + { +#line 1618 "Parser.rl" + {te = p+1;{ +#line 1618 "Parser.rl" + + auto cc = generateComponent(CLASS_WORD, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8849 "Parser.cpp" + + break; + } + case 356: { + { +#line 1623 "Parser.rl" + {te = p+1;{ +#line 1623 "Parser.rl" + + auto cc = generateComponent(CLASS_WORD, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8863 "Parser.cpp" + + break; + } + case 357: { + { +#line 1628 "Parser.rl" + {te = p+1;{ +#line 1628 "Parser.rl" + + auto cc = generateComponent(CLASS_SPACE, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8877 "Parser.cpp" + + break; + } + case 358: { + { +#line 1633 "Parser.rl" + {te = p+1;{ +#line 1633 "Parser.rl" + + auto cc = generateComponent(CLASS_SPACE, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8891 "Parser.cpp" + + break; + } + case 359: { + { +#line 1638 "Parser.rl" + {te = p+1;{ +#line 1638 "Parser.rl" + + auto cc = generateComponent(CLASS_DIGIT, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8905 "Parser.cpp" + + break; + } + case 360: { + { +#line 1643 "Parser.rl" + {te = p+1;{ +#line 1643 "Parser.rl" + + auto cc = generateComponent(CLASS_DIGIT, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8919 "Parser.cpp" + + break; + } + case 361: { + { +#line 1648 "Parser.rl" + {te = p+1;{ +#line 1648 "Parser.rl" + + auto cc = generateComponent(CLASS_HORZ, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8933 "Parser.cpp" + + break; + } + case 362: { + { +#line 1653 "Parser.rl" + {te = p+1;{ +#line 1653 "Parser.rl" + + auto cc = generateComponent(CLASS_HORZ, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8947 "Parser.cpp" + + break; + } + case 363: { + { +#line 1658 "Parser.rl" + {te = p+1;{ +#line 1658 "Parser.rl" + + auto cc = generateComponent(CLASS_VERT, false, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8961 "Parser.cpp" + + break; + } + case 364: { + { +#line 1663 "Parser.rl" + {te = p+1;{ +#line 1663 "Parser.rl" + + auto cc = generateComponent(CLASS_VERT, true, mode); + currentSeq->addComponent(move(cc)); + } + }} + +#line 8975 "Parser.cpp" + + break; + } + case 365: { + { +#line 1668 "Parser.rl" + {te = p+1;{ +#line 1668 "Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = false; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 8999 "Parser.cpp" + + break; + } + case 366: { + { +#line 1676 "Parser.rl" + {te = p+1;{ +#line 1676 "Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = false; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 9023 "Parser.cpp" + + break; + } + case 367: { + { +#line 1684 "Parser.rl" + {te = p+1;{ +#line 1684 "Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = true; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 559;goto _again;}} + }} + +#line 9047 "Parser.cpp" + + break; + } + case 368: { + { +#line 1692 "Parser.rl" + {te = p+1;{ +#line 1692 "Parser.rl" + + assert(!currentCls && !inCharClass); + currentCls = getComponentClass(mode); + negated = true; + {p = p - 1; } + {{ +#line 1903 "Parser.rl" + + DEBUG_PRINTF("stack %zu top %d\n", stack.size(), top); + if ((int)stack.size() == top) { + stack.resize(2 * (top + 1)); + } + } + stack[top] = cs; top += 1;cs = 818;goto _again;}} + }} + +#line 9071 "Parser.cpp" + + break; + } + case 369: { + { +#line 1704 "Parser.rl" + {te = p+1;{ +#line 1704 "Parser.rl" + + ostringstream str; + str << "\\R at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9086 "Parser.cpp" + + break; + } + case 370: { + { +#line 1711 "Parser.rl" + {te = p+1;{ +#line 1711 "Parser.rl" + + ostringstream str; + str << "\\K at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9101 "Parser.cpp" + + break; + } + case 371: { + { +#line 1726 "Parser.rl" + {te = p+1;{ +#line 1726 "Parser.rl" + + ostringstream str; + str << "\\G at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9116 "Parser.cpp" + + break; + } + case 372: { + { +#line 1732 "Parser.rl" + {te = p+1;{ +#line 1732 "Parser.rl" + + currentSeq->addComponent(std::make_unique(ts - ptr, mode)); + } + }} + +#line 9129 "Parser.cpp" + + break; + } + case 373: { + { +#line 1737 "Parser.rl" + {te = p+1;{ +#line 1737 "Parser.rl" + + addLiteral(currentSeq, *(ts + 1), mode); + } + }} + +#line 9142 "Parser.cpp" + + break; + } + case 374: { + { +#line 316 "Parser.rl" + {te = p+1;{ +#line 316 "Parser.rl" + + inComment = true; + {cs = 848;goto _again;}} + }} + +#line 9155 "Parser.cpp" + + break; + } + case 375: { + { +#line 433 "Parser.rl" + {te = p+1;{ +#line 433 "Parser.rl" + + mode = newMode; + currentSeq->addComponent(std::make_unique()); + } + }} + +#line 9169 "Parser.cpp" + + break; + } + case 376: { + { +#line 355 "Parser.rl" + {te = p+1;{ +#line 355 "Parser.rl" + + PUSH_SEQUENCE; + mode = newMode; + currentSeq = + enterSequence(currentSeq, std::make_unique()); + } + }} + +#line 9185 "Parser.cpp" + + break; + } + case 377: { + { +#line 369 "Parser.rl" + {te = p+1;{ +#line 369 "Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKAHEAD, + ComponentAssertion::POS)); + } + }} + +#line 9201 "Parser.cpp" + + break; + } + case 378: { + { +#line 375 "Parser.rl" + {te = p+1;{ +#line 375 "Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKAHEAD, + ComponentAssertion::NEG)); + } + }} + +#line 9217 "Parser.cpp" + + break; + } + case 379: { + { +#line 381 "Parser.rl" + {te = p+1;{ +#line 381 "Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKBEHIND, + ComponentAssertion::POS)); + } + }} + +#line 9233 "Parser.cpp" + + break; + } + case 380: { + { +#line 387 "Parser.rl" + {te = p+1;{ +#line 387 "Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(ComponentAssertion::LOOKBEHIND, + ComponentAssertion::NEG)); + } + }} + +#line 9249 "Parser.cpp" + + break; + } + case 381: { + { +#line 393 "Parser.rl" + {te = p+1;{ +#line 393 "Parser.rl" + + throw LocatedParseError("Embedded code is not supported"); + } + }} + +#line 9262 "Parser.cpp" + + break; + } + case 382: { + { +#line 393 "Parser.rl" + {te = p+1;{ +#line 393 "Parser.rl" + + throw LocatedParseError("Embedded code is not supported"); + } + }} + +#line 9275 "Parser.cpp" + + break; + } + case 383: { + { +#line 416 "Parser.rl" + {te = p+1;{ +#line 416 "Parser.rl" + + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique()); + } + }} + +#line 9290 "Parser.cpp" + + break; + } + case 384: { + { +#line 336 "Parser.rl" + {te = p+1;{ +#line 336 "Parser.rl" + + assert(!label.empty()); // should be guaranteed by machine + char c = *label.begin(); + if (c >= '0' && c <= '9') { + throw LocatedParseError("Group name cannot begin with a digit"); + } + if (!groupNames.insert(label).second) { + throw LocatedParseError("Two named subpatterns use the name '" + label + "'"); + } + PUSH_SEQUENCE; + auto seq = std::make_unique(); + seq->setCaptureIndex(groupIndex++); + seq->setCaptureName(label); + currentSeq = enterSequence(currentSeq, move(seq)); + } + }} + +#line 9315 "Parser.cpp" + + break; + } + case 385: { + { +#line 399 "Parser.rl" + {te = p+1;{ +#line 399 "Parser.rl" + + throw LocatedParseError("Subpattern reference unsupported"); + } + }} + +#line 9328 "Parser.cpp" + + break; + } + case 386: { + { +#line 399 "Parser.rl" + {te = p+1;{ +#line 399 "Parser.rl" + + throw LocatedParseError("Subpattern reference unsupported"); + } + }} + +#line 9341 "Parser.cpp" + + break; + } + case 387: { + { +#line 1783 "Parser.rl" + {te = p+1;{ +#line 1783 "Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKAHEAD, ComponentAssertion::POS); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9361 "Parser.cpp" + + break; + } + case 388: { + { +#line 1794 "Parser.rl" + {te = p+1;{ +#line 1794 "Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKAHEAD, ComponentAssertion::NEG); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9381 "Parser.cpp" + + break; + } + case 389: { + { +#line 1805 "Parser.rl" + {te = p+1;{ +#line 1805 "Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKBEHIND, ComponentAssertion::POS); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9401 "Parser.cpp" + + break; + } + case 390: { + { +#line 1816 "Parser.rl" + {te = p+1;{ +#line 1816 "Parser.rl" + + auto a = std::make_unique( + ComponentAssertion::LOOKBEHIND, ComponentAssertion::NEG); + ComponentAssertion *a_seq = a.get(); + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(move(a))); + PUSH_SEQUENCE; + currentSeq = a_seq; + } + }} + +#line 9421 "Parser.cpp" + + break; + } + case 391: { + { +#line 1828 "Parser.rl" + {te = p+1;{ +#line 1828 "Parser.rl" + + throw LocatedParseError("Pattern recursion not supported"); + } + }} + +#line 9434 "Parser.cpp" + + break; + } + case 392: { + { +#line 402 "Parser.rl" + {te = p+1;{ +#line 402 "Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + PUSH_SEQUENCE; + currentSeq = enterSequence(currentSeq, + std::make_unique(accumulator)); + } + }} + +#line 9452 "Parser.cpp" + + break; + } + case 393: { + { +#line 410 "Parser.rl" + {te = p+1;{ +#line 410 "Parser.rl" + + PUSH_SEQUENCE; + assert(!label.empty()); + currentSeq = enterSequence(currentSeq, + std::make_unique(label)); + } + }} + +#line 9468 "Parser.cpp" + + break; + } + case 394: { + { +#line 1844 "Parser.rl" + {te = p+1;{ +#line 1844 "Parser.rl" + + ostringstream str; + str << "Callout at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9483 "Parser.cpp" + + break; + } + case 395: { + { +#line 1852 "Parser.rl" + {te = p+1;{ +#line 1852 "Parser.rl" + + throw LocatedParseError("Unrecognised character after (?"); + } + }} + +#line 9496 "Parser.cpp" + + break; + } + case 396: { + { +#line 1857 "Parser.rl" + {te = p+1;{ +#line 1857 "Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint2c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 9514 "Parser.cpp" + + break; + } + case 397: { + { +#line 1866 "Parser.rl" + {te = p+1;{ +#line 1866 "Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint3c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 9532 "Parser.cpp" + + break; + } + case 398: { + { +#line 1875 "Parser.rl" + {te = p+1;{ +#line 1875 "Parser.rl" + + assert(mode.utf8); + /* leverage ComponentClass to generate the vertices */ + auto cc = getComponentClass(mode); + cc->add(readUtf8CodePoint4c(ts)); + cc->finalize(); + currentSeq->addComponent(move(cc)); + } + }} + +#line 9550 "Parser.cpp" + + break; + } + case 399: { + { +#line 1884 "Parser.rl" + {te = p+1;{ +#line 1884 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 9564 "Parser.cpp" + + break; + } + case 400: { + { +#line 1893 "Parser.rl" + {te = p+1;{ +#line 1893 "Parser.rl" + + if (mode.ignore_space == false) { + addLiteral(currentSeq, *ts, mode); + } + } + }} + +#line 9579 "Parser.cpp" + + break; + } + case 401: { + { +#line 1898 "Parser.rl" + {te = p+1;{ +#line 1898 "Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 9592 "Parser.cpp" + + break; + } + case 402: { + { +#line 328 "Parser.rl" + {te = p;p = p - 1;{ +#line 328 "Parser.rl" + + PUSH_SEQUENCE; + auto seq = std::make_unique(); + seq->setCaptureIndex(groupIndex++); + currentSeq = enterSequence(currentSeq, move(seq)); + } + }} + +#line 9608 "Parser.cpp" + + break; + } + case 403: { + { +#line 421 "Parser.rl" + {te = p;p = p - 1;{ +#line 421 "Parser.rl" + + assert(!currentCls); + assert(!inCharClass); // not reentrant + currentCls = getComponentClass(mode); + inCharClass = true; + inCharClassEarly = true; + currentClsBegin = ts; + {cs = 836;goto _again;}} + }} + +#line 9626 "Parser.cpp" + + break; + } + case 404: { + { +#line 1310 "Parser.rl" + {te = p;p = p - 1;{ +#line 1310 "Parser.rl" + + if (!currentSeq->addRepeat(0, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9642 "Parser.cpp" + + break; + } + case 405: { + { +#line 1331 "Parser.rl" + {te = p;p = p - 1;{ +#line 1331 "Parser.rl" + + if (!currentSeq->addRepeat(1, ComponentRepeat::NoLimit, + ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9658 "Parser.cpp" + + break; + } + case 406: { + { +#line 1352 "Parser.rl" + {te = p;p = p - 1;{ +#line 1352 "Parser.rl" + + if (!currentSeq->addRepeat( + 0, 1, ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9674 "Parser.cpp" + + break; + } + case 407: { + { +#line 1373 "Parser.rl" + {te = p;p = p - 1;{ +#line 1373 "Parser.rl" + + if (repeatN > repeatM || repeatM == 0) { + throwInvalidRepeat(); + } else if (!currentSeq->addRepeat( + repeatN, repeatM, + ComponentRepeat::REPEAT_GREEDY)) { + throwInvalidRepeat(); + } + } + }} + +#line 9693 "Parser.cpp" + + break; + } + case 408: { + { +#line 1488 "Parser.rl" + {te = p;p = p - 1;{ +#line 1488 "Parser.rl" + + addLiteral(currentSeq, octAccumulator, mode); + } + }} + +#line 9706 "Parser.cpp" + + break; + } + case 409: { + { +#line 1491 "Parser.rl" + {te = p;p = p - 1;{ +#line 1491 "Parser.rl" + + // If there are enough capturing sub expressions, this may be + // a back reference + accumulator = parseAsDecimal(octAccumulator); + if (accumulator < groupIndex) { + currentSeq->addComponent(std::make_unique(accumulator)); + } else { + addEscapedOctal(currentSeq, octAccumulator, mode); + } + } + }} + +#line 9726 "Parser.cpp" + + break; + } + case 410: { + { +#line 479 "Parser.rl" + {te = p;p = p - 1;{ +#line 479 "Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 9742 "Parser.cpp" + + break; + } + case 411: { + { +#line 479 "Parser.rl" + {te = p;p = p - 1;{ +#line 479 "Parser.rl" + + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + currentSeq->addComponent(std::make_unique(accumulator)); + } + }} + +#line 9758 "Parser.cpp" + + break; + } + case 412: { + { +#line 486 "Parser.rl" + {te = p;p = p - 1;{ +#line 486 "Parser.rl" + + // Accumulator is a negative offset. + if (accumulator == 0) { + throw LocatedParseError("Numbered reference cannot be zero"); + } + if (accumulator >= groupIndex) { + throw LocatedParseError("Invalid reference"); + } + unsigned idx = groupIndex - accumulator; + currentSeq->addComponent(std::make_unique(idx)); + } + }} + +#line 9779 "Parser.cpp" + + break; + } + case 413: { + { +#line 1557 "Parser.rl" + {te = p;p = p - 1;{ +#line 1557 "Parser.rl" + + throw LocatedParseError("Invalid reference after \\g"); + } + }} + +#line 9792 "Parser.cpp" + + break; + } + case 414: { + { +#line 1574 "Parser.rl" + {te = p;p = p - 1;{ +#line 1574 "Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 9805 "Parser.cpp" + + break; + } + case 415: { + { +#line 1578 "Parser.rl" + {te = p;p = p - 1;{ +#line 1578 "Parser.rl" + + addEscapedHex(currentSeq, accumulator, mode); + } + }} + +#line 9818 "Parser.cpp" + + break; + } + case 416: { + { +#line 1596 "Parser.rl" + {te = p;p = p - 1;{ +#line 1596 "Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 9831 "Parser.cpp" + + break; + } + case 417: { + { +#line 1600 "Parser.rl" + {te = p;p = p - 1;{ +#line 1600 "Parser.rl" + + if (te - ts < 3) { + assert(te - ts == 2); + throw LocatedParseError(SLASH_C_ERROR); + } else { + assert(te - ts == 3); + addLiteral(currentSeq, decodeCtrl(ts[2]), mode); + } + } + }} + +#line 9850 "Parser.cpp" + + break; + } + case 418: { + { +#line 1700 "Parser.rl" + {te = p;p = p - 1;{ +#line 1700 "Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 9861 "Parser.cpp" + + break; + } + case 419: { + { +#line 1701 "Parser.rl" + {te = p;p = p - 1;{ +#line 1701 "Parser.rl" + throw LocatedParseError("Malformed property"); } + }} + +#line 9872 "Parser.cpp" + + break; + } + case 420: { + { +#line 1719 "Parser.rl" + {te = p;p = p - 1;{ +#line 1719 "Parser.rl" + + ostringstream str; + str << "\\k at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 9887 "Parser.cpp" + + break; + } + case 421: { + { +#line 1742 "Parser.rl" + {te = p;p = p - 1;{ +#line 1742 "Parser.rl" + + assert(ts + 1 == pe); + ostringstream str; + str << "Unescaped \\ at end of input, index " << ts - ptr << "."; + throw ParseError(str.str()); + } + }} + +#line 9903 "Parser.cpp" + + break; + } + case 422: { + { +#line 396 "Parser.rl" + {te = p;p = p - 1;{ +#line 396 "Parser.rl" + + throw LocatedParseError("Conditional subpattern unsupported"); + } + }} + +#line 9916 "Parser.cpp" + + break; + } + case 423: { + { +#line 1852 "Parser.rl" + {te = p;p = p - 1;{ +#line 1852 "Parser.rl" + + throw LocatedParseError("Unrecognised character after (?"); + } + }} + +#line 9929 "Parser.cpp" + + break; + } + case 424: { + { +#line 1884 "Parser.rl" + {te = p;p = p - 1;{ +#line 1884 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 9943 "Parser.cpp" + + break; + } + case 425: { + { +#line 1898 "Parser.rl" + {te = p;p = p - 1;{ +#line 1898 "Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 9956 "Parser.cpp" + + break; + } + case 426: { + { +#line 328 "Parser.rl" + {p = ((te))-1; + { +#line 328 "Parser.rl" + + PUSH_SEQUENCE; + auto seq = std::make_unique(); + seq->setCaptureIndex(groupIndex++); + currentSeq = enterSequence(currentSeq, move(seq)); + } + }} + +#line 9973 "Parser.cpp" + + break; + } + case 427: { + { +#line 421 "Parser.rl" + {p = ((te))-1; + { +#line 421 "Parser.rl" + + assert(!currentCls); + assert(!inCharClass); // not reentrant + currentCls = getComponentClass(mode); + inCharClass = true; + inCharClassEarly = true; + currentClsBegin = ts; + {cs = 836;goto _again;}} + }} + +#line 9992 "Parser.cpp" + + break; + } + case 428: { + { +#line 1557 "Parser.rl" + {p = ((te))-1; + { +#line 1557 "Parser.rl" + + throw LocatedParseError("Invalid reference after \\g"); + } + }} + +#line 10006 "Parser.cpp" + + break; + } + case 429: { + { +#line 1574 "Parser.rl" + {p = ((te))-1; + { +#line 1574 "Parser.rl" + + throw LocatedParseError("Value in \\o{...} sequence is non-octal or missing braces"); + } + }} + +#line 10020 "Parser.cpp" + + break; + } + case 430: { + { +#line 1596 "Parser.rl" + {p = ((te))-1; + { +#line 1596 "Parser.rl" + + throw LocatedParseError("Value in \\x{...} sequence is non-hex or missing }"); + } + }} + +#line 10034 "Parser.cpp" + + break; + } + case 431: { + { +#line 1719 "Parser.rl" + {p = ((te))-1; + { +#line 1719 "Parser.rl" + + ostringstream str; + str << "\\k at index " << ts - ptr << " not supported."; + throw ParseError(str.str()); + } + }} + +#line 10050 "Parser.cpp" + + break; + } + case 432: { + { +#line 396 "Parser.rl" + {p = ((te))-1; + { +#line 396 "Parser.rl" + + throw LocatedParseError("Conditional subpattern unsupported"); + } + }} + +#line 10064 "Parser.cpp" + + break; + } + case 433: { + { +#line 1852 "Parser.rl" + {p = ((te))-1; + { +#line 1852 "Parser.rl" + + throw LocatedParseError("Unrecognised character after (?"); + } + }} + +#line 10078 "Parser.cpp" + + break; + } + case 434: { + { +#line 1884 "Parser.rl" + {p = ((te))-1; + { +#line 1884 "Parser.rl" + + assert(mode.utf8); + throwInvalidUtf8(); + } + }} + +#line 10093 "Parser.cpp" + + break; + } + case 435: { + { +#line 1898 "Parser.rl" + {p = ((te))-1; + { +#line 1898 "Parser.rl" + + addLiteral(currentSeq, *ts, mode); + } + }} + +#line 10107 "Parser.cpp" + + break; + } + case 436: { + { +#line 1 "NONE" + {switch( act ) { + case 288: { + p = ((te))-1; + { +#line 1491 "Parser.rl" + + // If there are enough capturing sub expressions, this may be + // a back reference + accumulator = parseAsDecimal(octAccumulator); + if (accumulator < groupIndex) { + currentSeq->addComponent(std::make_unique(accumulator)); + } else { + addEscapedOctal(currentSeq, octAccumulator, mode); + } + } + break; + } + case 290: { + p = ((te))-1; + { +#line 1508 "Parser.rl" + + // if there are enough left parens to this point, back ref + if (accumulator < groupIndex) { + currentSeq->addComponent(std::make_unique(accumulator)); + } else { + // Otherwise, we interpret the first three digits as an + // octal escape, and the remaining characters stand for + // themselves as literals. + const char *s = ts; + unsigned int accum = 0; + unsigned int oct_digits = 0; + assert(*s == '\\'); // token starts at backslash + for (++s; s < te && oct_digits < 3; ++oct_digits, ++s) { + u8 digit = *s - '0'; + if (digit < 8) { + accum = digit + accum * 8; + } else { + break; + } + } + + if (oct_digits > 0) { + addEscapedOctal(currentSeq, accum, mode); + } + + // And then the rest of the digits, if any, are literal. + for (; s < te; ++s) { + addLiteral(currentSeq, *s, mode); + } + } + } + break; + } + case 330: { + p = ((te))-1; + { +#line 1737 "Parser.rl" + + addLiteral(currentSeq, *(ts + 1), mode); + } + break; + } + }} + } + +#line 10179 "Parser.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + +} + +_again: {} +if ( p == eof ) { + if ( cs >= 746 ) + goto _out; +} +else { + _acts = ( _regex_actions + (_regex_to_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 23: { + { +#line 1 "NONE" + {ts = 0;}} + +#line 10205 "Parser.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + if ( cs != 0 ) { + p += 1; + goto _resume; + } +} +_out: {} +} + +#line 1983 "Parser.rl" + + +if (p != pe && *p != '\0') { +// didn't make it to the end of our input, but we didn't throw a ParseError? +assert(0); +ostringstream str; +str << "Parse error at index " << (p - ptr) << "."; +throw ParseError(str.str()); +} + +if (currentCls) { +assert(inCharClass); +assert(currentClsBegin); +ostringstream oss; +oss << "Unterminated character class starting at index " +<< currentClsBegin - ptr << "."; +throw ParseError(oss.str()); +} + +if (inComment) { +throw ParseError("Unterminated comment."); +} + +if (!sequences.empty()) { +ostringstream str; +str << "Missing close parenthesis for group started at index " +<< sequences.back().seqOffset << "."; +throw ParseError(str.str()); +} + +// Unlikely, but possible +if (groupIndex > 65535) { +throw ParseError("The maximum number of capturing subexpressions is 65535."); +} + +// Finalize the top-level sequence, which will take care of any +// top-level alternation. +currentSeq->finalize(); +assert(currentSeq == rootSeq.get()); + +// Ensure that all references are valid. +checkReferences(*rootSeq, groupIndex, groupNames); + +return move(rootSeq); +} catch (LocatedParseError &error) { +if (ts >= ptr && ts <= pe) { +error.locate(ts - ptr); +} else { +error.locate(0); +} +throw; +} +} + +} // namespace ue2 diff --git a/contrib/vectorscan-cmake/rageled_files/amd64/control_verbs.cpp b/contrib/vectorscan-cmake/rageled_files/amd64/control_verbs.cpp new file mode 100644 index 00000000000..e0393e49355 --- /dev/null +++ b/contrib/vectorscan-cmake/rageled_files/amd64/control_verbs.cpp @@ -0,0 +1,547 @@ +#line 1 "control_verbs.rl" +/* +* Copyright (c) 2017, Intel Corporation +* +* Redistribution and use in source and binary forms, with or without +* modification, are permitted provided that the following conditions are met: +* +* * Redistributions of source code must retain the above copyright notice, +* this list of conditions and the following disclaimer. +* * Redistributions in binary form must reproduce the above copyright +* notice, this list of conditions and the following disclaimer in the +* documentation and/or other materials provided with the distribution. +* * Neither the name of Intel Corporation nor the names of its contributors +* may be used to endorse or promote products derived from this software +* without specific prior written permission. +* +* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +* POSSIBILITY OF SUCH DAMAGE. +*/ + +/** +* \file +* \brief Parser for control verbs that can occur at the beginning of a pattern. +*/ + +#include "parser/control_verbs.h" + +#include "parser/Parser.h" +#include "parser/parse_error.h" + +#include +#include + +using namespace std; + +namespace ue2 { + + const char *read_control_verbs(const char *ptr, const char *end, size_t start, + ParseMode &mode) { + const char *p = ptr; + const char *pe = end; + const char *eof = pe; + const char *ts, *te; + int cs; + UNUSED int act; + + +#line 56 "control_verbs.cpp" +static const signed char _ControlVerbs_actions[] = { + 0, 1, 0, 1, 1, 1, 2, 1, + 3, 1, 4, 1, 5, 1, 6, 1, + 7, 1, 8, 1, 9, 0 + }; + + static const short _ControlVerbs_key_offsets[] = { + 0, 7, 8, 10, 12, 14, 16, 18, + 20, 21, 23, 25, 27, 30, 32, 34, + 36, 38, 40, 42, 44, 46, 48, 50, + 52, 55, 57, 59, 61, 63, 66, 68, + 70, 72, 74, 76, 79, 82, 84, 86, + 88, 90, 92, 94, 96, 98, 100, 102, + 105, 107, 109, 111, 113, 115, 117, 119, + 121, 123, 125, 127, 129, 131, 133, 135, + 137, 139, 141, 143, 146, 148, 149, 151, + 155, 157, 159, 160, 161, 0 + }; + + static const char _ControlVerbs_trans_keys[] = { + 41, 65, 66, 67, 76, 78, 85, 41, + 41, 78, 41, 89, 41, 67, 41, 82, + 41, 76, 41, 70, 41, 41, 83, 41, + 82, 41, 95, 41, 65, 85, 41, 78, + 41, 89, 41, 67, 41, 78, 41, 73, + 41, 67, 41, 79, 41, 68, 41, 69, + 41, 82, 41, 76, 41, 70, 73, 41, + 77, 41, 73, 41, 84, 41, 95, 41, + 77, 82, 41, 65, 41, 84, 41, 67, + 41, 72, 41, 61, 41, 48, 57, 41, + 48, 57, 41, 69, 41, 67, 41, 85, + 41, 82, 41, 83, 41, 73, 41, 79, + 41, 78, 41, 79, 41, 95, 41, 65, + 83, 41, 85, 41, 84, 41, 79, 41, + 95, 41, 80, 41, 79, 41, 83, 41, + 83, 41, 69, 41, 83, 41, 83, 41, + 84, 41, 65, 41, 82, 41, 84, 41, + 95, 41, 79, 41, 80, 41, 84, 41, + 67, 84, 41, 80, 41, 41, 70, 41, + 49, 51, 56, 41, 54, 41, 50, 41, + 40, 42, 0 + }; + + static const signed char _ControlVerbs_single_lengths[] = { + 7, 1, 2, 2, 2, 2, 2, 2, + 1, 2, 2, 2, 3, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, + 3, 2, 2, 2, 2, 3, 2, 2, + 2, 2, 2, 1, 1, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 3, + 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 3, 2, 1, 2, 4, + 2, 2, 1, 1, 1, 0 + }; + + static const signed char _ControlVerbs_range_lengths[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 1, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0 + }; + + static const short _ControlVerbs_index_offsets[] = { + 0, 8, 10, 13, 16, 19, 22, 25, + 28, 30, 33, 36, 39, 43, 46, 49, + 52, 55, 58, 61, 64, 67, 70, 73, + 76, 80, 83, 86, 89, 92, 96, 99, + 102, 105, 108, 111, 114, 117, 120, 123, + 126, 129, 132, 135, 138, 141, 144, 147, + 151, 154, 157, 160, 163, 166, 169, 172, + 175, 178, 181, 184, 187, 190, 193, 196, + 199, 202, 205, 208, 212, 215, 217, 220, + 225, 228, 231, 233, 235, 0 + }; + + static const signed char _ControlVerbs_cond_targs[] = { + 75, 2, 9, 22, 24, 45, 67, 1, + 75, 1, 75, 3, 1, 75, 4, 1, + 75, 5, 1, 75, 6, 1, 75, 7, + 1, 75, 8, 1, 75, 1, 75, 10, + 1, 75, 11, 1, 75, 12, 1, 75, + 13, 16, 1, 75, 14, 1, 75, 15, + 1, 75, 5, 1, 75, 17, 1, 75, + 18, 1, 75, 19, 1, 75, 20, 1, + 75, 21, 1, 75, 8, 1, 75, 23, + 1, 75, 7, 1, 75, 8, 25, 1, + 75, 26, 1, 75, 27, 1, 75, 28, + 1, 75, 29, 1, 75, 30, 37, 1, + 75, 31, 1, 75, 32, 1, 75, 33, + 1, 75, 34, 1, 75, 35, 1, 75, + 36, 1, 75, 36, 1, 75, 38, 1, + 75, 39, 1, 75, 40, 1, 75, 41, + 1, 75, 42, 1, 75, 43, 1, 75, + 44, 1, 75, 34, 1, 75, 46, 1, + 75, 47, 1, 75, 48, 59, 1, 75, + 49, 1, 75, 50, 1, 75, 51, 1, + 75, 52, 1, 75, 53, 1, 75, 54, + 1, 75, 55, 1, 75, 56, 1, 75, + 57, 1, 75, 58, 1, 75, 8, 1, + 75, 60, 1, 75, 61, 1, 75, 62, + 1, 75, 63, 1, 75, 64, 1, 75, + 65, 1, 75, 66, 1, 75, 8, 1, + 75, 68, 70, 1, 75, 69, 1, 75, + 1, 75, 71, 1, 75, 72, 73, 74, + 1, 75, 8, 1, 75, 8, 1, 75, + 1, 76, 75, 0, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 75, 75, 75, 75, 75, 75, + 75, 75, 0 + }; + + static const signed char _ControlVerbs_cond_actions[] = { + 19, 0, 0, 0, 0, 0, 0, 0, + 13, 0, 13, 0, 0, 13, 0, 0, + 11, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 11, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 11, 0, 0, 13, 0, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 11, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 13, 0, 0, 13, 0, + 0, 13, 0, 0, 13, 0, 0, 13, + 0, 0, 13, 0, 0, 13, 0, 0, + 13, 0, 0, 0, 13, 0, 0, 9, + 0, 13, 0, 0, 7, 0, 0, 0, + 0, 13, 0, 0, 13, 0, 0, 7, + 0, 5, 15, 0, 17, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 19, 19, 19, 19, 19, 19, 19, 19, + 0, 17, 0 + }; + + static const signed char _ControlVerbs_to_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 0, 0 + }; + + static const signed char _ControlVerbs_from_state_actions[] = { + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 3, 0, 0 + }; + + static const short _ControlVerbs_eof_trans[] = { + 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 277, + 278, 279, 280, 281, 282, 283, 284, 285, + 286, 287, 288, 289, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, + 310, 311, 312, 313, 314, 0 + }; + + static const int ControlVerbs_start = 75; + static const int ControlVerbs_first_final = 75; + static const int ControlVerbs_error = -1; + + static const int ControlVerbs_en_main = 75; + + +#line 269 "control_verbs.cpp" + { + cs = (int)ControlVerbs_start; + ts = 0; + te = 0; + } + +#line 105 "control_verbs.rl" + + + try { + +#line 278 "control_verbs.cpp" + { + int _klen; + unsigned int _trans = 0; + const char * _keys; + const signed char * _acts; + unsigned int _nacts; + _resume: {} + if ( p == pe && p != eof ) + goto _out; + _acts = ( _ControlVerbs_actions + (_ControlVerbs_from_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 1: { + { +#line 1 "NONE" + {ts = p;}} + +#line 297 "control_verbs.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + if ( p == eof ) { + if ( _ControlVerbs_eof_trans[cs] > 0 ) { + _trans = (unsigned int)_ControlVerbs_eof_trans[cs] - 1; + } + } + else { + _keys = ( _ControlVerbs_trans_keys + (_ControlVerbs_key_offsets[cs])); + _trans = (unsigned int)_ControlVerbs_index_offsets[cs]; + + _klen = (int)_ControlVerbs_single_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + _klen - 1; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _keys += _klen; + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + ((_upper-_lower) >> 1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 1; + else if ( ( (*( p))) > (*( _mid)) ) + _lower = _mid + 1; + else { + _trans += (unsigned int)(_mid - _keys); + goto _match; + } + } + } + + _klen = (int)_ControlVerbs_range_lengths[cs]; + if ( _klen > 0 ) { + const char *_lower = _keys; + const char *_upper = _keys + (_klen<<1) - 2; + const char *_mid; + while ( 1 ) { + if ( _upper < _lower ) { + _trans += (unsigned int)_klen; + break; + } + + _mid = _lower + (((_upper-_lower) >> 1) & ~1); + if ( ( (*( p))) < (*( _mid)) ) + _upper = _mid - 2; + else if ( ( (*( p))) > (*( _mid + 1)) ) + _lower = _mid + 2; + else { + _trans += (unsigned int)((_mid - _keys)>>1); + break; + } + } + } + + _match: {} + } + cs = (int)_ControlVerbs_cond_targs[_trans]; + + if ( _ControlVerbs_cond_actions[_trans] != 0 ) { + + _acts = ( _ControlVerbs_actions + (_ControlVerbs_cond_actions[_trans])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) + { + case 2: { + { +#line 1 "NONE" + {te = p+1;}} + +#line 378 "control_verbs.cpp" + + break; + } + case 3: { + { +#line 76 "control_verbs.rl" + {te = p+1;{ +#line 76 "control_verbs.rl" + + mode.utf8 = true; + } + }} + +#line 391 "control_verbs.cpp" + + break; + } + case 4: { + { +#line 80 "control_verbs.rl" + {te = p+1;{ +#line 80 "control_verbs.rl" + + mode.ucp = true; + } + }} + +#line 404 "control_verbs.cpp" + + break; + } + case 5: { + { +#line 84 "control_verbs.rl" + {te = p+1;{ +#line 84 "control_verbs.rl" + + ostringstream str; + str << "Unsupported control verb " << string(ts, te - ts); + throw LocatedParseError(str.str()); + } + }} + +#line 419 "control_verbs.cpp" + + break; + } + case 6: { + { +#line 90 "control_verbs.rl" + {te = p+1;{ +#line 90 "control_verbs.rl" + + ostringstream str; + str << "Unknown control verb " << string(ts, te - ts); + throw LocatedParseError(str.str()); + } + }} + +#line 434 "control_verbs.cpp" + + break; + } + case 7: { + { +#line 97 "control_verbs.rl" + {te = p+1;{ +#line 97 "control_verbs.rl" + + {p = p - 1; } + {p += 1; goto _out; } + } + }} + +#line 448 "control_verbs.cpp" + + break; + } + case 8: { + { +#line 97 "control_verbs.rl" + {te = p;p = p - 1;{ +#line 97 "control_verbs.rl" + + {p = p - 1; } + {p += 1; goto _out; } + } + }} + +#line 462 "control_verbs.cpp" + + break; + } + case 9: { + { +#line 97 "control_verbs.rl" + {p = ((te))-1; + { +#line 97 "control_verbs.rl" + + {p = p - 1; } + {p += 1; goto _out; } + } + }} + +#line 477 "control_verbs.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + } + + if ( p == eof ) { + if ( cs >= 75 ) + goto _out; + } + else { + _acts = ( _ControlVerbs_actions + (_ControlVerbs_to_state_actions[cs])); + _nacts = (unsigned int)(*( _acts)); + _acts += 1; + while ( _nacts > 0 ) { + switch ( (*( _acts)) ) { + case 0: { + { +#line 1 "NONE" + {ts = 0;}} + +#line 502 "control_verbs.cpp" + + break; + } + } + _nacts -= 1; + _acts += 1; + } + + p += 1; + goto _resume; + } + _out: {} + } + +#line 108 "control_verbs.rl" + + } catch (LocatedParseError &error) { + if (ts >= ptr && ts <= pe) { + error.locate(ts - ptr + start); + } else { + error.locate(0); + } + throw; + } + + return p; + } + +} // namespace ue2 From 6f06633df6c47e94c7dc1dc09819db899aaf8cc2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 7 Sep 2022 13:59:39 +0200 Subject: [PATCH 443/582] Update storing-data.md --- docs/en/operations/storing-data.md | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 19d5a9a1651..f52b388d517 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -154,7 +154,7 @@ Example of configuration for versions earlier than 21.8: ``` -Cache configuration settings (the list corresponds to latest ClickHouse version, for earlier versions something might be unsupported): +Cache **configuration settings**: - `path` - path to cache Default: None, this settings is obligatory. @@ -172,7 +172,7 @@ Cache configuration settings (the list corresponds to latest ClickHouse version, - `max_elements` a limit for a number of cache files. -Cache user settings (can be changes per query): +Cache **query settings**: - `enable_filesystem_cache` - allows to disable cache even if storage policy was configured with `cache` disk type. Default: `true`. @@ -186,15 +186,17 @@ Cache user settings (can be changes per query): - `skip_download_if_exceeds_query_cache` - allows to change the behaviour of setting `max_query_cache_size`. Default: `true`. If this setting is turned on and cache download limit during query was reached, no more cache will be downloaded to cache storage. If this setting is turned off and cache download limit during query was reached, cache will still be written by evicting previously written within current query cache data. E.g. second behaviour allows to preserve `last recentltly used` behaviour. -Cache system tables: +* Cache configuration settings and cache query settings correspond to the latest ClickHouse version, for earlier versions something might not be supported. + +Cache **system tables**: - `system.filesystem_cache` - system tables which shows current state of cache. - `system.filesystem_cache_log` - system table which shows detailed cache usage per query. Requires `enable_filesystem_cache_log` setting to be `true`. -Cache commands: +Cache **commands**: -- `SYSTEM DROP FILESYSTEM CACHE (ON CLUSTER)` +- `SYSTEM DROP FILESYSTEM CACHE () (ON CLUSTER)` - `SHOW CACHES` -- show list of caches which were configured on the server. From fe8f84dccc195d521b5339a601c25802f0196184 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 15:57:29 +0000 Subject: [PATCH 444/582] Exclude slow tests from fasttest profile 99% of all tests in the fasttest profile run in one sec or less. The excluded tests take 10 sec or more (the slowest being 02271_replace_partition_many_tables with 30 sec). Estimated savings: 7 min --- tests/queries/0_stateless/00705_drop_create_merge_tree.sh | 2 ++ .../0_stateless/00953_zookeeper_suetin_deduplication_bug.sh | 2 +- tests/queries/0_stateless/00956_sensitive_data_masking.sh | 1 + tests/queries/0_stateless/01014_lazy_database_basic.sh | 2 +- .../queries/0_stateless/01019_alter_materialized_view_atomic.sh | 1 + tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 2 +- tests/queries/0_stateless/01150_ddl_guard_rwr.sh | 2 +- tests/queries/0_stateless/01192_rename_database_zookeeper.sh | 2 +- tests/queries/0_stateless/01249_flush_interactive.sh | 1 + tests/queries/0_stateless/01280_ttl_where_group_by.sh | 2 +- tests/queries/0_stateless/01361_fover_remote_num_tries.sh | 1 + tests/queries/0_stateless/01632_tinylog_read_write.sh | 1 + tests/queries/0_stateless/01654_test_writer_block_sequence.sh | 1 + .../queries/0_stateless/01903_csvwithnames_subset_of_columns.sh | 1 + tests/queries/0_stateless/02015_async_inserts_stress_long.sh | 2 +- tests/queries/0_stateless/02044_url_glob_parallel.sh | 2 +- tests/queries/0_stateless/02067_lost_part_s3.sql | 2 +- tests/queries/0_stateless/02104_overcommit_memory.sh | 2 +- .../queries/0_stateless/02122_4letter_words_stress_zookeeper.sh | 1 + tests/queries/0_stateless/02124_buffer_with_type_map_long.sh | 1 + tests/queries/0_stateless/02125_lz4_compression_bug.sh | 2 +- tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql | 2 ++ tests/queries/0_stateless/02246_async_insert_quota.sh | 2 +- .../queries/0_stateless/02271_replace_partition_many_tables.sql | 2 ++ tests/queries/0_stateless/02294_overcommit_overflow.sh | 2 +- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 1 + tests/queries/0_stateless/02352_rwlock.sh | 2 +- .../0_stateless/02401_merge_tree_old_tmp_dirs_cleanup.sql | 2 ++ 28 files changed, 32 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh index 2ab35dacd0f..146d6e54c0b 100755 --- a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh +++ b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-fasttest + set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index 0705c955d60..c713c7c4926 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database +# Tags: zookeeper, no-replicated-database, no-fasttest # Tag no-replicated-database: Requires investigation set -e diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index e7179e1e002..e36031c54be 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest # Get all server logs export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="trace" diff --git a/tests/queries/0_stateless/01014_lazy_database_basic.sh b/tests/queries/0_stateless/01014_lazy_database_basic.sh index 496e9754089..ea7603b2519 100755 --- a/tests/queries/0_stateless/01014_lazy_database_basic.sh +++ b/tests/queries/0_stateless/01014_lazy_database_basic.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh index 54a7e940377..e50b974a48e 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest set -e diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index 60650cb9cc3..e4dad56bc29 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01150_ddl_guard_rwr.sh b/tests/queries/0_stateless/01150_ddl_guard_rwr.sh index 50e6f91b49b..8f77c470f09 100755 --- a/tests/queries/0_stateless/01150_ddl_guard_rwr.sh +++ b/tests/queries/0_stateless/01150_ddl_guard_rwr.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index 641e26870d4..dec1276111a 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel +# Tags: zookeeper, no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01249_flush_interactive.sh b/tests/queries/0_stateless/01249_flush_interactive.sh index 89167002ed5..8eb06cf4f06 100755 --- a/tests/queries/0_stateless/01249_flush_interactive.sh +++ b/tests/queries/0_stateless/01249_flush_interactive.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sh b/tests/queries/0_stateless/01280_ttl_where_group_by.sh index aa6b33d5935..ebef8f2a797 100755 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sh +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01361_fover_remote_num_tries.sh b/tests/queries/0_stateless/01361_fover_remote_num_tries.sh index 5e49b393c7a..12a65a96c9b 100755 --- a/tests/queries/0_stateless/01361_fover_remote_num_tries.sh +++ b/tests/queries/0_stateless/01361_fover_remote_num_tries.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01632_tinylog_read_write.sh b/tests/queries/0_stateless/01632_tinylog_read_write.sh index 3f41bcc5924..e45fdd91ff6 100755 --- a/tests/queries/0_stateless/01632_tinylog_read_write.sh +++ b/tests/queries/0_stateless/01632_tinylog_read_write.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest set -e diff --git a/tests/queries/0_stateless/01654_test_writer_block_sequence.sh b/tests/queries/0_stateless/01654_test_writer_block_sequence.sh index 3330148d91a..ab6b6bb7f38 100755 --- a/tests/queries/0_stateless/01654_test_writer_block_sequence.sh +++ b/tests/queries/0_stateless/01654_test_writer_block_sequence.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01903_csvwithnames_subset_of_columns.sh b/tests/queries/0_stateless/01903_csvwithnames_subset_of_columns.sh index 17d2c4a47d5..8de05a013bb 100755 --- a/tests/queries/0_stateless/01903_csvwithnames_subset_of_columns.sh +++ b/tests/queries/0_stateless/01903_csvwithnames_subset_of_columns.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index 086419baa61..437df01d445 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-settings +# Tags: no-random-settings, no-fasttest set -e diff --git a/tests/queries/0_stateless/02044_url_glob_parallel.sh b/tests/queries/0_stateless/02044_url_glob_parallel.sh index c9c779a9ddb..036ab990c51 100755 --- a/tests/queries/0_stateless/02044_url_glob_parallel.sh +++ b/tests/queries/0_stateless/02044_url_glob_parallel.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: distributed +# Tags: distributed, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02067_lost_part_s3.sql b/tests/queries/0_stateless/02067_lost_part_s3.sql index 463f80348b2..85ce2acddda 100644 --- a/tests/queries/0_stateless/02067_lost_part_s3.sql +++ b/tests/queries/0_stateless/02067_lost_part_s3.sql @@ -1,4 +1,4 @@ --- Tags: no-backward-compatibility-check +-- Tags: no-backward-compatibility-check, no-fasttest DROP TABLE IF EXISTS partslost_0; DROP TABLE IF EXISTS partslost_1; diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index 73d65eb40ed..384e30b1283 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh index 2deaf788ecf..180c7145a7a 100755 --- a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh +++ b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh b/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh index 1b2197ef943..8a057038d21 100755 --- a/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh +++ b/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02125_lz4_compression_bug.sh b/tests/queries/0_stateless/02125_lz4_compression_bug.sh index 4fddb46658e..77646b01298 100755 --- a/tests/queries/0_stateless/02125_lz4_compression_bug.sh +++ b/tests/queries/0_stateless/02125_lz4_compression_bug.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql b/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql index 7e68beb4b6f..86db9d04aa0 100644 --- a/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql +++ b/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SET log_queries = 1; DROP TABLE IF EXISTS t_async_insert_02193_1; diff --git a/tests/queries/0_stateless/02246_async_insert_quota.sh b/tests/queries/0_stateless/02246_async_insert_quota.sh index d1080313ed9..fa7749d35bb 100755 --- a/tests/queries/0_stateless/02246_async_insert_quota.sh +++ b/tests/queries/0_stateless/02246_async_insert_quota.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02271_replace_partition_many_tables.sql b/tests/queries/0_stateless/02271_replace_partition_many_tables.sql index a31a2bee58a..062b8335861 100644 --- a/tests/queries/0_stateless/02271_replace_partition_many_tables.sql +++ b/tests/queries/0_stateless/02271_replace_partition_many_tables.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + DROP TABLE IF EXISTS replace_partition_source; DROP TABLE IF EXISTS replace_partition_dest1; DROP TABLE IF EXISTS replace_partition_dest1_2; diff --git a/tests/queries/0_stateless/02294_overcommit_overflow.sh b/tests/queries/0_stateless/02294_overcommit_overflow.sh index 0fe7882cb3d..81b165cd6d8 100755 --- a/tests/queries/0_stateless/02294_overcommit_overflow.sh +++ b/tests/queries/0_stateless/02294_overcommit_overflow.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 4f3baa1f660..5079977fea0 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02352_rwlock.sh b/tests/queries/0_stateless/02352_rwlock.sh index 56b7ab27410..7a0b9ef8911 100755 --- a/tests/queries/0_stateless/02352_rwlock.sh +++ b/tests/queries/0_stateless/02352_rwlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # Tag no-parallel -- to avoid running it in parallel, this will avoid possible issues due to high pressure # Test that ensures that WRITE lock failure notifies READ. diff --git a/tests/queries/0_stateless/02401_merge_tree_old_tmp_dirs_cleanup.sql b/tests/queries/0_stateless/02401_merge_tree_old_tmp_dirs_cleanup.sql index 54579bca30b..01e054504dc 100644 --- a/tests/queries/0_stateless/02401_merge_tree_old_tmp_dirs_cleanup.sql +++ b/tests/queries/0_stateless/02401_merge_tree_old_tmp_dirs_cleanup.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + DROP TABLE IF EXISTS test_inserts; CREATE TABLE test_inserts (`key` Int, `part` Int) ENGINE = MergeTree PARTITION BY part ORDER BY key From 1ae54d3d16cef85f53b87f48b453cd985a04d0af Mon Sep 17 00:00:00 2001 From: peter279k Date: Thu, 8 Sep 2022 01:18:27 +0800 Subject: [PATCH 445/582] Improve clickhouse start command --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 1a17e63a274..92873cb5fbf 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -221,7 +221,7 @@ For non-Linux operating systems and for AArch64 CPU architecture, ClickHouse bui curl -O 'https://builds.clickhouse.com/master/aarch64/clickhouse' && chmod a+x ./clickhouse ``` -Run `sudo ./clickhouse install` to install ClickHouse system-wide (also with needed configuration files, configuring users etc.). Then run `clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it. +Run `sudo ./clickhouse install` to install ClickHouse system-wide (also with needed configuration files, configuring users etc.). Then run `sudo clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it. Use the `clickhouse client` to connect to the server, or `clickhouse local` to process local data. From 0cc6202706f5d5f4fe84341b717da33fc9b18cdb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 5 Sep 2022 15:07:11 +0200 Subject: [PATCH 446/582] Add macos builds to ReleaseBranchCI --- .github/workflows/release_branches.yml | 134 +++++++++++++++++++++++++ 1 file changed, 134 insertions(+) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 6403d00157f..1680798060c 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -426,6 +426,100 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwin: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwinAarch64: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -505,6 +599,46 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" + BuilderSpecialReport: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## From f3cf1069dc51457fc8e749b36b8a2e205f5f54b0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 5 Sep 2022 15:59:20 +0200 Subject: [PATCH 447/582] Fix header for autogenerated version --- tests/ci/version_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index de98b8431de..966858c0747 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -20,7 +20,7 @@ const char * auto_contributors[] {{ VERSIONS = Dict[str, Union[int, str]] -VERSIONS_TEMPLATE = """# This variables autochanged by release_lib.sh: +VERSIONS_TEMPLATE = """# This variables autochanged by tests/ci/version_helper.py: # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. From 257c4328f7b4f2a09d40d7c6c05a4170aba7ba28 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 7 Sep 2022 15:06:44 +0200 Subject: [PATCH 448/582] Rename get_build_urls to read_build_urls --- tests/ci/ast_fuzzer_check.py | 4 ++-- tests/ci/build_download_helper.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 9f3ddbe9932..f3939dc89ad 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -17,7 +17,7 @@ from env_helper import ( from s3_helper import S3Helper from get_robot_token import get_best_robot_token from pr_info import PRInfo -from build_download_helper import get_build_name_for_check, get_build_urls +from build_download_helper import get_build_name_for_check, read_build_urls from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse @@ -69,7 +69,7 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) print(build_name) - urls = get_build_urls(build_name, reports_path) + urls = read_build_urls(build_name, reports_path) if not urls: raise Exception("No build URLs found") diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index f5eb72dddee..fa6097cd680 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -45,7 +45,7 @@ def get_build_name_for_check(check_name): return CI_CONFIG["tests_config"][check_name]["required_build"] -def get_build_urls(build_name, reports_path): +def read_build_urls(build_name, reports_path): for root, _, files in os.walk(reports_path): for f in files: if build_name in f: @@ -111,7 +111,7 @@ def download_builds_filter( check_name, reports_path, result_path, filter_fn=lambda _: True ): build_name = get_build_name_for_check(check_name) - urls = get_build_urls(build_name, reports_path) + urls = read_build_urls(build_name, reports_path) print(urls) if not urls: From 7cdd8c3304155cca20699a6d69cb7498efff6c4f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 7 Sep 2022 17:10:58 +0200 Subject: [PATCH 449/582] Move download template to env_helper --- tests/ci/env_helper.py | 8 ++++++-- tests/ci/push_to_artifactory.py | 21 +++++---------------- 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py index 12c21398781..a18f47497fd 100644 --- a/tests/ci/env_helper.py +++ b/tests/ci/env_helper.py @@ -22,10 +22,14 @@ IMAGES_PATH = os.getenv("IMAGES_PATH", TEMP_PATH) REPORTS_PATH = os.getenv("REPORTS_PATH", p.abspath(p.join(module_dir, "./reports"))) REPO_COPY = os.getenv("REPO_COPY", git_root) RUNNER_TEMP = os.getenv("RUNNER_TEMP", p.abspath(p.join(module_dir, "./tmp"))) -S3_URL = os.getenv("S3_URL", "https://s3.amazonaws.com") -S3_DOWNLOAD = os.getenv("S3_DOWNLOAD", S3_URL) S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") S3_TEST_REPORTS_BUCKET = os.getenv("S3_TEST_REPORTS_BUCKET", "clickhouse-test-reports") +S3_URL = os.getenv("S3_URL", "https://s3.amazonaws.com") +S3_DOWNLOAD = os.getenv("S3_DOWNLOAD", S3_URL) +S3_ARTIFACT_DOWNLOAD_TEMPLATE = ( + f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/" + "{pr_or_release}/{commit}/{build_name}/{artifact}" +) # These parameters are set only on demand, and only once _GITHUB_JOB_ID = "" diff --git a/tests/ci/push_to_artifactory.py b/tests/ci/push_to_artifactory.py index 6b407eb5bd8..c472bcd6b4a 100755 --- a/tests/ci/push_to_artifactory.py +++ b/tests/ci/push_to_artifactory.py @@ -9,7 +9,7 @@ from typing import Dict, List, Tuple from artifactory import ArtifactorySaaSPath # type: ignore from build_download_helper import dowload_build_with_progress -from env_helper import RUNNER_TEMP, S3_BUILDS_BUCKET, S3_DOWNLOAD +from env_helper import S3_ARTIFACT_DOWNLOAD_TEMPLATE, RUNNER_TEMP from git_helper import TAG_REGEXP, commit, removeprefix, removesuffix @@ -97,18 +97,6 @@ class Packages: class S3: - template = ( - f"{S3_DOWNLOAD}/" - # "clickhouse-builds/" - f"{S3_BUILDS_BUCKET}/" - # "33333/" or "21.11/" from --release, if pull request is omitted - "{pr}/" - # "2bef313f75e4cacc6ea2ef2133e8849ecf0385ec/" - "{commit}/" - # "package_release/clickhouse-common-static_21.11.5.0_amd64.deb" - "{s3_path_suffix}" - ) - def __init__( self, pr: int, @@ -117,7 +105,7 @@ class S3: force_download: bool, ): self._common = dict( - pr=pr, + pr_or_release=pr, commit=commit, ) self.force_download = force_download @@ -133,8 +121,9 @@ class S3: self.packages.replace_with_fallback(package_file) return - url = self.template.format_map( - {**self._common, "s3_path_suffix": s3_path_suffix} + build_name, artifact = s3_path_suffix.split("/") + url = S3_ARTIFACT_DOWNLOAD_TEMPLATE.format_map( + {**self._common, "build_name": build_name, "artifact": artifact} ) try: dowload_build_with_progress(url, path) From bd83e905d3bf303660fded6a6a15813b37cdb464 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 7 Sep 2022 18:47:47 +0200 Subject: [PATCH 450/582] Fix a typo in download_build_with_progress --- tests/ci/build_download_helper.py | 4 ++-- tests/ci/push_to_artifactory.py | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index fa6097cd680..04d2cda2465 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -56,7 +56,7 @@ def read_build_urls(build_name, reports_path): return [] -def dowload_build_with_progress(url, path): +def download_build_with_progress(url, path): logging.info("Downloading from %s to temp path %s", url, path) for i in range(DOWNLOAD_RETRIES_COUNT): try: @@ -104,7 +104,7 @@ def download_builds(result_path, build_urls, filter_fn): if filter_fn(url): fname = os.path.basename(url.replace("%2B", "+").replace("%20", " ")) logging.info("Will download %s to %s", fname, result_path) - dowload_build_with_progress(url, os.path.join(result_path, fname)) + download_build_with_progress(url, os.path.join(result_path, fname)) def download_builds_filter( diff --git a/tests/ci/push_to_artifactory.py b/tests/ci/push_to_artifactory.py index c472bcd6b4a..dd8081227bf 100755 --- a/tests/ci/push_to_artifactory.py +++ b/tests/ci/push_to_artifactory.py @@ -8,7 +8,7 @@ from collections import namedtuple from typing import Dict, List, Tuple from artifactory import ArtifactorySaaSPath # type: ignore -from build_download_helper import dowload_build_with_progress +from build_download_helper import download_build_with_progress from env_helper import S3_ARTIFACT_DOWNLOAD_TEMPLATE, RUNNER_TEMP from git_helper import TAG_REGEXP, commit, removeprefix, removesuffix @@ -126,14 +126,14 @@ class S3: {**self._common, "build_name": build_name, "artifact": artifact} ) try: - dowload_build_with_progress(url, path) + download_build_with_progress(url, path) except Exception as e: if "Cannot download dataset from" in e.args[0]: new_url = Packages.fallback_to_all(url) logging.warning( "Fallback downloading %s for old release", fallback_path ) - dowload_build_with_progress(new_url, fallback_path) + download_build_with_progress(new_url, fallback_path) self.packages.replace_with_fallback(package_file) def download_deb(self): From a067907fbe0934c18de20a73f48f7c975bcb2006 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 7 Sep 2022 19:20:22 +0200 Subject: [PATCH 451/582] Add typing and order import --- tests/ci/build_download_helper.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 04d2cda2465..58997bed253 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -1,11 +1,11 @@ #!/usr/bin/env python3 -import os import json import logging +import os import sys import time -from typing import Optional +from typing import List, Optional import requests # type: ignore @@ -41,11 +41,11 @@ def get_with_retries( return response -def get_build_name_for_check(check_name): +def get_build_name_for_check(check_name) -> str: return CI_CONFIG["tests_config"][check_name]["required_build"] -def read_build_urls(build_name, reports_path): +def read_build_urls(build_name, reports_path) -> List[str]: for root, _, files in os.walk(reports_path): for f in files: if build_name in f: From c0ce4c2d6fa19b1515f964001ff2cbaf6af13b84 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 7 Sep 2022 19:20:58 +0200 Subject: [PATCH 452/582] Add macos binaries to GH release assets --- .github/workflows/release.yml | 4 ++ tests/ci/download_binary.py | 79 +++++++++++++++++++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100755 tests/ci/download_binary.py diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 7e12695990c..ae905aa62ba 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -29,8 +29,12 @@ jobs: rm -rf "$TEMP_PATH" && mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" cd "$REPO_COPY" + # Download and push packages to artifactory python3 ./tests/ci/push_to_artifactory.py --release "${{ github.ref }}" \ --commit '${{ github.sha }}' --artifactory-url "${{ secrets.JFROG_ARTIFACTORY_URL }}" --all + # Download macos binaries to ${{runner.temp}}/download_binary + python3 ./tests/ci/download_binary.py binary_darwin binary_darwin_aarch64 + mv '${{runner.temp}}/download_binary/'clickhouse-* '${{runner.temp}}/push_to_artifactory' - name: Upload packages to release assets uses: svenstaro/upload-release-action@v2 with: diff --git a/tests/ci/download_binary.py b/tests/ci/download_binary.py new file mode 100755 index 00000000000..b95c86aa0bd --- /dev/null +++ b/tests/ci/download_binary.py @@ -0,0 +1,79 @@ +#!/usr/bin/env python +""" +This file is needed to avoid cicle import build_download_helper.py <=> env_helper.py +""" + +import argparse +import logging +import os + +from build_download_helper import download_build_with_progress +from ci_config import CI_CONFIG, BuildConfig +from env_helper import RUNNER_TEMP, S3_ARTIFACT_DOWNLOAD_TEMPLATE +from git_helper import Git, commit +from version_helper import get_version_from_repo, version_arg + +TEMP_PATH = os.path.join(RUNNER_TEMP, "download_binary") + + +def parse_args() -> argparse.Namespace: + parser = argparse.ArgumentParser( + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + description="Script to download binary artifacts from S3. Downloaded artifacts " + "are renamed to clickhouse-{static_binary_name}", + ) + parser.add_argument( + "--version", + type=version_arg, + default=get_version_from_repo().string, + help="a version to generate a download url, get from the repo by default", + ) + parser.add_argument( + "--commit", + type=commit, + default=Git(True).sha, + help="a version to generate a download url, get from the repo by default", + ) + parser.add_argument("--rename", default=True, help=argparse.SUPPRESS) + parser.add_argument( + "--no-rename", + dest="rename", + action="store_false", + default=argparse.SUPPRESS, + help="if set, the downloaded binary won't be renamed to " + "clickhouse-{static_binary_name}, makes sense only for a single build name", + ) + parser.add_argument( + "build_names", + nargs="+", + help="the build names to download", + ) + args = parser.parse_args() + if not args.rename and len(args.build_names) > 1: + parser.error("`--no-rename` shouldn't be used with more than one build name") + return args + + +def main(): + logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") + args = parse_args() + os.makedirs(TEMP_PATH, exist_ok=True) + for build in args.build_names: + # check if it's in CI_CONFIG + config = CI_CONFIG["build_config"][build] # type: BuildConfig + if args.rename: + path = os.path.join(TEMP_PATH, f"clickhouse-{config['static_binary_name']}") + else: + path = os.path.join(TEMP_PATH, "clickhouse") + + url = S3_ARTIFACT_DOWNLOAD_TEMPLATE.format( + pr_or_release=f"{args.version.major}.{args.version.minor}", + commit=args.commit, + build_name=build, + artifact="clickhouse", + ) + download_build_with_progress(url, path) + + +if __name__ == "__main__": + main() From 945299de992d0cdd3729bd1f6e2b5b6ed882a807 Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 7 Sep 2022 21:46:17 +0800 Subject: [PATCH 453/582] Remove strange release trains --- docs/en/development/build.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index fa04fbf2680..8712aa3e2bc 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -140,6 +140,6 @@ hash cmake ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour. -They are built for stable, prestable and testing releases as long as for every commit to master and for every pull request. +Binaries are built for stable and LTS releases and also every commit to `master` for each pull request. To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green check mark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”. From 0071ef9e389bf2ce2076b0fe5268027da9f9c7a3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 7 Sep 2022 15:56:31 -0300 Subject: [PATCH 454/582] Update date-time-functions.md --- .../functions/date-time-functions.md | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3515e903adf..49fbb961af8 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -640,7 +640,8 @@ Result: ## date\_diff -Returns the difference between two dates or dates with time values. +Returns the difference between two dates or dates with time values. +The difference is calculated using relative units, e.g. the difference between `2022-01-01` and `2021-12-29` is 3 days for day unit (see (toRelativeDayNum)[#toRelativeDayNum]), 1 month for month unit (see (toRelativeMonthNum)[#toRelativeMonthNum]), 1 year for year unit (see (toRelativeYearNum)[#toRelativeYearNum]). **Syntax** @@ -692,6 +693,25 @@ Result: └────────────────────────────────────────────────────────────────────────────────────────┘ ``` +Query: + +``` sql +SELECT + toDate('2022-01-01') AS e, + toDate('2021-12-29') AS s, + dateDiff('day', s, e) AS day_diff, + dateDiff('month', s, e) AS month__diff, + dateDiff('year', s, e) AS year_diff; +``` + +Result: + +``` text +┌──────────e─┬──────────s─┬─day_diff─┬─month__diff─┬─year_diff─┐ +│ 2022-01-01 │ 2021-12-29 │ 3 │ 1 │ 1 │ +└────────────┴────────────┴──────────┴─────────────┴───────────┘ +``` + ## date\_sub Subtracts the time interval or date interval from the provided date or date with time. From fb6b26c7a48ae2688d106a4b69d76fd5a03d215c Mon Sep 17 00:00:00 2001 From: Yuko Takagi <70714860+yukotakagi@users.noreply.github.com> Date: Wed, 7 Sep 2022 12:58:36 -0600 Subject: [PATCH 455/582] Update README.md (#41091) --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index b173add94e3..49aed14f719 100644 --- a/README.md +++ b/README.md @@ -15,4 +15,5 @@ ClickHouse® is an open-source column-oriented database management system that a * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming events -* [**v22.8 Release Webinar**](https://clickhouse.com/company/events/v22-8-release-webinar) Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release, provide live demos, and share vision into what is coming in the roadmap. +* [**v22.9 Release Webinar**](https://clickhouse.com/company/events/v22-9-release-webinar) Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release, provide live demos, and share vision into what is coming in the roadmap. +* [**ClickHouse for Analytics @ Barracuda Networks**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/288140358/) Join us for this in person meetup hosted by our friends at Barracuda in Bay Area. From a75eb5ad84309227fd2721313f8d7a7754557032 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 7 Sep 2022 15:59:23 -0300 Subject: [PATCH 456/582] Update date-time-functions.md --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 49fbb961af8..2a2681ca1e8 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -641,7 +641,7 @@ Result: ## date\_diff Returns the difference between two dates or dates with time values. -The difference is calculated using relative units, e.g. the difference between `2022-01-01` and `2021-12-29` is 3 days for day unit (see (toRelativeDayNum)[#toRelativeDayNum]), 1 month for month unit (see (toRelativeMonthNum)[#toRelativeMonthNum]), 1 year for year unit (see (toRelativeYearNum)[#toRelativeYearNum]). +The difference is calculated using relative units, e.g. the difference between `2022-01-01` and `2021-12-29` is 3 days for day unit (see [toRelativeDayNum](#toRelativeDayNum)), 1 month for month unit (see [toRelativeMonthNum](#toRelativeMonthNum)), 1 year for year unit (see [toRelativeYearNum](#toRelativeYearNum)). **Syntax** From e74bb00e90fbe2f8c891b7adf43e86c1b6a21761 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 7 Sep 2022 19:11:25 +0000 Subject: [PATCH 457/582] Fix: EXPLAIN PLAN - make settings in SETTING clause effective --- src/Interpreters/IInterpreterUnionOrSelectQuery.h | 2 ++ src/Interpreters/InterpreterExplainQuery.cpp | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index a1c86f9de85..6f893d4703e 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -58,6 +58,8 @@ public: /// Add limits from external query. void addStorageLimits(const StorageLimitsList & limits); + ContextPtr getContext() const { return context; } + protected: ASTPtr query_ptr; ContextMutablePtr context; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4799970b6a1..746d382198d 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -316,7 +316,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() interpreter.buildQueryPlan(plan); if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); + plan.optimize(QueryPlanOptimizationSettings::fromContext(interpreter.getContext())); if (settings.json) { @@ -326,7 +326,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto plan_array = std::make_unique(); plan_array->add(std::move(plan_map)); - auto format_settings = getFormatSettings(getContext()); + auto format_settings = getFormatSettings(interpreter.getContext()); format_settings.json.quote_64bit_integers = false; JSONBuilder::FormatSettings json_format_settings{.settings = format_settings}; From 151bf1e074c3a93197c6f3b9283514504d0eb70e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 20:06:41 +0000 Subject: [PATCH 458/582] fix: suppress warnings for ARM ragel assets --- contrib/vectorscan-cmake/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index 5596fdd23fd..bec93885e9e 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -247,6 +247,11 @@ elseif (ARCH_AARCH64) "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/aarch64/Parser.cpp" "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/aarch64/control_verbs.cpp" ) + set_source_files_properties( + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/aarch64/Parser.cpp" + "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/aarch64/control_verbs.cpp" + COMPILE_FLAGS -Wno-c++11-narrowing + ) endif() # Platform-dependent files From 9b868f8b7db799f0258d5514aa5b5883bd8f7e23 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Sep 2022 20:10:17 +0000 Subject: [PATCH 459/582] doc: add note about upstream colm/ragel --- contrib/vectorscan-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index bec93885e9e..50a5c90db02 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -237,6 +237,8 @@ set (SRCS # # Please regenerate these files if you update vectorscan. They must be regenerated for each platform separately because ragel produces for # weird reasons different constants in the output. +# +# Also, please use upstream versions of colm and ragel, the packages in Ubuntu 22.04 seem to produce wrong output on ARM. if (ARCH_AMD64) list (APPEND SRCS "${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/amd64/Parser.cpp" From 3af51f4340589570888cbb041af22d1a3f68bf73 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 7 Sep 2022 22:21:46 +0200 Subject: [PATCH 460/582] Update storing-data.md --- docs/en/operations/storing-data.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index f52b388d517..926ee92b95c 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -116,7 +116,7 @@ Example of disk configuration: It is possible to configure cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. Cache uses `LRU` cache policy. -Example of configuration for versions later or equal to 21.8: +Example of configuration for versions later or equal to 22.8: ``` xml @@ -137,7 +137,7 @@ Example of configuration for versions later or equal to 21.8: ``` -Example of configuration for versions earlier than 21.8: +Example of configuration for versions earlier than 22.8: ``` xml @@ -156,11 +156,11 @@ Example of configuration for versions earlier than 21.8: Cache **configuration settings**: -- `path` - path to cache Default: None, this settings is obligatory. +- `path` - path to cache. Default: None, this setting is obligatory. -- `max_size` - size of cache in bytes Default: None, this settings is obligatory. +- `max_size` - size of the cache in bytes. Default: None, this setting is obligatory. -- `cache_on_write_operations` - turn on `write-through` cache. Default: `false`. The `write-through` cache is enabled if `cache_on_write_operations` is `true` and user setting `filesystem`. +- `cache_on_write_operations` - turn on `write-through` cache. Default: `false`. The `write-through` cache is enabled if `cache_on_write_operations` is `true` and user setting `enable_filesystem_cache_on_write_operations`. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. @@ -168,9 +168,9 @@ Cache **configuration settings**: - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `true`. -- `max_file_segment_size` - a max size for a single cache file. Default: `100 Mb`. +- `max_file_segment_size` - a max size for a single cache file. Default: `104857600` (100 Mb). -- `max_elements` a limit for a number of cache files. +- `max_elements` a limit for a number of cache files. Default: `1048576`. Cache **query settings**: From eb53df48d1c29238637eb2c4511ce27ec5853e3a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 7 Sep 2022 22:26:52 +0200 Subject: [PATCH 461/582] Update storing-data.md --- docs/en/operations/storing-data.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 926ee92b95c..c0d2c828247 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,7 +184,7 @@ Cache **query settings**: - `max_query_cache_size` - a limit for the cache size, which can be written to local cache storage. Requires enabled `enable_filesystem_query_cache_limit` in cache configuration. Default: `false`. -- `skip_download_if_exceeds_query_cache` - allows to change the behaviour of setting `max_query_cache_size`. Default: `true`. If this setting is turned on and cache download limit during query was reached, no more cache will be downloaded to cache storage. If this setting is turned off and cache download limit during query was reached, cache will still be written by evicting previously written within current query cache data. E.g. second behaviour allows to preserve `last recentltly used` behaviour. +- `skip_download_if_exceeds_query_cache` - allows to change the behaviour of setting `max_query_cache_size`. Default: `true`. If this setting is turned on and cache download limit during query was reached, no more cache will be downloaded to cache storage. If this setting is turned off and cache download limit during query was reached, cache will still be written by cost of evicting previously downloaded (within current query) data, e.g. second behaviour allows to preserve `last recentltly used` behaviour while keeping query cache limit. * Cache configuration settings and cache query settings correspond to the latest ClickHouse version, for earlier versions something might not be supported. From 5a790b15b46b304a57779a4c197788ee4bbf27ce Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Sep 2022 00:20:11 +0000 Subject: [PATCH 462/582] try to fix filling of missed Nested columns with multiple levels --- src/Interpreters/inplaceBlockConversions.cpp | 23 ++++++++------------ src/Storages/MergeTree/IMergeTreeReader.cpp | 6 ++++- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 116cd91c7cf..a4791690f4e 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -200,23 +200,19 @@ static std::unordered_map collectOffsetsColumns( continue; auto serialization = IDataType::getSerialization(*available_column); - auto name_in_storage = Nested::extractTableName(available_column->name); - serialization->enumerateStreams([&](const auto & subpath) { if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) return; + auto stream_name = ISerialization::getFileNameForStream(*available_column, subpath); const auto & current_offsets_column = subpath.back().data.column; /// If for some reason multiple offsets columns are present /// for the same nested data structure, choose the one that is not empty. if (current_offsets_column && !current_offsets_column->empty()) { - auto subname = ISerialization::getSubcolumnNameForStream(subpath); - auto full_name = Nested::concatenateName(name_in_storage, subname); - auto & offsets_column = offsets_columns[full_name]; - + auto & offsets_column = offsets_columns[stream_name]; if (!offsets_column) offsets_column = current_offsets_column; @@ -227,7 +223,7 @@ static std::unordered_map collectOffsetsColumns( if (offsets_data != current_offsets_data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Found non-equal columns with offsets (sizes: {} and {}) for stream {}", - offsets_data.size(), current_offsets_data.size(), full_name); + offsets_data.size(), current_offsets_data.size(), stream_name); #endif } }, available_column->type, res_columns[i]); @@ -255,7 +251,7 @@ void fillMissingColumns( /// but a column of arrays of correct length. /// First, collect offset columns for all arrays in the block. - auto offset_columns = collectOffsetsColumns(available_columns, res_columns); + auto offsets_columns = collectOffsetsColumns(available_columns, res_columns); /// Insert default values only for columns without default expressions. auto requested_column = requested_columns.begin(); @@ -275,14 +271,13 @@ void fillMissingColumns( std::vector current_offsets; size_t num_dimensions = 0; - if (const auto * array_type = typeid_cast(type.get())) + const auto * array_type = typeid_cast(type.get()); + if (array_type && !offsets_columns.empty()) { num_dimensions = getNumberOfDimensions(*array_type); current_offsets.resize(num_dimensions); auto serialization = IDataType::getSerialization(*requested_column); - auto name_in_storage = Nested::extractTableName(requested_column->name); - serialization->enumerateStreams([&](const auto & subpath) { if (subpath.empty() || subpath.back().type != ISerialization::Substream::ArraySizes) @@ -291,9 +286,9 @@ void fillMissingColumns( size_t level = ISerialization::getArrayLevel(subpath); assert(level < num_dimensions); - auto subname = ISerialization::getSubcolumnNameForStream(subpath); - auto it = offset_columns.find(Nested::concatenateName(name_in_storage, subname)); - if (it != offset_columns.end()) + auto stream_name = ISerialization::getFileNameForStream(*requested_column, subpath); + auto it = offsets_columns.find(stream_name); + if (it != offsets_columns.end()) current_offsets[level] = it->second; }); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 73f9cc8b75d..521de3515cf 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -65,7 +65,11 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e try { NamesAndTypesList available_columns(columns_to_read.begin(), columns_to_read.end()); - DB::fillMissingColumns(res_columns, num_rows, requested_columns, available_columns, partially_read_columns, metadata_snapshot); + DB::fillMissingColumns( + res_columns, num_rows, + Nested::convertToSubcolumns(requested_columns), + Nested::convertToSubcolumns(available_columns), + partially_read_columns, metadata_snapshot); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); From 329f31e7ab2f2436a9dc45ec001a06563893852d Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 8 Sep 2022 11:38:10 +0800 Subject: [PATCH 463/582] Address review comments Signed-off-by: Frank Chen --- src/Storages/Distributed/DirectoryMonitor.cpp | 6 +-- ...entelemetry_insert_on_distributed_table.sh | 41 ++++++++----------- 2 files changed, 19 insertions(+), 28 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index f84ddeb4f5e..b3d2494c7e1 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -141,7 +141,7 @@ namespace size_t bytes = 0; UInt32 shard_num = 0; - std::string cluster_name; + std::string cluster; std::string distributed_table; std::string remote_table; @@ -203,7 +203,7 @@ namespace if (header_buf.hasPendingData()) { readVarUInt(distributed_header.shard_num, header_buf); - readStringBinary(distributed_header.cluster_name, header_buf); + readStringBinary(distributed_header.cluster, header_buf); readStringBinary(distributed_header.distributed_table, header_buf); readStringBinary(distributed_header.remote_table, header_buf); } @@ -638,7 +638,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa distributed_header.client_info.client_trace_context, this->storage.getContext()->getOpenTelemetrySpanLog()); thread_trace_context->root_span.addAttribute("clickhouse.shard_num", distributed_header.shard_num); - thread_trace_context->root_span.addAttribute("clickhouse.cluster", distributed_header.cluster_name); + thread_trace_context->root_span.addAttribute("clickhouse.cluster", distributed_header.cluster); thread_trace_context->root_span.addAttribute("clickhouse.distributed", distributed_header.distributed_table); thread_trace_context->root_span.addAttribute("clickhouse.remote", distributed_header.remote_table); thread_trace_context->root_span.addAttribute("clickhouse.rows", distributed_header.rows); diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index b9b5dd2d424..319f0151b1d 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: distributed +# Tags: no-fasttest, distributed set -ue @@ -10,9 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq " -SET distributed_ddl_output_mode = 'none'; - +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; TRUNCATE TABLE IF EXISTS system.opentelemetry_span_log ON CLUSTER test_cluster_two_shards; @@ -28,16 +26,17 @@ CREATE TABLE default.local_opentelemetry ON CLUSTER test_cluster_two_shards (key # Do test with opentelemetry enabled # ${CLICKHOUSE_CLIENT} -nq " --- Make sure it's async -SET insert_distributed_sync=0; -INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1 VALUES(1),(2); +INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1, insert_distributed_sync=0 VALUES(1),(2); " -# Wait 10s to complete of ASYNC INSERT on distributed table and flush of system.opentelemetry_span_log -sleep 10 - # Check log -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " +-- Make sure INSERT on distributed finishes +SYSTEM FLUSH DISTRIBUTED default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; + +-- Make sure opentelemetry span log flushed +SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; + -- Above INSERT will insert data to two shards respectively, so there will be two spans generated SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%writeToLocal%'; SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%processFile%'; @@ -47,24 +46,17 @@ SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_s # INSERT SYNC test # Do test with opentelemetry enabled and in SYNC mode # -${CLICKHOUSE_CLIENT} -nq " - +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " -- Clear log -SET distributed_ddl_output_mode = 'none'; TRUNCATE TABLE IF EXISTS system.opentelemetry_span_log ON CLUSTER test_cluster_two_shards; --- Make sure it's SYNC -SET insert_distributed_sync=1; - --- INSERT test -INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1 VALUES(1),(2); +INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1, insert_distributed_sync=1 VALUES(1),(2); " -# Wait 10s to flush system.opentelemetry_span_log -sleep 10 - # Check log -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " +SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; + -- Above INSERT will insert data to two shards in the same flow, so there should be two spans generated with the same operation name SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%runWritingJob%'; " @@ -72,8 +64,7 @@ SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_s # # Cleanup # -${CLICKHOUSE_CLIENT} -nq " -SET distributed_ddl_output_mode = 'none'; +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " DROP TABLE default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; DROP TABLE default.local_opentelemetry ON CLUSTER test_cluster_two_shards; " From 09c0bf29310ec71626d039173d75467d28a77c7a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Sep 2022 08:16:38 +0000 Subject: [PATCH 464/582] Add unit tests for match path --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 14 +++++++++++++- src/Common/ZooKeeper/tests/gtest_zookeeper.cpp | 15 +++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 src/Common/ZooKeeper/tests/gtest_zookeeper.cpp diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 749052cbba3..3aba0a0651d 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -898,10 +898,22 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); } -PathMatchResult matchPath(const std::string_view path, const std::string_view match_to) +PathMatchResult matchPath(std::string_view path, std::string_view match_to) { using enum PathMatchResult; + if (match_to == "/") + return path == "/" ? EXACT : IS_CHILD; + + const auto clean_path = [](auto & p) + { + if (p.ends_with('/')) + p.remove_suffix(1); + }; + + clean_path(path); + clean_path(match_to); + auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), match_to.begin(), match_to.end()); if (second_it != match_to.end()) diff --git a/src/Common/ZooKeeper/tests/gtest_zookeeper.cpp b/src/Common/ZooKeeper/tests/gtest_zookeeper.cpp new file mode 100644 index 00000000000..5a989e5932f --- /dev/null +++ b/src/Common/ZooKeeper/tests/gtest_zookeeper.cpp @@ -0,0 +1,15 @@ +#include + +#include + +TEST(ZooKeeperTest, TestMatchPath) +{ + using namespace Coordination; + + ASSERT_EQ(matchPath("/path/file", "/path"), PathMatchResult::IS_CHILD); + ASSERT_EQ(matchPath("/path/file", "/path/"), PathMatchResult::IS_CHILD); + ASSERT_EQ(matchPath("/path/file", "/"), PathMatchResult::IS_CHILD); + ASSERT_EQ(matchPath("/", "/"), PathMatchResult::EXACT); + ASSERT_EQ(matchPath("/path", "/path/"), PathMatchResult::EXACT); + ASSERT_EQ(matchPath("/path/", "/path"), PathMatchResult::EXACT); +} From bd10a2195b9730a4131a5c8624804d2a6856bf91 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 8 Sep 2022 10:33:24 +0200 Subject: [PATCH 465/582] Build macos binaries in backport CI, add BuilderSpecialReport to Finish --- .github/workflows/backport_branches.yml | 135 ++++++++++++++++++++++++ .github/workflows/release_branches.yml | 1 + 2 files changed, 136 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index a1086452184..1c51d06f395 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -349,6 +349,100 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwin: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwinAarch64: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -425,6 +519,46 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" + BuilderSpecialReport: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -592,6 +726,7 @@ jobs: - DockerHubPush - DockerServerImages - BuilderReport + - BuilderSpecialReport - FunctionalStatelessTestAsan - FunctionalStatefulTestDebug - StressTestTsan diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 1680798060c..f579d1fee63 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1981,6 +1981,7 @@ jobs: - DockerHubPush - DockerServerImages - BuilderReport + - BuilderSpecialReport - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 From a9863805222bbb90152b57540a3577caa104096a Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 8 Sep 2022 17:25:29 +0800 Subject: [PATCH 466/582] Update src/Storages/Distributed/DirectoryMonitor.cpp Co-authored-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index b3d2494c7e1..e8d48431a9e 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -202,10 +202,10 @@ namespace if (header_buf.hasPendingData()) { - readVarUInt(distributed_header.shard_num, header_buf); - readStringBinary(distributed_header.cluster, header_buf); - readStringBinary(distributed_header.distributed_table, header_buf); - readStringBinary(distributed_header.remote_table, header_buf); + readVarUInt(distributed_header.shard_num, header_buf); + readStringBinary(distributed_header.cluster, header_buf); + readStringBinary(distributed_header.distributed_table, header_buf); + readStringBinary(distributed_header.remote_table, header_buf); } /// Add handling new data here, for example: From 7f086a94a7309b7a708cad806301d553fc20da2b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 8 Sep 2022 12:34:58 +0200 Subject: [PATCH 467/582] Update storing-data.md --- docs/en/operations/storing-data.md | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index c0d2c828247..546e3d7b7a6 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -114,7 +114,7 @@ Example of disk configuration: ## Using local cache {#using-local-cache} -It is possible to configure cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. Cache uses `LRU` cache policy. +It is possible to configure local cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. Cache uses `LRU` cache policy. Example of configuration for versions later or equal to 22.8: @@ -156,37 +156,38 @@ Example of configuration for versions earlier than 22.8: Cache **configuration settings**: -- `path` - path to cache. Default: None, this setting is obligatory. +- `path` - path to the directory with cache. Default: None, this setting is obligatory. -- `max_size` - size of the cache in bytes. Default: None, this setting is obligatory. +- `max_size` - maximum size of the cache in bytes. When the limit is reached, cache files are evicted according to the cache eviction policy. Default: None, this setting is obligatory. -- `cache_on_write_operations` - turn on `write-through` cache. Default: `false`. The `write-through` cache is enabled if `cache_on_write_operations` is `true` and user setting `enable_filesystem_cache_on_write_operations`. +- `cache_on_write_operations` - allow to turn on `write-through` cache (caching data on any write operations: `INSERT` queries, background merges). Default: `false`. The `write-through` cache can be disabled per query using setting `enable_filesystem_cache_on_write_operations` (data is cached only if both cache config settings and corresponding query setting are enabled). - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - a number which defines the number of times some data needs to be read before it will be cached. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - a number, which defines how many times some data needs to be read before it will be cached. Default: `0`, e.g. the data is cached at the first attempt to read it. - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `true`. -- `max_file_segment_size` - a max size for a single cache file. Default: `104857600` (100 Mb). +- `max_file_segment_size` - a maximum size of a single cache file. Default: `104857600` (100 Mb). -- `max_elements` a limit for a number of cache files. Default: `1048576`. +- `max_elements` - a limit for a number of cache files. Default: `1048576`. Cache **query settings**: -- `enable_filesystem_cache` - allows to disable cache even if storage policy was configured with `cache` disk type. Default: `true`. +- `enable_filesystem_cache` - allows to disable cache per query even if storage policy was configured with `cache` disk type. Default: `true`. -- `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` - allows to use cache in query only if it already exists, otherwise cache will not be filled with the query data. Default: `false`. +- `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` - allows to use cache in query only if it already exists, otherwise query data will not be written to local cache storage. Default: `false`. -- `enable_filesystem_cache_on_write_operations` - turn on `write-through` cache. This setting works only if settings `cache_on_write_operations` in cache configuration is turned on. +- `enable_filesystem_cache_on_write_operations` - turn on `write-through` cache. This setting works only if setting `cache_on_write_operations` in cache configuration is turned on. -- `enable_filesystem_cache_log` - turn on writing to `system.filesystem_cache_log` table. Gives a detailed view of cache usage per query. Default: `false`. +- `enable_filesystem_cache_log` - turn on logging to `system.filesystem_cache_log` table. Gives a detailed view of cache usage per query. Default: `false`. - `max_query_cache_size` - a limit for the cache size, which can be written to local cache storage. Requires enabled `enable_filesystem_query_cache_limit` in cache configuration. Default: `false`. - `skip_download_if_exceeds_query_cache` - allows to change the behaviour of setting `max_query_cache_size`. Default: `true`. If this setting is turned on and cache download limit during query was reached, no more cache will be downloaded to cache storage. If this setting is turned off and cache download limit during query was reached, cache will still be written by cost of evicting previously downloaded (within current query) data, e.g. second behaviour allows to preserve `last recentltly used` behaviour while keeping query cache limit. -* Cache configuration settings and cache query settings correspond to the latest ClickHouse version, for earlier versions something might not be supported. +** Warning ** +Cache configuration settings and cache query settings correspond to the latest ClickHouse version, for earlier versions something might not be supported. Cache **system tables**: From 6da37b92e12e0072ad4f87a2ab952d151553dc36 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Sep 2022 13:11:53 +0200 Subject: [PATCH 468/582] Improve logging one time --- src/IO/ReadBufferFromS3.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 3 +++ 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 7e02addd21c..380365f9b95 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -142,7 +142,7 @@ bool ReadBufferFromS3::nextImpl() /// It doesn't make sense to retry Access Denied or No Such Key if (!s3_exception->isRetryableError()) { - tryLogCurrentException(log); + tryLogCurrentException(log, fmt::format("while reading key: {}, from bucket: {}", key, bucket)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f73e0357735..b04d47cc2d4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1046,12 +1046,12 @@ void MergeTreeData::loadDataPartsFromDisk( throw; broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while loading part {} on path {}", part->name, part_path)); + tryLogCurrentException(log, fmt::format("while loading part {} on path {}", part->name, part_path)); } catch (...) { broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while loading part {} on path {}", part->name, part_path)); + tryLogCurrentException(log, fmt::format("while loading part {} on path {}", part->name, part_path)); } /// Ignore broken parts that can appear as a result of hard server restart. @@ -1065,7 +1065,7 @@ void MergeTreeData::loadDataPartsFromDisk( } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while calculating part size {} on path {}", part->name, part_path)); + tryLogCurrentException(log, fmt::format("while calculating part size {} on path {}", part->name, part_path)); } std::string part_size_str = "failed to calculate size"; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4cd0d5fc636..0b05428ceaf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7543,6 +7543,9 @@ std::pair StorageReplicatedMergeTree::unlockSharedData(const IMer if (!settings->allow_remote_fs_zero_copy_replication) return std::make_pair(true, NameSet{}); + if (!part.data_part_storage) + LOG_WARNING(log, "Datapart storage for part {} (temp: {}) is not initialzied", part.name, part.is_temp); + if (!part.data_part_storage || !part.isStoredOnDisk()) { LOG_TRACE(log, "Part {} is not stored on disk, blobs can be removed", part.name); From 122009a2bde416447ef6cbb98ea36c95b47f2b31 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 8 Sep 2022 08:29:31 +0200 Subject: [PATCH 469/582] Use table lock if database is ordinary and zero-copy-replication is enabled. --- src/Backups/BackupEntryWrappedWith.h | 9 +++ src/Storages/MergeTree/MergeTreeData.cpp | 60 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../test_concurrency.py | 9 ++- 6 files changed, 64 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index 893a88db9fd..97244650b6b 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace DB { @@ -25,4 +27,11 @@ private: T custom_value; }; +template +void wrapBackupEntriesWith(std::vector> & backup_entries, const T & custom_value) +{ + for (auto & [_, backup_entry] : backup_entries) + backup_entry = std::make_shared>(std::move(backup_entry), custom_value); +} + } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8b077e6fb13..b9deeeaffb7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4108,25 +4108,49 @@ void MergeTreeData::backupData(BackupEntriesCollector & backup_entries_collector else data_parts = getVisibleDataPartsVector(local_context); - backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup)); + backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, local_context)); } -BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup) const +BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const ContextPtr & local_context) { BackupEntries backup_entries; std::map> temp_dirs; - - /// Tables in atomic databases have UUID. When using atomic database we don't have to create hard links to make a backup, we can just - /// keep smart pointers to data parts instead. That's because the files of a data part are removed only by the destructor of the data part - /// and so keeping a smart pointer to a data part is enough to protect those files from deleting. - bool use_hard_links = !getStorageID().hasUUID(); + TableLockHolder table_lock; for (const auto & part : data_parts) { - BackupEntries new_backup_entries; + /// Hard links is the default way to ensure that we'll be keeping access to the files of parts. + bool make_temporary_hard_links = true; + bool hold_storage_and_part_ptrs = false; + bool hold_table_lock = false; + if (getStorageID().hasUUID()) + { + /// Tables in atomic databases have UUIDs. When using atomic database we don't have to create hard links to make a backup, + /// we can just hold smart pointers to a storage and to data parts instead. That's enough to protect those files from deleting + /// until the backup is done (see the calls `part.unique()` in grabOldParts() and table.unique() in DatabaseCatalog). + make_temporary_hard_links = false; + hold_storage_and_part_ptrs = true; + } + else if (supportsReplication() && part->data_part_storage->supportZeroCopyReplication() && getSettings()->allow_remote_fs_zero_copy_replication) + { + /// Hard links don't work correctly with zero copy replication. + make_temporary_hard_links = false; + hold_storage_and_part_ptrs = true; + hold_table_lock = true; + } + + if (hold_table_lock && !table_lock) + table_lock = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); + + BackupEntries backup_entries_from_part; part->data_part_storage->backup( - part->checksums, part->getFileNamesWithoutChecksums(), data_path_in_backup, new_backup_entries, use_hard_links, &temp_dirs); + part->checksums, + part->getFileNamesWithoutChecksums(), + data_path_in_backup, + backup_entries_from_part, + make_temporary_hard_links, + &temp_dirs); auto projection_parts = part->getProjectionParts(); for (const auto & [projection_name, projection_part] : projection_parts) @@ -4135,19 +4159,23 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con projection_part->checksums, projection_part->getFileNamesWithoutChecksums(), fs::path{data_path_in_backup} / part->name, - new_backup_entries, - use_hard_links, + backup_entries_from_part, + make_temporary_hard_links, &temp_dirs); } - if (!use_hard_links) + if (hold_storage_and_part_ptrs) { - /// Wrap backup entries with data parts in order to keep the data parts alive while the backup entries in use. - for (auto & [_, backup_entry] : new_backup_entries) - backup_entry = std::make_shared>(std::move(backup_entry), part); + /// Wrap backup entries with smart pointers to data parts and to the storage itself + /// (we'll be holding those smart pointers for as long as we'll be using the backup entries). + auto storage_and_part = std::make_pair(shared_from_this(), part); + if (hold_table_lock) + wrapBackupEntriesWith(backup_entries_from_part, std::make_pair(storage_and_part, table_lock)); + else + wrapBackupEntriesWith(backup_entries_from_part, storage_and_part); } - insertAtEnd(backup_entries, std::move(new_backup_entries)); + insertAtEnd(backup_entries, std::move(backup_entries_from_part)); } return backup_entries; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 93f9e6157d8..824c2b41f00 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1243,7 +1243,7 @@ protected: bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space); /// Makes backup entries to backup the parts of this table. - BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup) const; + BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const ContextPtr & local_context); class RestoredPartsHolder; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 66e570fdc3b..5adc1974257 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1785,7 +1785,7 @@ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collec for (const auto & data_part : data_parts) min_data_version = std::min(min_data_version, data_part->info.getDataVersion()); - backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup)); + backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, local_context)); backup_entries_collector.addBackupEntries(backupMutations(min_data_version + 1, data_path_in_backup)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4be97e01293..d662682d228 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8264,7 +8264,7 @@ void StorageReplicatedMergeTree::backupData( else data_parts = getVisibleDataPartsVector(local_context); - auto backup_entries = backupParts(data_parts, ""); + auto backup_entries = backupParts(data_parts, "", local_context); auto coordination = backup_entries_collector.getBackupCoordination(); String shared_id = getTableSharedID(); diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index 315c6b94507..fa6b09a89cf 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -29,7 +29,6 @@ def generate_cluster_def(): main_configs = ["configs/backups_disk.xml", generate_cluster_def()] - user_configs = ["configs/allow_database_types.xml"] nodes = [] @@ -184,6 +183,7 @@ def test_concurrent_backups_on_different_nodes(): def test_create_or_drop_tables_during_backup(db_engine, table_engine): if db_engine == "Replicated": db_engine = "Replicated('/clickhouse/path/','{shard}','{replica}')" + if table_engine.endswith("MergeTree"): table_engine += " ORDER BY tuple()" @@ -219,6 +219,12 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): f"RENAME TABLE {table_name1} TO {table_name2}" ) + def truncate_table(): + while time.time() < end_time: + table_name = f"mydb.tbl{randint(1, num_nodes)}" + node = nodes[randint(0, num_nodes - 1)] + node.query(f"TRUNCATE TABLE IF EXISTS {table_name} NO DELAY") + def make_backup(): ids = [] while time.time() < end_time: @@ -240,6 +246,7 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): futures.append(executor.submit(create_table)) futures.append(executor.submit(drop_table)) futures.append(executor.submit(rename_table)) + futures.append(executor.submit(truncate_table)) for future in futures: future.result() ids = ids_future.result() From 8121904dc4210cfba9e17ab0886dfb1db04e7150 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Sep 2022 12:43:38 +0000 Subject: [PATCH 470/582] Minor update StorageEmbeddedRocksDB::getSampleBlock --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 31cb2f2f9c2..20b1de51a30 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -523,8 +523,7 @@ Chunk StorageEmbeddedRocksDB::getByKeys( Block StorageEmbeddedRocksDB::getSampleBlock(const Names &) const { - auto metadata = getInMemoryMetadataPtr(); - return metadata ? metadata->getSampleBlock() : Block(); + return getInMemoryMetadataPtr()->getSampleBlock(); } Chunk StorageEmbeddedRocksDB::getBySerializedKeys( From b99996961ddcb3126dae15692e21a5710c692f54 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Sep 2022 12:45:36 +0000 Subject: [PATCH 471/582] Address PR comments --- src/Storages/StorageKeeperMap.cpp | 201 ++++++++++++------ src/Storages/StorageKeeperMap.h | 6 +- tests/integration/test_keeper_map/test.py | 4 - .../queries/0_stateless/02416_keeper_map.sql | 3 +- .../02418_keeper_map_keys_limit.sql | 4 +- .../02419_keeper_map_primary_key.reference | 6 + .../02419_keeper_map_primary_key.sh | 23 ++ 7 files changed, 174 insertions(+), 73 deletions(-) create mode 100644 tests/queries/0_stateless/02419_keeper_map_primary_key.reference create mode 100755 tests/queries/0_stateless/02419_keeper_map_primary_key.sh diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index ea333c44362..de7456d0df1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -63,6 +64,29 @@ std::string_view getBaseName(const std::string_view path) return path.substr(last_slash + 1); } +std::string formattedAST(const ASTPtr & ast) +{ + if (!ast) + return ""; + WriteBufferFromOwnString buf; + formatAST(*ast, buf, false, true); + return buf.str(); +} + +void verifyTableId(const StorageID & table_id) +{ + if (!table_id.hasUUID()) + { + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", + table_id.getDatabaseName(), + database->getEngineName()); + } + +} + } class StorageKeeperMapSink : public SinkToStorage @@ -118,6 +142,8 @@ public: size_t current_keys_num = 0; size_t new_keys_num = 0; + // We use keys limit as a soft limit so we ignore some cases when it can be still exceeded + // (e.g if parallel insert queries are being run) if (keys_limit != 0) { Coordination::Stat root_stat; @@ -226,7 +252,6 @@ StorageKeeperMap::StorageKeeperMap( bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path, UInt64 keys_limit_) : IStorage(table_id) , WithContext(context_->getGlobalContext()) @@ -234,44 +259,47 @@ StorageKeeperMap::StorageKeeperMap( , primary_key(primary_key_) , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) , keys_limit(keys_limit_) - , log(&Poco::Logger::get("StorageKeeperMap")) + , log(&Poco::Logger::get(fmt::format("StorageKeeperMap ({})", table_id.getNameForLogs()))) { std::string path_prefix = context_->getConfigRef().getString("keeper_map_path_prefix", ""); if (path_prefix.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap is disabled because 'keeper_map_path_prefix' config is not defined"); - auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); - if (!table_id.hasUUID()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", table_id.getDatabaseName(), database->getEngineName()); + verifyTableId(table_id); setInMemoryMetadata(metadata); + WriteBufferFromOwnString out; + out << "KeeperMap metadata format version: 1\n" + << "columns: " << metadata.columns.toString() + << "primary key: " << formattedAST(metadata.getPrimaryKey().expression_list_ast) << "\n"; + metadata_string = out.str(); + if (root_path.empty()) throw Exception("root_path should not be empty", ErrorCodes::BAD_ARGUMENTS); if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); - if (config_keys_limit != 0 && keys_limit > config_keys_limit) + if (config_keys_limit != 0 && (keys_limit == 0 || keys_limit > config_keys_limit)) { LOG_WARNING( log, - "Keys limit for {} defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use " + "Keys limit defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use " "config defined value", - getStorageID().getFullTableName(), keys_limit, config_keys_limit); keys_limit = config_keys_limit; } else if (keys_limit > 0) { - LOG_INFO(log, "Keys limit for {} will be set to {}", getStorageID().getFullTableName(), keys_limit); + LOG_INFO(log, "Keys limit will be set to {}", keys_limit); } auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1); root_path = root_path_fs.generic_string(); - auto metadata_path_fs = root_path_fs / "ch_metadata"; + auto metadata_path_fs = root_path_fs / "metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; @@ -291,19 +319,9 @@ StorageKeeperMap::StorageKeeperMap( if (root_path != "/" && !client->exists(root_path)) { - if (!create_missing_root_path) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path '{}' doesn't exist. Please create it or set 'create_missing_root_path' to true'", - root_path_); - } - else - { - LOG_TRACE(log, "Creating root path {}", root_path); - client->createAncestors(root_path); - client->createIfNotExists(root_path, ""); - } + LOG_TRACE(log, "Creating root path {}", root_path); + client->createAncestors(root_path); + client->createIfNotExists(root_path, ""); } for (size_t i = 0; i < 1000; ++i) @@ -313,10 +331,15 @@ StorageKeeperMap::StorageKeeperMap( LOG_INFO(log, "Removing leftover nodes"); auto code = client->tryCreate(dropped_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + if (code == Coordination::Error::ZNONODE) { LOG_INFO(log, "Someone else removed leftover nodes"); } + else if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "Someone else is removing leftover nodes"); + continue; + } else if (code != Coordination::Error::ZOK) { throw Coordination::Exception(code, dropped_lock_path); @@ -329,7 +352,29 @@ StorageKeeperMap::StorageKeeperMap( } } - client->createIfNotExists(metadata_path, ""); + std::string stored_metadata_string; + auto exists = client->tryGet(metadata_path, stored_metadata_string); + + if (exists) + { + // this requires same name for columns + // maybe we can do a smarter comparison for columns and primary key expression + if (stored_metadata_string != metadata_string) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", + root_path, + stored_metadata_string); + } + else + { + auto code = client->tryCreate(metadata_path, metadata_string, zkutil::CreateMode::Persistent); + if (code == Coordination::Error::ZNODEEXISTS) + continue; + else if (code != Coordination::Error::ZOK) + throw Coordination::Exception(code, metadata_path); + } + client->createIfNotExists(tables_path, ""); auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); @@ -399,7 +444,15 @@ Pipe StorageKeeperMap::read( auto client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(root_path))); + { + auto children = std::make_shared>(client->getChildren(root_path)); + std::erase_if(*children, [&](const std::string_view key) + { + return fullPathForKey(key) == metadata_path; + }); + + return process_keys(std::move(children)); + } return process_keys(std::move(filtered_keys)); } @@ -525,29 +578,51 @@ std::optional StorageKeeperMap::isTableValid() const if (table_is_valid.has_value()) return *table_is_valid; - try + [&] { - // validate all metadata nodes are present - Coordination::Requests requests; - requests.push_back(zkutil::makeCheckRequest(table_path, -1)); + try + { + auto client = getClient(); - Coordination::Responses responses; - auto client = getClient(); - auto res = client->tryMulti(requests, responses); - table_is_valid = res == Coordination::Error::ZOK; - } - catch (const Coordination::Exception & e) - { - tryLogCurrentException(log); + std::string stored_metadata_string; + Coordination::Stat metadata_stat; + client->tryGet(metadata_path, stored_metadata_string, &metadata_stat); - if (!Coordination::isHardwareError(e.code)) - table_is_valid = false; - } - catch (const Exception &) - { - tryLogCurrentException(log); - table_is_valid = false; - } + if (metadata_stat.numChildren == 0) + { + table_is_valid = false; + return; + } + + if (metadata_string != stored_metadata_string) + { + LOG_ERROR( + log, + "Table definition does not match to the one stored in the path {}. Stored defintion: {}", + root_path, + stored_metadata_string); + table_is_valid = false; + return; + } + + // validate all metadata nodes are present + Coordination::Requests requests; + requests.push_back(zkutil::makeCheckRequest(table_path, -1)); + requests.push_back(zkutil::makeCheckRequest(dropped_path, -1)); + + Coordination::Responses responses; + client->tryMulti(requests, responses); + + table_is_valid = responses[0]->error == Coordination::Error::ZOK && responses[1]->error == Coordination::Error::ZNONODE; + } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log); + + if (!Coordination::isHardwareError(e.code)) + table_is_valid = false; + } + }(); return table_is_valid; } @@ -585,12 +660,6 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (const auto & key : keys) { const auto full_path = fullPathForKey(key); - if (full_path == metadata_path) - { - values.emplace_back(); - continue; - } - values.emplace_back(client->asyncTryGet(full_path)); } @@ -599,9 +668,6 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (size_t i = 0; i < keys.size(); ++i) { auto & value = values[i]; - if (!value.valid()) - continue; - if (value.wait_until(wait_until) != std::future_status::ready) throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch values: timeout"); @@ -634,7 +700,17 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k Block StorageKeeperMap::getSampleBlock(const Names &) const { auto metadata = getInMemoryMetadataPtr(); - return metadata ? metadata->getSampleBlock() : Block(); + return metadata->getSampleBlock(); +} + +void StorageKeeperMap::checkTableCanBeRenamed(const StorageID & new_name) const +{ + verifyTableId(new_name); +} + +void StorageKeeperMap::rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id) +{ + checkTableCanBeRenamed(new_table_id); } namespace @@ -643,24 +719,19 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 3) + if (engine_args.empty() || engine_args.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-3 arguments:\n" "root_path: path in the Keeper where the values will be stored (required)\n" - "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); const auto root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); auto root_path = checkAndGetLiteralArgument(root_path_node, "root_path"); - bool create_missing_root_path = true; - if (engine_args.size() > 1) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); - UInt64 keys_limit = 0; - if (engine_args.size() > 2) - keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); + if (engine_args.size() > 1) + keys_limit = checkAndGetLiteralArgument(engine_args[1], "keys_limit"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -675,7 +746,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path, keys_limit); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, keys_limit); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 03360dd31dd..504d3df087d 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -30,7 +30,6 @@ public: bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path, UInt64 keys_limit_); Pipe read( @@ -55,6 +54,9 @@ public: Block getSampleBlock(const Names &) const override; + void checkTableCanBeRenamed(const StorageID & new_name) const override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; + bool supportsParallelInsert() const override { return true; } bool supportsIndexForIn() const override { return true; } bool mayBenefitFromIndexForIn( @@ -118,6 +120,8 @@ private: std::string zookeeper_name; + std::string metadata_string; + uint64_t keys_limit{0}; mutable std::mutex zookeeper_mutex; diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 7dce77dec42..7801dd3c57f 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -47,10 +47,6 @@ def remove_children(client, path): def test_create_keeper_map(started_cluster): - assert "Path '/test1' doesn't exist" in node.query_and_get_error( - "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1', 0) PRIMARY KEY(key);" - ) - node.query( "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" ) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index 3866c33f03d..a87ad6d914b 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,10 +1,11 @@ --- Tags: no-ordinary-database, no-fasttest, no-parallel +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02416_test SYNC; CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416'); -- { serverError 36 } CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key2); -- { serverError 47 } CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(concat(key, value)); -- { serverError 36 } CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); DROP TABLE IF EXISTS 02416_test SYNC; diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql index de84bde3aaf..3d2055b85ea 100644 --- a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS 02418_test SYNC; -CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 3) PRIMARY KEY(key); +CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 3) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2); SELECT count() FROM 02418_test; @@ -12,7 +12,7 @@ INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3); SELECT count() FROM 02418_test; -CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 4) PRIMARY KEY(key); +CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 4) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test_another VALUES (4, 4.4); diff --git a/tests/queries/0_stateless/02419_keeper_map_primary_key.reference b/tests/queries/0_stateless/02419_keeper_map_primary_key.reference new file mode 100644 index 00000000000..8394d9f34a7 --- /dev/null +++ b/tests/queries/0_stateless/02419_keeper_map_primary_key.reference @@ -0,0 +1,6 @@ +1.1 +2.2 +1.1 +2.2 +1.1 +2.2 diff --git a/tests/queries/0_stateless/02419_keeper_map_primary_key.sh b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh new file mode 100755 index 00000000000..d5ec3be58da --- /dev/null +++ b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02419_test SYNC;" + +test_primary_key() +{ + $CLICKHOUSE_CLIENT -nm -q " + CREATE TABLE 02419_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 3) PRIMARY KEY($1); + INSERT INTO 02419_test VALUES (1, 1.1), (2, 2.2); + SELECT value FROM 02419_test WHERE key = 1; + SELECT value FROM 02419_test WHERE key IN (2, 3); + DROP TABLE 02419_test SYNC; + " +} + +test_primary_key "sipHash64(key + 42) * 12212121212121" +test_primary_key "reverse(concat(CAST(key, 'String'), 'some string'))" +test_primary_key "hex(toFloat32(key))" From b6f8e067080dbb7929aee9154e34416fc6f4d286 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Sep 2022 14:57:34 +0200 Subject: [PATCH 472/582] Typo fix --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index de7456d0df1..fbc4d34b2e7 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -598,7 +598,7 @@ std::optional StorageKeeperMap::isTableValid() const { LOG_ERROR( log, - "Table definition does not match to the one stored in the path {}. Stored defintion: {}", + "Table definition does not match to the one stored in the path {}. Stored definition: {}", root_path, stored_metadata_string); table_is_valid = false; From fab1b40928ed0ffe6b2e0de120f3612d7e572b29 Mon Sep 17 00:00:00 2001 From: Vincent Bernat Date: Thu, 8 Sep 2022 15:21:59 +0200 Subject: [PATCH 473/582] docs: mention SYNC modifier for DROP and DETACH statements --- docs/en/sql-reference/statements/detach.md | 4 +++- docs/en/sql-reference/statements/drop.md | 10 +++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/detach.md b/docs/en/sql-reference/statements/detach.md index c7089b0714d..aa87b1ef613 100644 --- a/docs/en/sql-reference/statements/detach.md +++ b/docs/en/sql-reference/statements/detach.md @@ -10,7 +10,7 @@ Makes the server "forget" about the existence of a table, a materialized view, o **Syntax** ``` sql -DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] +DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] [SYNC] ``` Detaching does not delete the data or metadata of a table, a materialized view or a dictionary. If an entity was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view/dictionary again. If an entity was detached `PERMANENTLY`, there will be no automatic recall. @@ -24,6 +24,8 @@ Note that you can not detach permanently the table which is already detached (te Also you can not [DROP](../../sql-reference/statements/drop#drop-table) the detached table, or [CREATE TABLE](../../sql-reference/statements/create/table.md) with the same name as detached permanently, or replace it with the other table with [RENAME TABLE](../../sql-reference/statements/rename.md) query. +The `SYNC` modifier executes the action without delay. + **Example** Creating a table: diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index 28d379421f1..8a83a8fae1d 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -6,7 +6,7 @@ sidebar_label: DROP # DROP Statements -Deletes existing entity. If the `IF EXISTS` clause is specified, these queries do not return an error if the entity does not exist. +Deletes existing entity. If the `IF EXISTS` clause is specified, these queries do not return an error if the entity does not exist. If the `SYNC` modifier is specified, the entity is dropped without delay. ## DROP DATABASE @@ -15,7 +15,7 @@ Deletes all tables inside the `db` database, then deletes the `db` database itse Syntax: ``` sql -DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] +DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] [SYNC] ``` ## DROP TABLE @@ -25,7 +25,7 @@ Deletes the table. Syntax: ``` sql -DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] +DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] [SYNC] ``` ## DROP DICTIONARY @@ -35,7 +35,7 @@ Deletes the dictionary. Syntax: ``` sql -DROP DICTIONARY [IF EXISTS] [db.]name +DROP DICTIONARY [IF EXISTS] [db.]name [SYNC] ``` ## DROP USER @@ -95,7 +95,7 @@ Deletes a view. Views can be deleted by a `DROP TABLE` command as well but `DROP Syntax: ``` sql -DROP VIEW [IF EXISTS] [db.]name [ON CLUSTER cluster] +DROP VIEW [IF EXISTS] [db.]name [ON CLUSTER cluster] [SYNC] ``` ## DROP FUNCTION From 3502718a2c7e842f78490f08c8b0b021f549072f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Sep 2022 13:52:14 +0000 Subject: [PATCH 474/582] Less complications --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 3aba0a0651d..4ab93d814df 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -902,17 +902,11 @@ PathMatchResult matchPath(std::string_view path, std::string_view match_to) { using enum PathMatchResult; - if (match_to == "/") - return path == "/" ? EXACT : IS_CHILD; + if (path.ends_with('/')) + path.remove_suffix(1); - const auto clean_path = [](auto & p) - { - if (p.ends_with('/')) - p.remove_suffix(1); - }; - - clean_path(path); - clean_path(match_to); + if (match_to.ends_with('/')) + match_to.remove_suffix(1); auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), match_to.begin(), match_to.end()); From 5e00124f8a044b6f76adb592ee6c982eb66dad14 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Sep 2022 16:18:21 +0200 Subject: [PATCH 475/582] More logging for S3 --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 3 +++ src/IO/WriteBufferFromS3.cpp | 12 ++++++++---- src/Storages/MergeTree/DataPartsExchange.cpp | 14 +++++++++++--- src/Storages/MergeTree/DataPartsExchange.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 5 files changed, 28 insertions(+), 13 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 913fd76bf8a..db8f90e777d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -282,7 +282,10 @@ String DiskObjectStorage::getUniqueId(const String & path) const bool DiskObjectStorage::checkUniqueId(const String & id) const { if (!id.starts_with(object_storage_root_path)) + { + LOG_DEBUG(log, "Blob with id {} doesn't start with blob storage prefix {}", id, object_storage_root_path); return false; + } auto object = StoredObject::create(*object_storage, id, {}, {}, true); return object_storage->exists(object); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index cc46af361cd..b09abda85db 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -346,9 +346,10 @@ void WriteBufferFromS3::completeMultipartUpload() LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, tags.size()); else { - throw Exception(ErrorCodes::S3_ERROR, "{} Tags:{}", - outcome.GetError().GetMessage(), - fmt::join(tags.begin(), tags.end(), " ")); + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Tags: {}", + outcome.GetError().GetMessage(), key, bucket, fmt::join(tags.begin(), tags.end(), " ")); } } @@ -433,7 +434,10 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) if (outcome.IsSuccess()) LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); else - throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Object size: {}, WithPool: {}", + outcome.GetError().GetMessage(), key, bucket, task.req.GetContentLength(), with_pool); } void WriteBufferFromS3::waitForReadyBackGroundTasks() diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 9f8313a4700..ecb692aaf9b 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -399,7 +399,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in table", name); } -MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( +MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, @@ -420,6 +420,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( if (blocker.isCancelled()) throw Exception("Fetching of part was cancelled", ErrorCodes::ABORTED); + const auto data_settings = data.getSettings(); + + if (data_settings->allow_remote_fs_zero_copy_replication && !try_zero_copy) + LOG_WARNING(log, "Zero copy replication enabled, but trying to fetch part {} without zero copy", part_name); + /// It should be "tmp-fetch_" and not "tmp_fetch_", because we can fetch part to detached/, /// but detached part name prefix should not contain underscore. static const String TMP_PREFIX = "tmp-fetch_"; @@ -429,7 +434,6 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( /// Validation of the input that may come from malicious replica. auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); - const auto data_settings = data.getSettings(); Poco::URI uri; uri.setScheme(interserver_scheme); @@ -465,6 +469,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( capability.push_back(toString(disk->getDataSourceDescription().type)); } } + if (!capability.empty()) { ::sort(capability.begin(), capability.end()); @@ -474,6 +479,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( } else { + if (data_settings->allow_remote_fs_zero_copy_replication) + LOG_WARNING(log, "Cannot select any zero-copy disk for {}", part_name); + try_zero_copy = false; } @@ -585,7 +593,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( temporary_directory_lock = {}; /// Try again but without zero-copy - return fetchPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts, + return fetchSelectedPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts, user, password, interserver_scheme, throttler, to_detached, tmp_prefix, nullptr, false, disk); } } diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 0e19bf4cdcd..e2582c42dfb 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -66,7 +66,7 @@ public: explicit Fetcher(StorageReplicatedMergeTree & data_) : data(data_), log(&Poco::Logger::get("Fetcher")) {} /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. - MergeTreeData::MutableDataPartPtr fetchPart( + MergeTreeData::MutableDataPartPtr fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0b05428ceaf..3491f6c9d4a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2180,7 +2180,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (interserver_scheme != address.scheme) throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR); - part_desc->res_part = fetcher.fetchPart( + part_desc->res_part = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_desc->found_new_part_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_"); @@ -2299,7 +2299,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR); - return fetcher.fetchPart( + return fetcher.fetchSelectedPart( metadata_snapshot, getContext(), entry.new_part_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, @@ -3934,7 +3934,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH); - return fetcher.fetchPart( + return fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_name, @@ -4071,7 +4071,7 @@ DataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( currently_fetching_parts.erase(part_name); }); - LOG_DEBUG(log, "Fetching part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetching already known part {} from {}", part_name, source_replica_path); TableLockHolder table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -4101,7 +4101,7 @@ DataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH); - return fetcher.fetchPart( + return fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), From 9acc73d811474b61a164debc02a9b730ffab1eb9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 8 Sep 2022 17:51:29 +0200 Subject: [PATCH 476/582] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d662682d228..93626f768be 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8264,7 +8264,7 @@ void StorageReplicatedMergeTree::backupData( else data_parts = getVisibleDataPartsVector(local_context); - auto backup_entries = backupParts(data_parts, "", local_context); + auto backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", local_context); auto coordination = backup_entries_collector.getBackupCoordination(); String shared_id = getTableSharedID(); From e48cd2b6f6d70236d6b7b66dc47c1cb03d5d8488 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 8 Sep 2022 18:13:28 +0200 Subject: [PATCH 477/582] Add more test cases. --- .../test_concurrency.py | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index fa6b09a89cf..6882cbff5e2 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -178,12 +178,17 @@ def test_concurrent_backups_on_different_nodes(): ("Ordinary", "MergeTree"), ("Atomic", "MergeTree"), ("Replicated", "ReplicatedMergeTree"), + ("Memory", "MergeTree"), + ("Lazy", "Log") ], ) def test_create_or_drop_tables_during_backup(db_engine, table_engine): if db_engine == "Replicated": db_engine = "Replicated('/clickhouse/path/','{shard}','{replica}')" + if db_engine == "Lazy": + db_engine = "Lazy(20)" + if table_engine.endswith("MergeTree"): table_engine += " ORDER BY tuple()" @@ -193,7 +198,7 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): start_time = time.time() end_time = start_time + 60 - def create_table(): + def create_tables(): while time.time() < end_time: node = nodes[randint(0, num_nodes - 1)] table_name = f"mydb.tbl{randint(1, num_nodes)}" @@ -204,13 +209,13 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): f"INSERT INTO {table_name} SELECT rand32() FROM numbers(10)" ) - def drop_table(): + def drop_tables(): while time.time() < end_time: table_name = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] node.query(f"DROP TABLE IF EXISTS {table_name} NO DELAY") - def rename_table(): + def rename_tables(): while time.time() < end_time: table_name1 = f"mydb.tbl{randint(1, num_nodes)}" table_name2 = f"mydb.tbl{randint(1, num_nodes)}" @@ -219,13 +224,13 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): f"RENAME TABLE {table_name1} TO {table_name2}" ) - def truncate_table(): + def truncate_tables(): while time.time() < end_time: table_name = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] node.query(f"TRUNCATE TABLE IF EXISTS {table_name} NO DELAY") - def make_backup(): + def make_backups(): ids = [] while time.time() < end_time: time.sleep( @@ -241,12 +246,12 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): ids = [] with concurrent.futures.ThreadPoolExecutor(max_workers=5) as executor: futures = [] - ids_future = executor.submit(make_backup) + ids_future = executor.submit(make_backups) futures.append(ids_future) - futures.append(executor.submit(create_table)) - futures.append(executor.submit(drop_table)) - futures.append(executor.submit(rename_table)) - futures.append(executor.submit(truncate_table)) + futures.append(executor.submit(create_tables)) + futures.append(executor.submit(drop_tables)) + futures.append(executor.submit(rename_tables)) + futures.append(executor.submit(truncate_tables)) for future in futures: future.result() ids = ids_future.result() From 132727d2121a677f5f97fcf2ccf3bb960af40b56 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 8 Sep 2022 14:37:42 -0300 Subject: [PATCH 478/582] Doc. Fix links. --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 2a2681ca1e8..ced96078ce1 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -641,7 +641,7 @@ Result: ## date\_diff Returns the difference between two dates or dates with time values. -The difference is calculated using relative units, e.g. the difference between `2022-01-01` and `2021-12-29` is 3 days for day unit (see [toRelativeDayNum](#toRelativeDayNum)), 1 month for month unit (see [toRelativeMonthNum](#toRelativeMonthNum)), 1 year for year unit (see [toRelativeYearNum](#toRelativeYearNum)). +The difference is calculated using relative units, e.g. the difference between `2022-01-01` and `2021-12-29` is 3 days for day unit (see [toRelativeDayNum](#torelativedaynum)), 1 month for month unit (see [toRelativeMonthNum](#torelativemonthnum)), 1 year for year unit (see [toRelativeYearNum](#torelativeyearnum)). **Syntax** From 10629a66e582c7a2250b01d7a1e4853223dcd631 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 8 Sep 2022 20:58:51 +0200 Subject: [PATCH 479/582] Fix black. --- .../test_backup_restore_on_cluster/test_concurrency.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index 6882cbff5e2..dd8b6aa50da 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -179,7 +179,7 @@ def test_concurrent_backups_on_different_nodes(): ("Atomic", "MergeTree"), ("Replicated", "ReplicatedMergeTree"), ("Memory", "MergeTree"), - ("Lazy", "Log") + ("Lazy", "Log"), ], ) def test_create_or_drop_tables_during_backup(db_engine, table_engine): From 18f5b5e5b07dd02be68beca38484e85190568cee Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 8 Sep 2022 16:05:56 -0600 Subject: [PATCH 480/582] Add docs for lightweight deletes --- .../sql-reference/statements/alter/delete.md | 6 +++- docs/en/sql-reference/statements/delete.md | 36 +++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql-reference/statements/delete.md diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index 809715b5423..1cd6d466788 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -12,7 +12,11 @@ ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr Deletes data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). -:::note +:::info +The `ALTER TABLE` command is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](../delete.md), which performs a lightweight delete and can be considerably faster. +::: + +:::note The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. ::: diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md new file mode 100644 index 00000000000..982e8a0e2da --- /dev/null +++ b/docs/en/sql-reference/statements/delete.md @@ -0,0 +1,36 @@ +--- +slug: /en/sql-reference/statements/delete +sidebar_position: 36 +sidebar_label: DELETE +--- + +# DELETE Statement + +``` sql +DELETE FROM [db.]table [WHERE expr] +``` + +For MergeTree tables, `DELETE FROM` performs a lightweight delete on the given table, which means that the deleted rows are marked as deleted immediately and deleted rows will be filtered out of all subsequent queries. The underlying data is permanently deleted whenever merges occur. + +For example, the following query deletes all rows from the `hits` table where the `Title` column contains the text `hello`: + +```sql +DELETE FROM hits WHERE Title LIKE '%hello%'; +``` + + +:::note +This feature is experimental and requires you to set `allow_experimental_lightweight_delete` to true: + +```sql +SET allow_experimental_lightweight_delete = true; +``` + +::: + +The [traditional way to delete rows](./alter/delete.md) in ClickHouse was to use `ALTER TABLE ... DELETE`, which is still a valid method for deleting rows. However, in most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. + +:::info +Lightweight deletes are asynchronous by default. Set `mutations_sync` equal to 1 to wait for one replica to process the statement, and set `mutations_sync` to 2 to wait for all replicas. +::: + From d4dd6b05b27b41c4db47bae627992d500983fcaf Mon Sep 17 00:00:00 2001 From: lthaooo <354210402@qq.com> Date: Fri, 9 Sep 2022 10:22:35 +0800 Subject: [PATCH 481/582] optimize compact part mutation --- src/Interpreters/MutationsInterpreter.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index c496995ba65..26b8bce1f4a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -473,6 +473,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); + std::vector read_columns; /// First, break a sequence of commands into stages. for (auto & command : commands) { @@ -706,17 +707,23 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) else if (command.type == MutationCommand::READ_COLUMN) { mutation_kind.set(MutationKind::MUTATE_OTHER); - if (stages.empty() || !stages.back().column_to_updated.empty()) - stages.emplace_back(context); - if (stages.size() == 1) /// First stage only supports filtering and can't update columns. - stages.emplace_back(context); - - stages.back().column_to_updated.emplace(command.column_name, std::make_shared(command.column_name)); + read_columns.emplace_back(command.column_name); } else throw Exception("Unknown mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); } + if (!read_columns.empty()) + { + if (stages.empty() || !stages.back().column_to_updated.empty()) + stages.emplace_back(context); + if (stages.size() == 1) /// First stage only supports filtering and can't update columns. + stages.emplace_back(context); + + for (auto & column_name : read_columns) + stages.back().column_to_updated.emplace(column_name, std::make_shared(column_name)); + } + /// We care about affected indices and projections because we also need to rewrite them /// when one of index columns updated or filtered with delete. /// The same about columns, that are needed for calculation of TTL expressions. From 237abffdba538c9a4acc85db4b15dcdba4e735ac Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 9 Sep 2022 11:59:53 +0800 Subject: [PATCH 482/582] Improve test Signed-off-by: Frank Chen --- ...etry_insert_on_distributed_table.reference | 8 +- ...entelemetry_insert_on_distributed_table.sh | 92 ++++++++++++++----- 2 files changed, 72 insertions(+), 28 deletions(-) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference index fac9fabce8a..98fb6a68656 100644 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference @@ -1,4 +1,4 @@ -{'clickhouse.shard_num':'1','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} -{'clickhouse.shard_num':'2','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} -{'clickhouse.shard_num':'1','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} -{'clickhouse.shard_num':'2','clickhouse.cluster':'test_cluster_two_shards','clickhouse.distributed':'default.dist_opentelemetry','clickhouse.remote':'default.local_opentelemetry','clickhouse.rows':'1','clickhouse.bytes':'8'} +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index 319f0151b1d..7ec0650aaac 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -11,60 +11,104 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " -SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; -TRUNCATE TABLE IF EXISTS system.opentelemetry_span_log ON CLUSTER test_cluster_two_shards; +DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.dist_opentelemetry; +DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.local_opentelemetry; -DROP TABLE IF EXISTS default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; -DROP TABLE IF EXISTS default.local_opentelemetry ON CLUSTER test_cluster_two_shards; - -CREATE TABLE default.dist_opentelemetry ON CLUSTER test_cluster_two_shards (key UInt64) Engine=Distributed('test_cluster_two_shards', default, local_opentelemetry, key % 2); -CREATE TABLE default.local_opentelemetry ON CLUSTER test_cluster_two_shards (key UInt64) Engine=MergeTree ORDER BY key; +CREATE TABLE ${CLICKHOUSE_DATABASE}.dist_opentelemetry (key UInt64) Engine=Distributed('test_cluster_two_shards', ${CLICKHOUSE_DATABASE}, local_opentelemetry, key % 2); +CREATE TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry (key UInt64) Engine=MergeTree ORDER BY key; " # # INSERT ASYNC test # Do test with opentelemetry enabled # -${CLICKHOUSE_CLIENT} -nq " -INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1, insert_distributed_sync=0 VALUES(1),(2); -" +trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); +echo "INSERT INTO ${CLICKHOUSE_DATABASE}.dist_opentelemetry SETTINGS insert_distributed_sync=0 VALUES(1),(2)" | +${CLICKHOUSE_CURL} \ + -X POST \ + -H "traceparent: 00-$trace_id-5250000000000525-01" \ + -H "tracestate: some custom state" \ + "${CLICKHOUSE_URL}" \ + --data @- # Check log ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " -- Make sure INSERT on distributed finishes -SYSTEM FLUSH DISTRIBUTED default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; +SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry; -- Make sure opentelemetry span log flushed -SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; +SYSTEM FLUSH LOGS; -- Above INSERT will insert data to two shards respectively, so there will be two spans generated -SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%writeToLocal%'; -SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%processFile%'; +SELECT count() FROM system.opentelemetry_span_log +WHERE lower(hex(trace_id)) = '${trace_id}' +AND operation_name like '%writeToLocal%' +AND attribute['clickhouse.shard_num'] = '1' +AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' +AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' +AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' +AND attribute['clickhouse.rows'] = '1' +AND attribute['clickhouse.bytes'] = '8' +; + +SELECT count() FROM system.opentelemetry_span_log +WHERE lower(hex(trace_id)) = '${trace_id}' +AND operation_name like '%writeToLocal%' +AND attribute['clickhouse.shard_num'] = '2' +AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' +AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' +AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' +AND attribute['clickhouse.rows'] = '1' +AND attribute['clickhouse.bytes'] = '8' +; + " # # INSERT SYNC test # Do test with opentelemetry enabled and in SYNC mode # -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " --- Clear log -TRUNCATE TABLE IF EXISTS system.opentelemetry_span_log ON CLUSTER test_cluster_two_shards; - -INSERT INTO default.dist_opentelemetry SETTINGS opentelemetry_start_trace_probability=1, insert_distributed_sync=1 VALUES(1),(2); -" +trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); +echo "INSERT INTO ${CLICKHOUSE_DATABASE}.dist_opentelemetry SETTINGS insert_distributed_sync=1 VALUES(1),(2)" | +${CLICKHOUSE_CURL} \ + -X POST \ + -H "traceparent: 00-$trace_id-5250000000000525-01" \ + -H "tracestate: some custom state" \ + "${CLICKHOUSE_URL}" \ + --data @- # Check log ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " -SYSTEM FLUSH LOGS ON CLUSTER test_cluster_two_shards; +SYSTEM FLUSH LOGS; -- Above INSERT will insert data to two shards in the same flow, so there should be two spans generated with the same operation name -SELECT attribute FROM cluster('test_cluster_two_shards', system, opentelemetry_span_log) WHERE operation_name like '%runWritingJob%'; +SELECT count() FROM system.opentelemetry_span_log +WHERE lower(hex(trace_id)) = '${trace_id}' +AND operation_name like '%runWritingJob%' +AND attribute['clickhouse.shard_num'] = '1' +AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' +AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' +AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' +AND attribute['clickhouse.rows'] = '1' +AND attribute['clickhouse.bytes'] = '8' +; + +SELECT count() FROM system.opentelemetry_span_log +WHERE lower(hex(trace_id)) = '${trace_id}' +AND operation_name like '%runWritingJob%' +AND attribute['clickhouse.shard_num'] = '2' +AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' +AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' +AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' +AND attribute['clickhouse.rows'] = '1' +AND attribute['clickhouse.bytes'] = '8' +; " # # Cleanup # ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " -DROP TABLE default.dist_opentelemetry ON CLUSTER test_cluster_two_shards; -DROP TABLE default.local_opentelemetry ON CLUSTER test_cluster_two_shards; +DROP TABLE ${CLICKHOUSE_DATABASE}.dist_opentelemetry; +DROP TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry; " From 92a92baa33842f6a1aeee232f22a17b4fd71e923 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 9 Sep 2022 12:18:27 +0800 Subject: [PATCH 483/582] Simplify test Signed-off-by: Frank Chen --- ...entelemetry_insert_on_distributed_table.sh | 126 ++++++------------ 1 file changed, 42 insertions(+), 84 deletions(-) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index 7ec0650aaac..1b4e1da97f9 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -1,16 +1,44 @@ #!/usr/bin/env bash # Tags: no-fasttest, distributed -set -ue - -unset CLICKHOUSE_LOG_COMMENT - CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " +function insert() +{ + echo "INSERT INTO ${CLICKHOUSE_DATABASE}.dist_opentelemetry SETTINGS insert_distributed_sync=$2 VALUES(1),(2)" | + ${CLICKHOUSE_CURL} \ + -X POST \ + -H "traceparent: 00-$1-5150000000000515-01" \ + -H "tracestate: some custom state" \ + "${CLICKHOUSE_URL}" \ + --data @- +} + +function check_span() +{ +${CLICKHOUSE_CLIENT} -nq " + SYSTEM FLUSH LOGS; + + SELECT count() FROM system.opentelemetry_span_log + WHERE lower(hex(trace_id)) = '${1}' + AND operation_name like '${2}' + AND attribute['clickhouse.shard_num'] = '${3}' + AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' + AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' + AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' + AND attribute['clickhouse.rows'] = '1' + AND attribute['clickhouse.bytes'] = '8' + ;" +} + + +# +# Prepare tables for tests +# +${CLICKHOUSE_CLIENT} -nq " DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.dist_opentelemetry; DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.local_opentelemetry; @@ -19,96 +47,26 @@ CREATE TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry (key UInt64) Engine=Merg " # -# INSERT ASYNC test -# Do test with opentelemetry enabled +# ASYNC INSERT test with opentelemetry enabled # trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); -echo "INSERT INTO ${CLICKHOUSE_DATABASE}.dist_opentelemetry SETTINGS insert_distributed_sync=0 VALUES(1),(2)" | -${CLICKHOUSE_CURL} \ - -X POST \ - -H "traceparent: 00-$trace_id-5250000000000525-01" \ - -H "tracestate: some custom state" \ - "${CLICKHOUSE_URL}" \ - --data @- +insert $trace_id 0 +check_span $trace_id '%writeToLocal%' '1' +check_span $trace_id '%writeToLocal%' '2' -# Check log -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " --- Make sure INSERT on distributed finishes -SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry; - --- Make sure opentelemetry span log flushed -SYSTEM FLUSH LOGS; - --- Above INSERT will insert data to two shards respectively, so there will be two spans generated -SELECT count() FROM system.opentelemetry_span_log -WHERE lower(hex(trace_id)) = '${trace_id}' -AND operation_name like '%writeToLocal%' -AND attribute['clickhouse.shard_num'] = '1' -AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' -AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' -AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' -AND attribute['clickhouse.rows'] = '1' -AND attribute['clickhouse.bytes'] = '8' -; - -SELECT count() FROM system.opentelemetry_span_log -WHERE lower(hex(trace_id)) = '${trace_id}' -AND operation_name like '%writeToLocal%' -AND attribute['clickhouse.shard_num'] = '2' -AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' -AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' -AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' -AND attribute['clickhouse.rows'] = '1' -AND attribute['clickhouse.bytes'] = '8' -; - -" # -# INSERT SYNC test -# Do test with opentelemetry enabled and in SYNC mode +# SYNC INSERT SYNC test with opentelemetry enabled # trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); -echo "INSERT INTO ${CLICKHOUSE_DATABASE}.dist_opentelemetry SETTINGS insert_distributed_sync=1 VALUES(1),(2)" | -${CLICKHOUSE_CURL} \ - -X POST \ - -H "traceparent: 00-$trace_id-5250000000000525-01" \ - -H "tracestate: some custom state" \ - "${CLICKHOUSE_URL}" \ - --data @- - -# Check log -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " -SYSTEM FLUSH LOGS; - --- Above INSERT will insert data to two shards in the same flow, so there should be two spans generated with the same operation name -SELECT count() FROM system.opentelemetry_span_log -WHERE lower(hex(trace_id)) = '${trace_id}' -AND operation_name like '%runWritingJob%' -AND attribute['clickhouse.shard_num'] = '1' -AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' -AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' -AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' -AND attribute['clickhouse.rows'] = '1' -AND attribute['clickhouse.bytes'] = '8' -; - -SELECT count() FROM system.opentelemetry_span_log -WHERE lower(hex(trace_id)) = '${trace_id}' -AND operation_name like '%runWritingJob%' -AND attribute['clickhouse.shard_num'] = '2' -AND attribute['clickhouse.cluster'] = 'test_cluster_two_shards' -AND attribute['clickhouse.distributed'] = '${CLICKHOUSE_DATABASE}.dist_opentelemetry' -AND attribute['clickhouse.remote'] = '${CLICKHOUSE_DATABASE}.local_opentelemetry' -AND attribute['clickhouse.rows'] = '1' -AND attribute['clickhouse.bytes'] = '8' -; -" +insert $trace_id 1 +check_span $trace_id '%runWritingJob%' '1' +check_span $trace_id '%runWritingJob%' '2' # # Cleanup # -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -nq " +${CLICKHOUSE_CLIENT} -nq " DROP TABLE ${CLICKHOUSE_DATABASE}.dist_opentelemetry; DROP TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry; " From 43b6f395fb201d0834105d6daaee511ab052e4f5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Sep 2022 06:38:44 +0000 Subject: [PATCH 484/582] Mark the test as long --- tests/queries/0_stateless/02416_keeper_map.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index a87ad6d914b..c191b539de6 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest +-- Tags: no-ordinary-database, no-fasttest, long DROP TABLE IF EXISTS 02416_test SYNC; From 927f763cbbb7a7059e9f06a139bd1ef51a0760f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 9 Sep 2022 10:12:32 +0200 Subject: [PATCH 485/582] tests: disable zero copy replication to suppress warning in 01650_fetch_patition_with_macro_in_zk_path_long Fixes: #41108 Signed-off-by: Azat Khuzhin --- .../01650_fetch_patition_with_macro_in_zk_path_long.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql index 4357aa199dc..1dae8e7b383 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql @@ -5,13 +5,15 @@ DROP TABLE IF EXISTS restore_01640; CREATE TABLE test_01640(i Int64, d Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/{shard}/tables/test_01640','{replica}') -PARTITION BY toYYYYMM(d) ORDER BY i; +PARTITION BY toYYYYMM(d) ORDER BY i +SETTINGS allow_remote_fs_zero_copy_replication=0; insert into test_01640 values (1, '2021-01-01','some'); CREATE TABLE restore_01640(i Int64, d Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/{shard}/tables/restore_01640','{replica}') -PARTITION BY toYYYYMM(d) ORDER BY i; +PARTITION BY toYYYYMM(d) ORDER BY i +SETTINGS allow_remote_fs_zero_copy_replication=0; ALTER TABLE restore_01640 FETCH PARTITION tuple(toYYYYMM(toDate('2021-01-01'))) FROM '/clickhouse/{database}/{shard}/tables/test_01640'; From e3fc78aa5fe50bcf23cf5e24241f4c21da4b381e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Sep 2022 12:14:42 +0200 Subject: [PATCH 486/582] Fix bad warnings in part fetches --- src/Storages/MergeTree/DataPartsExchange.cpp | 8 ++++---- src/Storages/StorageReplicatedMergeTree.cpp | 15 +++++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 1 + 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index ecb692aaf9b..e10881c1eb3 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -422,8 +422,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( const auto data_settings = data.getSettings(); - if (data_settings->allow_remote_fs_zero_copy_replication && !try_zero_copy) - LOG_WARNING(log, "Zero copy replication enabled, but trying to fetch part {} without zero copy", part_name); + if (data.canUseZeroCopyReplication() && !try_zero_copy) + LOG_INFO(log, "Zero copy replication enabled, but trying to fetch part {} without zero copy", part_name); /// It should be "tmp-fetch_" and not "tmp_fetch_", because we can fetch part to detached/, /// but detached part name prefix should not contain underscore. @@ -479,8 +479,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( } else { - if (data_settings->allow_remote_fs_zero_copy_replication) - LOG_WARNING(log, "Cannot select any zero-copy disk for {}", part_name); + if (data.canUseZeroCopyReplication()) + LOG_INFO(log, "Cannot select any zero-copy disk for {}", part_name); try_zero_copy = false; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3491f6c9d4a..46401f08da1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7336,6 +7336,21 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context } +bool StorageReplicatedMergeTree::canUseZeroCopyReplication() const +{ + auto settings_ptr = getSettings(); + if (!settings_ptr->allow_remote_fs_zero_copy_replication) + return false; + + auto disks = getStoragePolicy()->getDisks(); + for (const auto & disk : disks) + { + if (disk->supportZeroCopyReplication()) + return true; + } + return false; +} + void StorageReplicatedMergeTree::checkBrokenDisks() { auto disks = getStoragePolicy()->getDisks(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 79df4f11490..14def28309b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -327,6 +327,7 @@ public: static bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context); + bool canUseZeroCopyReplication() const; private: std::atomic_bool are_restoring_replica {false}; From 2fb0ae7002938720751fe606c3725a84dfbcad88 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 9 Sep 2022 19:02:42 +0800 Subject: [PATCH 487/582] Update test case --- .../02417_opentelemetry_insert_on_distributed_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index 1b4e1da97f9..55457d26249 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -52,7 +52,7 @@ CREATE TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry (key UInt64) Engine=Merg trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); insert $trace_id 0 check_span $trace_id '%writeToLocal%' '1' -check_span $trace_id '%writeToLocal%' '2' +check_span $trace_id '%processFile%' '2' # From c2c35278ba38971c5674767f1bf260ef80903aea Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 6 Sep 2022 13:30:02 +0200 Subject: [PATCH 488/582] Refactor cache for async download --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 207 ++++--- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 4 +- src/Interpreters/Cache/FileCache.cpp | 150 +++-- src/Interpreters/Cache/FileCache.h | 44 +- src/Interpreters/Cache/FileCache_fwd.h | 1 + src/Interpreters/Cache/FileSegment.cpp | 546 ++++++++++-------- src/Interpreters/Cache/FileSegment.h | 200 ++++--- .../tests/gtest_lru_file_cache.cpp | 57 +- 8 files changed, 678 insertions(+), 531 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 21ed465e205..4306cf7ae4d 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -118,7 +118,11 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) } else { - file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, is_persistent)); + CreateFileSegmentSettings create_settings{ + .is_persistent = is_persistent + }; + + file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); } /** @@ -156,7 +160,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(size_t offset) const } CachedOnDiskReadBufferFromFile::ImplementationBufferPtr -CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segment, ReadType read_type_) +CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_) { switch (read_type_) { @@ -178,7 +182,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm * Implementation buffer from segment1 is passed to segment2 once segment1 is loaded. */ - auto remote_fs_segment_reader = file_segment->getRemoteFileReader(); + auto remote_fs_segment_reader = file_segment.getRemoteFileReader(); if (!remote_fs_segment_reader) { @@ -189,7 +193,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm ErrorCodes::CANNOT_USE_CACHE, "Cache cannot be used with a ReadBuffer which does not support right bounded reads"); - file_segment->setRemoteFileReader(remote_fs_segment_reader); + file_segment.setRemoteFileReader(remote_fs_segment_reader); } return remote_fs_segment_reader; @@ -201,8 +205,8 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm if (remote_file_reader && remote_file_reader->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end) return remote_file_reader; - auto remote_fs_segment_reader = file_segment->extractRemoteFileReader(); - if (remote_fs_segment_reader) + auto remote_fs_segment_reader = file_segment.extractRemoteFileReader(); + if (remote_fs_segment_reader && file_offset_of_buffer_end == implementation_buffer->getFileOffsetOfBufferEnd()) remote_file_reader = remote_fs_segment_reader; else remote_file_reader = implementation_buffer_creator(); @@ -217,6 +221,19 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm } } +bool CachedOnDiskReadBufferFromFile::canStartFromCache(size_t current_offset, const FileSegment & file_segment) +{ + /// segment{k} state: DOWNLOADING + /// cache: [______|___________ + /// ^ + /// first_non_downloaded_offset (in progress) + /// requested_range: [__________] + /// ^ + /// current_offset + size_t first_non_downloaded_offset = file_segment.getFirstNonDownloadedOffset(); + return first_non_downloaded_offset > current_offset; +} + CachedOnDiskReadBufferFromFile::ImplementationBufferPtr CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment) { @@ -236,7 +253,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil { LOG_DEBUG(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - return getRemoteFSReadBuffer(file_segment, read_type); + return getRemoteFSReadBuffer(*file_segment, read_type); } } @@ -248,21 +265,16 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil { LOG_DEBUG(log, "Bypassing cache because file segment state is `SKIP_CACHE`"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - return getRemoteFSReadBuffer(file_segment, read_type); + return getRemoteFSReadBuffer(*file_segment, read_type); } case FileSegment::State::DOWNLOADING: { - size_t download_offset = file_segment->getDownloadOffset(); - bool can_start_from_cache = download_offset > file_offset_of_buffer_end; - - /// If file segment is being downloaded but we can already read - /// from already downloaded part, do that. - if (can_start_from_cache) + if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) { /// segment{k} state: DOWNLOADING /// cache: [______|___________ /// ^ - /// download_offset (in progress) + /// first_non_downloaded_offset (in progress) /// requested_range: [__________] /// ^ /// file_offset_of_buffer_end @@ -282,12 +294,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil case FileSegment::State::EMPTY: case FileSegment::State::PARTIALLY_DOWNLOADED: { - if (file_segment->getDownloadOffset() > file_offset_of_buffer_end) + if (file_segment->getFirstNonDownloadedOffset() > file_offset_of_buffer_end) { /// segment{k} state: PARTIALLY_DOWNLOADED /// cache: [______|___________ /// ^ - /// download_offset (in progress) + /// first_non_downloaded_offset (in progress) /// requested_range: [__________] /// ^ /// file_offset_of_buffer_end @@ -299,20 +311,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil auto downloader_id = file_segment->getOrSetDownloader(); if (downloader_id == file_segment->getCallerId()) { - size_t download_offset = file_segment->getDownloadOffset(); - bool can_start_from_cache = download_offset > file_offset_of_buffer_end; - - LOG_TEST( - log, - "Current download offset: {}, file offset of buffer end: {}", - download_offset, file_offset_of_buffer_end); - - if (can_start_from_cache) + if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) { /// segment{k} /// cache: [______|___________ /// ^ - /// download_offset + /// first_non_downloaded_offset /// requested_range: [__________] /// ^ /// file_offset_of_buffer_end @@ -322,27 +326,24 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil return getCacheReadBuffer(range.left); } - if (download_offset < file_offset_of_buffer_end) + if (file_segment->getCurrentWriteOffset() < file_offset_of_buffer_end) { /// segment{1} /// cache: [_____|___________ /// ^ - /// download_offset + /// current_write_offset /// requested_range: [__________] /// ^ /// file_offset_of_buffer_end - assert(file_offset_of_buffer_end > file_segment->getDownloadOffset()); - bytes_to_predownload = file_offset_of_buffer_end - file_segment->getDownloadOffset(); - assert(bytes_to_predownload < range.size()); + LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog()); + chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset()); + bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset(); + chassert(bytes_to_predownload < range.size()); } - download_offset = file_segment->getDownloadOffset(); - can_start_from_cache = download_offset > file_offset_of_buffer_end; - assert(!can_start_from_cache); - read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; - return getRemoteFSReadBuffer(file_segment, read_type); + return getRemoteFSReadBuffer(*file_segment, read_type); } download_state = file_segment->state(); @@ -350,10 +351,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil } case FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { - size_t download_offset = file_segment->getDownloadOffset(); - bool can_start_from_cache = download_offset > file_offset_of_buffer_end; - - if (can_start_from_cache) + if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) { read_type = ReadType::CACHED; return getCacheReadBuffer(range.left); @@ -364,7 +362,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil log, "Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - return getRemoteFSReadBuffer(file_segment, read_type); + return getRemoteFSReadBuffer(*file_segment, read_type); } } } @@ -374,8 +372,8 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil CachedOnDiskReadBufferFromFile::ImplementationBufferPtr CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_segment) { - assert(!file_segment->isDownloader()); - assert(file_offset_of_buffer_end >= file_segment->range().left); + chassert(!file_segment->isDownloader()); + chassert(file_offset_of_buffer_end >= file_segment->range().left); auto range = file_segment->range(); bytes_to_predownload = 0; @@ -389,10 +387,10 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds()); [[maybe_unused]] auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; - assert(download_current_segment == file_segment->isDownloader()); + chassert(download_current_segment == file_segment->isDownloader()); - assert(file_segment->range() == range); - assert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right); + chassert(file_segment->range() == range); + chassert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right); LOG_TEST( log, @@ -441,12 +439,12 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se } case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: { - assert(file_segment->isDownloader()); + chassert(file_segment->isDownloader()); if (bytes_to_predownload) { - size_t download_offset = file_segment->getDownloadOffset(); - read_buffer_for_file_segment->seek(download_offset, SEEK_SET); + size_t current_write_offset = file_segment->getCurrentWriteOffset(); + read_buffer_for_file_segment->seek(current_write_offset, SEEK_SET); } else { @@ -456,18 +454,15 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se assert(static_cast(read_buffer_for_file_segment->getFileOffsetOfBufferEnd()) == file_offset_of_buffer_end); } - auto download_offset = file_segment->getDownloadOffset(); - if (download_offset != static_cast(read_buffer_for_file_segment->getPosition())) + auto current_write_offset = file_segment->getCurrentWriteOffset(); + if (current_write_offset != static_cast(read_buffer_for_file_segment->getPosition())) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Buffer's offsets mismatch; cached buffer offset: {}, download_offset: {}, " - "position: {}, implementation buffer remaining read range: {}, file segment info: {}", - file_offset_of_buffer_end, - download_offset, - read_buffer_for_file_segment->getPosition(), - read_buffer_for_file_segment->getRemainingReadRange().toString(), - file_segment->getInfoForLog()); + "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {} implementation buffer offset: {}, " + "implementation buffer remaining range: {}, file segment info: {}", + file_offset_of_buffer_end, current_write_offset, read_buffer_for_file_segment->getPosition(), + read_buffer_for_file_segment->getRemainingReadRange().toString(), file_segment->getInfoForLog()); } break; @@ -488,7 +483,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() auto & file_segment = *file_segment_it; [[maybe_unused]] const auto & range = file_segment->range(); - assert(file_offset_of_buffer_end > range.right); + chassert(file_offset_of_buffer_end > range.right); LOG_TEST( log, @@ -499,10 +494,8 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() /// Do not hold pointer to file segment if it is not needed anymore /// so can become releasable and can be evicted from cache. - /// If the status of filesegment state is SKIP_CACHE, it will not be deleted. - /// It will be deleted from the cache when the holder is destructed. - if ((*file_segment_it)->state() != FileSegment::State::SKIP_CACHE) - file_segments_holder->file_segments.erase(file_segment_it); + file_segment->completeWithoutState(); + file_segments_holder->file_segments.erase(file_segment_it); if (current_file_segment_it == file_segments_holder->file_segments.end()) return false; @@ -545,8 +538,8 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) /// download from offset a'' < a', but return buffer from offset a'. LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId()); - assert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getDownloadOffset()); - size_t current_offset = file_segment->getDownloadOffset(); + chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getCurrentWriteOffset()); + size_t current_offset = file_segment->getCurrentWriteOffset(); const auto & current_range = file_segment->range(); while (true) @@ -572,7 +565,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) "current download offset: {}, expected: {}, eof: {}", bytes_to_predownload, current_range.toString(), - file_segment->getDownloadOffset(), + file_segment->getCurrentWriteOffset(), file_offset_of_buffer_end, implementation_buffer->eof()); @@ -582,18 +575,20 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) { nextimpl_working_buffer_offset = implementation_buffer->offset(); - auto download_offset = file_segment->getDownloadOffset(); - if (download_offset != static_cast(implementation_buffer->getPosition()) - || download_offset != file_offset_of_buffer_end) + auto current_write_offset = file_segment->getCurrentWriteOffset(); + if (current_write_offset != static_cast(implementation_buffer->getPosition()) + || current_write_offset != file_offset_of_buffer_end) + { throw Exception( ErrorCodes::LOGICAL_ERROR, "Buffer's offsets mismatch after predownloading; download offset: {}, " "cached buffer offset: {}, implementation buffer offset: {}, " "file segment info: {}", - download_offset, + current_write_offset, file_offset_of_buffer_end, implementation_buffer->getPosition(), file_segment->getInfoForLog()); + } } break; @@ -609,7 +604,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) { LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); - assert(file_segment->getDownloadOffset() == static_cast(implementation_buffer->getPosition())); + chassert(file_segment->getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, *file_segment); if (success) @@ -635,7 +630,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) /// segment{1} /// cache: [_____|___________ /// ^ - /// download_offset + /// current_write_offset /// requested_range: [__________] /// ^ /// file_offset_of_buffer_end @@ -649,17 +644,18 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) bytes_to_predownload = 0; file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - LOG_TEST(log, "Bypassing cache because space reservation failed"); + LOG_TEST(log, "Bypassing cache because for {}", file_segment->getInfoForLog()); + read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; swap(*implementation_buffer); resetWorkingBuffer(); - implementation_buffer = getRemoteFSReadBuffer(file_segment, read_type); + implementation_buffer = getRemoteFSReadBuffer(*file_segment, read_type); swap(*implementation_buffer); - implementation_buffer->setReadUntilPosition(current_range.right + 1); /// [..., range.right] + implementation_buffer->setReadUntilPosition(file_segment->range().right + 1); /// [..., range.right] implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET); LOG_TEST( @@ -680,8 +676,8 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() auto current_read_range = file_segment->range(); auto current_state = file_segment->state(); - assert(current_read_range.left <= file_offset_of_buffer_end); - assert(!file_segment->isDownloader()); + chassert(current_read_range.left <= file_offset_of_buffer_end); + chassert(!file_segment->isDownloader()); if (file_offset_of_buffer_end > current_read_range.right) { @@ -695,13 +691,15 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() /// segment{k} /// cache: [______|___________ /// ^ - /// download_offset + /// current_write_offset /// requested_range: [__________] /// ^ /// file_offset_of_buffer_end - size_t download_offset = file_segment->getDownloadOffset(); - bool cached_part_is_finished = download_offset == file_offset_of_buffer_end; + auto current_write_offset = file_segment->getCurrentWriteOffset(); + bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end; + + LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end); if (cached_part_is_finished) { @@ -710,12 +708,12 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() return true; } - else if (download_offset < file_offset_of_buffer_end) + else if (current_write_offset < file_offset_of_buffer_end) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Expected {} >= {} ({})", - download_offset, file_offset_of_buffer_end, getInfoForLog()); + current_write_offset, file_offset_of_buffer_end, getInfoForLog()); } } @@ -725,7 +723,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() * ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE means that on previous getImplementationBuffer() call * current buffer successfully called file_segment->getOrSetDownloader() and became a downloader * for this file segment. However, the downloader's term has a lifespan of 1 nextImpl() call, - * e.g. downloader reads buffer_size byte and calls completeBatchAndResetDownloader() and some other + * e.g. downloader reads buffer_size byte and calls completePartAndResetDownloader() and some other * thread can become a downloader if it calls getOrSetDownloader() faster. * * So downloader is committed to download only buffer_size bytes and then is not a downloader anymore, @@ -817,11 +815,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (need_complete_file_segment) { LOG_TEST(log, "Resetting downloader {} from scope exit", file_segment->getDownloader()); - file_segment->completeBatchAndResetDownloader(); + file_segment->completePartAndResetDownloader(); } } - assert(!file_segment->isDownloader()); + chassert(!file_segment->isDownloader()); } catch (...) { @@ -845,7 +843,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() (*current_file_segment_it)->incrementHitsCount(); } - assert(!internal_buffer.empty()); + chassert(!internal_buffer.empty()); swap(*implementation_buffer); @@ -854,15 +852,14 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TEST( log, - "Current segment: {}, downloader: {}, current count: {}, position: {}, read range: {}", - current_read_range.toString(), - file_segment->getDownloader(), + "Current count: {}, position: {}, read range: {}, file segment: {}", implementation_buffer->count(), implementation_buffer->getPosition(), - implementation_buffer->getRemainingReadRange().toString()); + implementation_buffer->getRemainingReadRange().toString(), + file_segment->getInfoForLog()); - assert(current_read_range.left <= file_offset_of_buffer_end); - assert(current_read_range.right >= file_offset_of_buffer_end); + chassert(current_read_range.left <= file_offset_of_buffer_end); + chassert(current_read_range.right >= file_offset_of_buffer_end); bool result = false; size_t size = 0; @@ -939,24 +936,26 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { if (download_current_segment) { - assert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right); + chassert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right); bool success = file_segment->reserve(size); if (success) { - assert(file_segment->getDownloadOffset() == static_cast(implementation_buffer->getPosition())); + chassert(file_segment->getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, *file_segment); if (success) { - assert(file_segment->getDownloadOffset() <= file_segment->range().right + 1); - assert( + chassert(file_segment->getCurrentWriteOffset() <= file_segment->range().right + 1); + chassert( std::next(current_file_segment_it) == file_segments_holder->file_segments.end() - || file_segment->getDownloadOffset() == implementation_buffer->getFileOffsetOfBufferEnd()); + || file_segment->getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd()); + + LOG_TEST(log, "Successfully written {} bytes", size); } else { - assert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + chassert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); LOG_TEST(log, "Bypassing cache because writeCache method failed"); } } @@ -984,7 +983,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; size = std::min(size, remaining_size_to_read); - assert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); + chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); } @@ -996,15 +995,15 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() current_file_segment_counters.increment(ProfileEvents::FileSegmentUsedBytes, available()); if (download_current_segment) - file_segment->completeBatchAndResetDownloader(); + file_segment->completePartAndResetDownloader(); - assert(!file_segment->isDownloader()); + chassert(!file_segment->isDownloader()); LOG_TEST( log, "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), " "buffer available: {}, current range: {}, current offset: {}, file segment state: {}, " - "download offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " + "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " "remaining ranges: {}", getHexUIntLowercase(cache_key), working_buffer.size(), @@ -1015,7 +1014,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() current_read_range.toString(), file_offset_of_buffer_end, FileSegment::stateToString(file_segment->state()), - file_segment->getDownloadOffset(), + file_segment->getCurrentWriteOffset(), toString(read_type), read_until_position, first_offset, diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index ed623272c12..b86e53ec160 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -80,7 +80,7 @@ private: void assertCorrectness() const; - std::shared_ptr getRemoteFSReadBuffer(FileSegmentPtr & file_segment, ReadType read_type_); + std::shared_ptr getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_); size_t getTotalSizeToRead(); @@ -90,6 +90,8 @@ private: bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment); + static bool canStartFromCache(size_t current_offset, const FileSegment & file_segment); + Poco::Logger * log; FileCache::Key cache_key; String source_file_path; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index f51df9ae737..20a9f6cce1d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -122,7 +122,6 @@ void FileCache::initialize() fs::create_directories(cache_base_path); } - status_file = make_unique(fs::path(cache_base_path) / "status", StatusFile::write_full_info); is_initialized = true; } } @@ -258,7 +257,7 @@ FileSegments FileCache::splitRangeIntoCells( size_t offset, size_t size, FileSegment::State state, - bool is_persistent, + const CreateFileSegmentSettings & settings, std::lock_guard & cache_lock) { assert(size > 0); @@ -275,7 +274,7 @@ FileSegments FileCache::splitRangeIntoCells( current_cell_size = std::min(remaining_size, max_file_segment_size); remaining_size -= current_cell_size; - auto * cell = addCell(key, current_pos, current_cell_size, state, is_persistent, cache_lock); + auto * cell = addCell(key, current_pos, current_cell_size, state, settings, cache_lock); if (cell) file_segments.push_back(cell->file_segment); assert(cell); @@ -292,7 +291,7 @@ void FileCache::fillHolesWithEmptyFileSegments( const Key & key, const FileSegment::Range & range, bool fill_with_detached_file_segments, - bool is_persistent, + const CreateFileSegmentSettings & settings, std::lock_guard & cache_lock) { /// There are segments [segment1, ..., segmentN] @@ -339,16 +338,16 @@ void FileCache::fillHolesWithEmptyFileSegments( if (fill_with_detached_file_segments) { - auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY); + auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings); { - std::lock_guard segment_lock(file_segment->mutex); - file_segment->markAsDetached(segment_lock); + std::unique_lock segment_lock(file_segment->mutex); + file_segment->detachAssumeStateFinalized(segment_lock); } file_segments.insert(it, file_segment); } else { - file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, is_persistent, cache_lock)); + file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock)); } current_pos = segment_range.right + 1; @@ -366,22 +365,23 @@ void FileCache::fillHolesWithEmptyFileSegments( if (fill_with_detached_file_segments) { - auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY); + auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings); { - std::lock_guard segment_lock(file_segment->mutex); - file_segment->markAsDetached(segment_lock); + std::unique_lock segment_lock(file_segment->mutex); + file_segment->detachAssumeStateFinalized(segment_lock); } file_segments.insert(file_segments.end(), file_segment); } else { file_segments.splice( - file_segments.end(), splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, is_persistent, cache_lock)); + file_segments.end(), + splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock)); } } } -FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, bool is_persistent) +FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings) { std::lock_guard cache_lock(mutex); @@ -398,11 +398,11 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si if (file_segments.empty()) { - file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, is_persistent, cache_lock); + file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock); } else { - fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, is_persistent, cache_lock); + fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock); } assert(!file_segments.empty()); @@ -426,16 +426,17 @@ FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size) if (file_segments.empty()) { - auto file_segment = std::make_shared(offset, size, key, this, FileSegment::State::EMPTY); + auto file_segment = std::make_shared( + offset, size, key, this, FileSegment::State::EMPTY, CreateFileSegmentSettings{}); { - std::lock_guard segment_lock(file_segment->mutex); - file_segment->markAsDetached(segment_lock); + std::unique_lock segment_lock(file_segment->mutex); + file_segment->detachAssumeStateFinalized(segment_lock); } file_segments = { file_segment }; } else { - fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, /* is_persistent */false, cache_lock); + fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, {}, cache_lock); } return FileSegmentsHolder(std::move(file_segments)); @@ -443,7 +444,7 @@ FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size) FileCache::FileSegmentCell * FileCache::addCell( const Key & key, size_t offset, size_t size, - FileSegment::State state, bool is_persistent, + FileSegment::State state, const CreateFileSegmentSettings & settings, std::lock_guard & cache_lock) { /// Create a file segment cell and put it in `files` map by [key][offset]. @@ -475,18 +476,23 @@ FileCache::FileSegmentCell * FileCache::addCell( stash_records.erase({remove_priority_iter->key(), remove_priority_iter->offset()}); remove_priority_iter->removeAndGetNext(cache_lock); } - /// For segments that do not reach the download threshold, we do not download them, but directly read them + + /// For segments that do not reach the download threshold, + /// we do not download them, but directly read them result_state = FileSegment::State::SKIP_CACHE; } else { auto priority_iter = record->second; priority_iter->use(cache_lock); - result_state = priority_iter->hits() >= enable_cache_hits_threshold ? FileSegment::State::EMPTY : FileSegment::State::SKIP_CACHE; + + result_state = priority_iter->hits() >= enable_cache_hits_threshold + ? FileSegment::State::EMPTY + : FileSegment::State::SKIP_CACHE; } } - return std::make_shared(offset, size, key, this, result_state, is_persistent); + return std::make_shared(offset, size, key, this, result_state, settings); }; FileSegmentCell cell(skip_or_download(), this, cache_lock); @@ -495,6 +501,7 @@ FileCache::FileSegmentCell * FileCache::addCell( if (offsets.empty()) { auto key_path = getPathInLocalCache(key); + if (!fs::exists(key_path)) fs::create_directories(key_path); } @@ -513,7 +520,7 @@ FileSegmentPtr FileCache::createFileSegmentForDownload( const Key & key, size_t offset, size_t size, - bool is_persistent, + const CreateFileSegmentSettings & settings, std::lock_guard & cache_lock) { #ifndef NDEBUG @@ -530,7 +537,7 @@ FileSegmentPtr FileCache::createFileSegmentForDownload( "Cache cell already exists for key `{}` and offset {}", key.toString(), offset); - cell = addCell(key, offset, size, FileSegment::State::EMPTY, is_persistent, cache_lock); + cell = addCell(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock); if (!cell) throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to add a new cell for download"); @@ -542,18 +549,21 @@ bool FileCache::tryReserve(const Key & key, size_t offset, size_t size, std::loc { auto query_context = enable_filesystem_query_cache_limit ? getCurrentQueryContext(cache_lock) : nullptr; if (!query_context) + { return tryReserveForMainList(key, offset, size, nullptr, cache_lock); - + } /// The maximum cache capacity of the request is not reached, thus the //// cache block is evicted from the main LRU queue by tryReserveForMainList(). else if (query_context->getCacheSize() + size <= query_context->getMaxCacheSize()) + { return tryReserveForMainList(key, offset, size, query_context, cache_lock); - + } /// When skip_download_if_exceeds_query_cache is true, there is no need /// to evict old data, skip the cache and read directly from remote fs. else if (query_context->isSkipDownloadIfExceed()) + { return false; - + } /// The maximum cache size of the query is reached, the cache will be /// evicted from the history cache accessed by the current query. else @@ -833,7 +843,7 @@ void FileCache::removeIfExists(const Key & key) auto file_segment = cell->file_segment; if (file_segment) { - std::lock_guard segment_lock(file_segment->mutex); + std::unique_lock segment_lock(file_segment->mutex); file_segment->detach(cache_lock, segment_lock); remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); } @@ -863,9 +873,11 @@ void FileCache::removeIfReleasable() auto * cell = getCell(key, offset, cache_lock); if (!cell) + { throw Exception( ErrorCodes::LOGICAL_ERROR, "Cache is in inconsistent state: LRU queue contains entries with no cache cell"); + } if (cell->releasable()) { @@ -880,7 +892,7 @@ void FileCache::removeIfReleasable() for (auto & file_segment : to_remove) { - std::lock_guard segment_lock(file_segment->mutex); + std::unique_lock segment_lock(file_segment->mutex); file_segment->detach(cache_lock, segment_lock); remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); } @@ -896,13 +908,13 @@ void FileCache::removeIfReleasable() void FileCache::remove(FileSegmentPtr file_segment, std::lock_guard & cache_lock) { - std::lock_guard segment_lock(file_segment->mutex); + std::unique_lock segment_lock(file_segment->mutex); remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); } void FileCache::remove( Key key, size_t offset, - std::lock_guard & cache_lock, std::lock_guard & /* segment_lock */) + std::lock_guard & cache_lock, std::unique_lock & /* segment_lock */) { LOG_DEBUG(log, "Remove from cache. Key: {}, offset: {}", key.toString(), offset); @@ -976,7 +988,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock { if (!key_it->is_directory()) { - LOG_DEBUG(log, "Unexpected file {} (not a directory), will skip it", key_it->path().string()); + LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); continue; } @@ -1012,7 +1024,10 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock if (tryReserve(key, offset, size, cache_lock)) { - auto * cell = addCell(key, offset, size, FileSegment::State::DOWNLOADED, is_persistent, cache_lock); + auto * cell = addCell( + key, offset, size, FileSegment::State::DOWNLOADED, + CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock); + if (cell) queue_entries.emplace_back(cell->queue_iterator, cell->file_segment); } @@ -1049,7 +1064,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock void FileCache::reduceSizeToDownloaded( const Key & key, size_t offset, - std::lock_guard & cache_lock, std::lock_guard & /* segment_lock */) + std::lock_guard & cache_lock, std::unique_lock & segment_lock) { /** * In case file was partially downloaded and it's download cannot be continued @@ -1069,20 +1084,25 @@ void FileCache::reduceSizeToDownloaded( const auto & file_segment = cell->file_segment; size_t downloaded_size = file_segment->downloaded_size; - if (downloaded_size == file_segment->range().size()) + size_t full_size = file_segment->range().size(); + + if (downloaded_size == full_size) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Nothing to reduce, file segment fully downloaded, key: {}, offset: {}", - key.toString(), offset); + "Nothing to reduce, file segment fully downloaded: {}", + file_segment->getInfoForLogUnlocked(segment_lock)); } - cell->file_segment = std::make_shared(offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED); + cell->file_segment = std::make_shared( + offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings{}); + + assert(file_segment->reserved_size == downloaded_size); } bool FileCache::isLastFileSegmentHolder( const Key & key, size_t offset, - std::lock_guard & cache_lock, std::lock_guard & /* segment_lock */) + std::lock_guard & cache_lock, std::unique_lock & /* segment_lock */) { auto * cell = getCell(key, offset, cache_lock); @@ -1167,7 +1187,8 @@ FileCache::FileSegmentCell::FileSegmentCell( { case FileSegment::State::DOWNLOADED: { - queue_iterator = cache->main_priority->add(file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock); + queue_iterator = cache->main_priority->add( + file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock); break; } case FileSegment::State::SKIP_CACHE: @@ -1246,14 +1267,41 @@ void FileCache::assertPriorityCorrectness(std::lock_guard & cache_lo ErrorCodes::LOGICAL_ERROR, "Cache is in inconsistent state: LRU queue contains entries with no cache cell (assertCorrectness())"); } - assert(cell->size() == size); + + if (cell->size() != size) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected {} == {} size ({})", + cell->size(), size, cell->file_segment->getInfoForLog()); + } + total_size += size; } + assert(total_size == main_priority->getCacheSize(cache_lock)); assert(main_priority->getCacheSize(cache_lock) <= max_size); assert(main_priority->getElementsNum(cache_lock) <= max_element_size); } +FileCache::QueryContextHolder::QueryContextHolder( + const String & query_id_, + FileCache * cache_, + FileCache::QueryContextPtr context_) + : query_id(query_id_) + , cache(cache_) + , context(context_) +{ +} + +FileCache::QueryContextHolder::~QueryContextHolder() +{ + /// If only the query_map and the current holder hold the context_query, + /// the query has been completed and the query_context is released. + if (context && context.use_count() == 2) + cache->removeQueryContext(query_id); +} + FileCache::QueryContextPtr FileCache::getCurrentQueryContext(std::lock_guard & cache_lock) { if (!isQueryInitialized()) @@ -1362,22 +1410,4 @@ void FileCache::QueryContext::use(const Key & key, size_t offset, std::lock_guar record->second->use(cache_lock); } -FileCache::QueryContextHolder::QueryContextHolder( - const String & query_id_, - FileCache * cache_, - FileCache::QueryContextPtr context_) - : query_id(query_id_) - , cache(cache_) - , context(context_) -{ -} - -FileCache::QueryContextHolder::~QueryContextHolder() -{ - /// If only the query_map and the current holder hold the context_query, - /// the query has been completed and the query_context is released. - if (context && context.use_count() == 2) - cache->removeQueryContext(query_id); -} - } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 3f5a5c9e1c5..07aea230803 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -12,13 +12,14 @@ #include #include +#include +#include #include +#include +#include #include #include -#include -#include -#include -#include + namespace DB { @@ -43,7 +44,6 @@ public: ~FileCache() = default; - /// Restore cache from local filesystem. void initialize(); const String & getBasePath() const { return cache_base_path; } @@ -59,7 +59,7 @@ public: * As long as pointers to returned file segments are hold * it is guaranteed that these file segments are not removed from cache. */ - FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, bool is_persistent); + FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); /** * Segments in returned list are ordered in ascending order and represent a full contiguous @@ -104,7 +104,7 @@ public: const Key & key, size_t offset, size_t size, - bool is_persistent, + const CreateFileSegmentSettings & create_settings, std::lock_guard & cache_lock); FileSegments getSnapshot() const; @@ -132,21 +132,21 @@ public: private: String cache_base_path; - size_t max_size; - size_t max_element_size; - size_t max_file_segment_size; + const size_t max_size; + const size_t max_element_size; + const size_t max_file_segment_size; - bool allow_persistent_files; - size_t enable_cache_hits_threshold; - bool enable_filesystem_query_cache_limit; + const bool allow_persistent_files; + const size_t enable_cache_hits_threshold; + const bool enable_filesystem_query_cache_limit; + mutable std::mutex mutex; Poco::Logger * log; bool is_initialized = false; std::exception_ptr initialization_exception; - std::unique_ptr status_file; - mutable std::mutex mutex; + void assertInitialized(std::lock_guard & cache_lock) const; bool tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock); @@ -154,7 +154,7 @@ private: Key key, size_t offset, std::lock_guard & cache_lock, - std::lock_guard & segment_lock); + std::unique_lock & segment_lock); void remove( FileSegmentPtr file_segment, @@ -164,15 +164,13 @@ private: const Key & key, size_t offset, std::lock_guard & cache_lock, - std::lock_guard & segment_lock); + std::unique_lock & segment_lock); void reduceSizeToDownloaded( const Key & key, size_t offset, std::lock_guard & cache_lock, - std::lock_guard & segment_lock); - - void assertInitialized(std::lock_guard & cache_lock) const; + std::unique_lock & segment_lock); struct FileSegmentCell : private boost::noncopyable { @@ -225,7 +223,7 @@ private: size_t offset, size_t size, FileSegment::State state, - bool is_persistent, + const CreateFileSegmentSettings & create_settings, std::lock_guard & cache_lock); static void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard & cache_lock); @@ -242,7 +240,7 @@ private: size_t offset, size_t size, FileSegment::State state, - bool is_persistent, + const CreateFileSegmentSettings & create_settings, std::lock_guard & cache_lock); String dumpStructureUnlocked(const Key & key_, std::lock_guard & cache_lock); @@ -252,7 +250,7 @@ private: const Key & key, const FileSegment::Range & range, bool fill_with_detached_file_segments, - bool is_persistent, + const CreateFileSegmentSettings & settings, std::lock_guard & cache_lock); size_t getUsedCacheSizeUnlocked(std::lock_guard & cache_lock) const; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 13f037783b0..25c16b4e840 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -12,5 +12,6 @@ class FileCache; using FileCachePtr = std::shared_ptr; struct FileCacheSettings; +struct CreateFileSegmentSettings; } diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 547e6849dd6..8a67dfc1dc6 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1,10 +1,10 @@ #include "FileSegment.h" #include -#include +#include +#include #include #include -#include #include #include #include @@ -29,7 +29,7 @@ FileSegment::FileSegment( const Key & key_, FileCache * cache_, State download_state_, - bool is_persistent_) + const CreateFileSegmentSettings & settings) : segment_range(offset_, offset_ + size_ - 1) , download_state(download_state_) , file_key(key_) @@ -39,7 +39,7 @@ FileSegment::FileSegment( #else , log(&Poco::Logger::get("FileSegment")) #endif - , is_persistent(is_persistent_) + , is_persistent(settings.is_persistent) { /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. switch (download_state) @@ -64,50 +64,86 @@ FileSegment::FileSegment( } default: { - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state"); + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state"); } } } +String FileSegment::getPathInLocalCache() const +{ + return cache->getPathInLocalCache(key(), offset(), isPersistent()); +} + FileSegment::State FileSegment::state() const { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); return download_state; } -size_t FileSegment::getDownloadOffset() const +void FileSegment::setDownloadState(State state) +{ + LOG_TEST(log, "Updated state from {} to {}", stateToString(download_state), stateToString(state)); + download_state = state; +} + +size_t FileSegment::getFirstNonDownloadedOffset() const +{ + std::unique_lock segment_lock(mutex); + return getFirstNonDownloadedOffsetUnlocked(segment_lock); +} + +size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock & segment_lock) const { - std::lock_guard segment_lock(mutex); return range().left + getDownloadedSizeUnlocked(segment_lock); } +size_t FileSegment::getCurrentWriteOffset() const +{ + std::unique_lock segment_lock(mutex); + return getCurrentWriteOffsetUnlocked(segment_lock); +} + +size_t FileSegment::getCurrentWriteOffsetUnlocked(std::unique_lock & segment_lock) const +{ + return getFirstNonDownloadedOffsetUnlocked(segment_lock); +} + size_t FileSegment::getDownloadedSize() const { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); return getDownloadedSizeUnlocked(segment_lock); } -size_t FileSegment::getRemainingSizeToDownload() const -{ - std::lock_guard segment_lock(mutex); - return range().size() - getDownloadedSizeUnlocked(segment_lock); -} - -bool FileSegment::isDetached() const -{ - std::lock_guard segment_lock(mutex); - return is_detached; -} - -size_t FileSegment::getDownloadedSizeUnlocked(std::lock_guard & /* segment_lock */) const +size_t FileSegment::getDownloadedSizeUnlocked(std::unique_lock & /* segment_lock */) const { if (download_state == State::DOWNLOADED) return downloaded_size; - std::lock_guard download_lock(download_mutex); + std::unique_lock download_lock(download_mutex); return downloaded_size; } +size_t FileSegment::getAvailableSizeUnlocked(std::unique_lock & segment_lock) const +{ + auto current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + chassert(reserved_size >= current_downloaded_size); + return reserved_size - current_downloaded_size; +} + +size_t FileSegment::getRemainingSizeToDownload() const +{ + std::unique_lock segment_lock(mutex); + return range().size() - getDownloadedSizeUnlocked(segment_lock); +} + +bool FileSegment::isDownloaded() const +{ + std::lock_guard segment_lock(mutex); + return is_downloaded; +} + String FileSegment::getCallerId() { if (!CurrentThread::isInitialized() @@ -118,84 +154,108 @@ String FileSegment::getCallerId() return std::string(CurrentThread::getQueryId()) + ":" + toString(getThreadId()); } +String FileSegment::getDownloader() const +{ + std::unique_lock segment_lock(mutex); + return getDownloaderUnlocked(segment_lock); +} + +String FileSegment::getDownloaderUnlocked(std::unique_lock & /* segment_lock */) const +{ + return downloader_id; +} + String FileSegment::getOrSetDownloader() { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); - assertNotDetached(segment_lock); + assertNotDetachedUnlocked(segment_lock); - if (downloader_id.empty()) + auto current_downloader = getDownloaderUnlocked(segment_lock); + + if (current_downloader.empty()) { - assert(download_state != State::DOWNLOADING); - - if (download_state != State::EMPTY - && download_state != State::PARTIALLY_DOWNLOADED) + bool allow_new_downloader = download_state == State::EMPTY || download_state == State::PARTIALLY_DOWNLOADED; + if (!allow_new_downloader) return "None"; - downloader_id = getCallerId(); - download_state = State::DOWNLOADING; - } - else if (downloader_id == getCallerId()) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to set the same downloader for segment {} for the second time", range().toString()); + chassert(download_state != State::DOWNLOADING); - return downloader_id; + current_downloader = downloader_id = getCallerId(); + setDownloadState(State::DOWNLOADING); + } + + return current_downloader; +} + +void FileSegment::resetDownloadingStateUnlocked([[maybe_unused]] std::unique_lock & segment_lock) +{ + assert(isDownloaderUnlocked(segment_lock)); + assert(download_state == State::DOWNLOADING); + + size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + /// range().size() can equal 0 in case of write-though cache. + if (current_downloaded_size != 0 && current_downloaded_size == range().size()) + setDownloadedUnlocked(segment_lock); + else + setDownloadState(State::PARTIALLY_DOWNLOADED); } void FileSegment::resetDownloader() { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); - assertNotDetached(segment_lock); + assertNotDetachedUnlocked(segment_lock); + assertIsDownloaderUnlocked("resetDownloader", segment_lock); - if (downloader_id.empty()) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "There is no downloader"); - - if (getCallerId() != downloader_id) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Downloader can be reset only by downloader"); - - resetDownloaderImpl(segment_lock); + resetDownloadingStateUnlocked(segment_lock); + resetDownloaderUnlocked(segment_lock); } -void FileSegment::resetDownloaderImpl(std::lock_guard & segment_lock) +void FileSegment::resetDownloaderUnlocked(std::unique_lock & /* segment_lock */) { - if (getDownloadedSizeUnlocked(segment_lock) == range().size()) - setDownloaded(segment_lock); - else - download_state = State::PARTIALLY_DOWNLOADED; - + LOG_TEST(log, "Resetting downloader from {}", downloader_id); downloader_id.clear(); } -String FileSegment::getDownloader() const +void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const { - std::lock_guard segment_lock(mutex); - return downloader_id; + auto caller = getCallerId(); + auto current_downloader = getDownloaderUnlocked(segment_lock); + LOG_TEST(log, "Downloader id: {}, caller id: {}", current_downloader, caller); + + if (caller != current_downloader) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Operation `{}` can be done only by downloader. " + "(CallerId: {}, downloader id: {})", + operation, caller, downloader_id); + } } bool FileSegment::isDownloader() const { - std::lock_guard segment_lock(mutex); - return getCallerId() == downloader_id; + std::unique_lock segment_lock(mutex); + return isDownloaderUnlocked(segment_lock); } -bool FileSegment::isDownloaderImpl(std::lock_guard & /* segment_lock */) const +bool FileSegment::isDownloaderUnlocked(std::unique_lock & segment_lock) const { - return getCallerId() == downloader_id; + return getCallerId() == getDownloaderUnlocked(segment_lock); } FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() { - if (!isDownloader()) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader"); - + std::unique_lock segment_lock(mutex); + assertIsDownloaderUnlocked("getRemoteFileReader", segment_lock); return remote_file_reader; } FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { std::lock_guard cache_lock(cache->mutex); - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); if (!is_detached) { @@ -210,8 +270,8 @@ FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) { - if (!isDownloader()) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader"); + std::unique_lock segment_lock(mutex); + assertIsDownloaderUnlocked("setRemoteFileReader", segment_lock); if (remote_file_reader) throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader already exists"); @@ -221,8 +281,8 @@ void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) void FileSegment::resetRemoteFileReader() { - if (!isDownloader()) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader"); + std::unique_lock segment_lock(mutex); + assertIsDownloaderUnlocked("resetRemoteFileReader", segment_lock); if (!remote_file_reader) throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader does not exist"); @@ -230,45 +290,47 @@ void FileSegment::resetRemoteFileReader() remote_file_reader.reset(); } -void FileSegment::write(const char * from, size_t size, size_t offset_) +void FileSegment::write(const char * from, size_t size, size_t offset) { if (!size) throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed"); - if (availableSize() < size) - throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Not enough space is reserved. Available: {}, expected: {}", availableSize(), size); - - if (!isDownloader()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Only downloader can do the downloading. (CallerId: {}, DownloaderId: {})", - getCallerId(), downloader_id); - - if (getDownloadedSize() == range().size()) - throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to write {} bytes to offset: {}, but current file segment is already fully downloaded", - size, offset_); - - auto download_offset = range().left + downloaded_size; - if (offset_ != download_offset) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to write {} bytes to offset: {}, but current download offset is {}", - size, offset_, download_offset); - { - std::lock_guard segment_lock(mutex); - assertNotDetached(segment_lock); + std::unique_lock segment_lock(mutex); + + assertIsDownloaderUnlocked("write", segment_lock); + assertNotDetachedUnlocked(segment_lock); + + if (download_state != State::DOWNLOADING) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected DOWNLOADING state, got {}", stateToString(download_state)); + + size_t first_non_downloaded_offset = getFirstNonDownloadedOffsetUnlocked(segment_lock); + if (offset != first_non_downloaded_offset) + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Attempt to write {} bytes to offset: {}, but current write offset is {}", + size, offset, first_non_downloaded_offset); + + size_t available_size = getAvailableSizeUnlocked(segment_lock); + if (available_size < size) + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Not enough space is reserved. Available: {}, expected: {}", available_size, size); + + if (getDownloadedSizeUnlocked(segment_lock) == range().size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded"); } if (!cache_writer) { - if (downloaded_size > 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cache writer was finalized (downloaded size: {}, state: {})", - downloaded_size, stateToString(download_state)); + auto current_downloaded_size = getDownloadedSize(); + if (current_downloaded_size > 0) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cache writer was finalized (downloaded size: {}, state: {})", + current_downloaded_size, stateToString(download_state)); auto download_path = getPathInLocalCache(); cache_writer = std::make_unique(download_path); @@ -278,7 +340,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset_) { cache_writer->write(from, size); - std::lock_guard download_lock(download_mutex); + std::unique_lock download_lock(download_mutex); cache_writer->next(); @@ -286,23 +348,21 @@ void FileSegment::write(const char * from, size_t size, size_t offset_) } catch (Exception & e) { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); wrapWithCacheInfo(e, "while writing into cache", segment_lock); - setDownloadFailed(segment_lock); + setDownloadFailedUnlocked(segment_lock); cv.notify_all(); throw; } - assert(getDownloadOffset() == offset_ + size); -} - -String FileSegment::getPathInLocalCache() const -{ - return cache->getPathInLocalCache(key(), offset(), isPersistent()); +#ifndef NDEBUG + std::unique_lock segment_lock(mutex); + chassert(getFirstNonDownloadedOffsetUnlocked(segment_lock) == offset + size); +#endif } FileSegment::State FileSegment::wait() @@ -324,8 +384,8 @@ FileSegment::State FileSegment::wait() { LOG_TEST(log, "{} waiting on: {}, current downloader: {}", getCallerId(), range().toString(), downloader_id); - assert(!downloader_id.empty()); - assert(downloader_id != getCallerId()); + chassert(!getDownloaderUnlocked(segment_lock).empty()); + chassert(!isDownloaderUnlocked(segment_lock)); cv.wait_for(segment_lock, std::chrono::seconds(60)); } @@ -338,30 +398,23 @@ bool FileSegment::reserve(size_t size_to_reserve) if (!size_to_reserve) throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed"); + size_t expected_downloaded_size; + { - std::lock_guard segment_lock(mutex); - assertNotDetached(segment_lock); + std::unique_lock segment_lock(mutex); - auto caller_id = getCallerId(); - bool is_downloader = caller_id == downloader_id; - if (!is_downloader) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Space can be reserved only by downloader (current: {}, expected: {})", - caller_id, downloader_id); - } + assertNotDetachedUnlocked(segment_lock); + assertIsDownloaderUnlocked("reserve", segment_lock); - size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); - if (current_downloaded_size + size_to_reserve > range().size()) - { + expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + + if (expected_downloaded_size + size_to_reserve > range().size()) throw Exception( ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to reserve space too much space: {} ({})", - size_to_reserve, getInfoForLogImpl(segment_lock)); - } + "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", + size_to_reserve, range().toString(), downloaded_size); - assert(reserved_size >= current_downloaded_size); + chassert(reserved_size >= expected_downloaded_size); } /** @@ -370,9 +423,7 @@ bool FileSegment::reserve(size_t size_to_reserve) * and the caller is going to continue; */ - size_t current_downloaded_size = getDownloadedSize(); - assert(reserved_size >= current_downloaded_size); - size_t already_reserved_size = reserved_size - current_downloaded_size; + size_t already_reserved_size = reserved_size - expected_downloaded_size; bool reserved = already_reserved_size >= size_to_reserve; if (!reserved) @@ -392,23 +443,13 @@ bool FileSegment::reserve(size_t size_to_reserve) return reserved; } -bool FileSegment::isDownloaded() const -{ - std::lock_guard segment_lock(mutex); - return isDownloadedUnlocked(segment_lock); -} - -bool FileSegment::isDownloadedUnlocked(std::lock_guard & /* segment_lock */) const -{ - return is_downloaded; -} - -void FileSegment::setDownloaded([[maybe_unused]] std::lock_guard & segment_lock) +void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock & segment_lock) { if (is_downloaded) return; - downloader_id.clear(); + setDownloadState(State::DOWNLOADED); + is_downloaded = true; if (cache_writer) { @@ -424,10 +465,12 @@ void FileSegment::setDownloaded([[maybe_unused]] std::lock_guard & s assert(std::filesystem::file_size(getPathInLocalCache()) > 0); } -void FileSegment::setDownloadFailed(std::lock_guard & /* segment_lock */) +void FileSegment::setDownloadFailedUnlocked(std::unique_lock & segment_lock) { - download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; - downloader_id.clear(); + LOG_INFO(log, "Settings download as failed: {}", getInfoForLogUnlocked(segment_lock)); + + setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + resetDownloaderUnlocked(segment_lock); if (cache_writer) { @@ -437,43 +480,31 @@ void FileSegment::setDownloadFailed(std::lock_guard & /* segment_loc } } -void FileSegment::completeBatchAndResetDownloader() +void FileSegment::completePartAndResetDownloader() { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); + completePartAndResetDownloaderUnlocked(segment_lock); +} - assertNotDetached(segment_lock); +void FileSegment::completePartAndResetDownloaderUnlocked(std::unique_lock & segment_lock) +{ + assertNotDetachedUnlocked(segment_lock); + assertIsDownloaderUnlocked("completePartAndResetDownloader", segment_lock); - if (!isDownloaderImpl(segment_lock)) - { - cv.notify_all(); - throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "File segment can be completed only by downloader ({} != {})", - downloader_id, getCallerId()); - } - - resetDownloaderImpl(segment_lock); - - LOG_TEST(log, "Complete batch. Current downloaded size: {}", getDownloadedSizeUnlocked(segment_lock)); + resetDownloadingStateUnlocked(segment_lock); + resetDownloaderUnlocked(segment_lock); + LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(segment_lock)); cv.notify_all(); } void FileSegment::completeWithState(State state) { std::lock_guard cache_lock(cache->mutex); - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); - assertNotDetached(segment_lock); - - auto caller_id = getCallerId(); - if (caller_id != downloader_id) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "File segment completion can be done only by downloader. (CallerId: {}, downloader id: {}", - caller_id, downloader_id); - } + assertNotDetachedUnlocked(segment_lock); + assertIsDownloaderUnlocked("complete", segment_lock); if (state != State::DOWNLOADED && state != State::PARTIALLY_DOWNLOADED @@ -485,24 +516,29 @@ void FileSegment::completeWithState(State state) "Cannot complete file segment with state: {}", stateToString(state)); } - download_state = state; + setDownloadState(state); completeBasedOnCurrentState(cache_lock, segment_lock); } -void FileSegment::completeWithoutState(std::lock_guard & cache_lock) +void FileSegment::completeWithoutState() { - std::lock_guard segment_lock(mutex); + std::lock_guard cache_lock(cache->mutex); + completeWithoutStateUnlocked(cache_lock); +} + +void FileSegment::completeWithoutStateUnlocked(std::lock_guard & cache_lock) +{ + std::unique_lock segment_lock(mutex); completeBasedOnCurrentState(cache_lock, segment_lock); } -void FileSegment::completeBasedOnCurrentState(std::lock_guard & cache_lock, std::lock_guard & segment_lock) +void FileSegment::completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock) { if (is_detached) return; - bool is_downloader = isDownloaderImpl(segment_lock); + bool is_downloader = isDownloaderUnlocked(segment_lock); bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); - bool can_update_segment_state = is_downloader || is_last_holder; size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); SCOPE_EXIT({ @@ -512,16 +548,15 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach } }); - LOG_TEST(log, "Complete without state (is_last_holder: {}). File segment info: {}", is_last_holder, getInfoForLogImpl(segment_lock)); + LOG_TEST( + log, + "Complete based on current state (is_last_holder: {}, {})", + is_last_holder, getInfoForLogUnlocked(segment_lock)); - if (can_update_segment_state) + if (is_downloader) { - if (current_downloaded_size == range().size()) - setDownloaded(segment_lock); - else - download_state = State::PARTIALLY_DOWNLOADED; - - resetDownloaderImpl(segment_lock); + resetDownloadingStateUnlocked(segment_lock); + resetDownloaderUnlocked(segment_lock); } switch (download_state) @@ -535,16 +570,16 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach } case State::DOWNLOADED: { - assert(getDownloadedSizeUnlocked(segment_lock) == range().size()); - assert(isDownloadedUnlocked(segment_lock)); + chassert(getDownloadedSizeUnlocked(segment_lock) == range().size()); + assert(is_downloaded); break; } case State::DOWNLOADING: - case State::EMPTY: { - assert(!is_last_holder); + chassert(!is_last_holder); break; } + case State::EMPTY: case State::PARTIALLY_DOWNLOADED: case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { @@ -554,7 +589,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach { LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString()); - download_state = State::SKIP_CACHE; + setDownloadState(State::SKIP_CACHE); cache->remove(key(), offset(), cache_lock, segment_lock); } else @@ -567,7 +602,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach * in FileSegmentsHolder represent a contiguous range, so we can resize * it only when nobody needs it. */ - download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; + setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); /// Resize this file segment by creating a copy file segment with DOWNLOADED state, /// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state, @@ -576,23 +611,22 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach cache->reduceSizeToDownloaded(key(), offset(), cache_lock, segment_lock); } - markAsDetached(segment_lock); + detachAssumeStateFinalized(segment_lock); } break; } } - LOG_TEST(log, "Completed file segment: {}", getInfoForLogImpl(segment_lock)); - assertCorrectnessImpl(segment_lock); + LOG_TEST(log, "Completed file segment: {}", getInfoForLogUnlocked(segment_lock)); } String FileSegment::getInfoForLog() const { - std::lock_guard segment_lock(mutex); - return getInfoForLogImpl(segment_lock); + std::unique_lock segment_lock(mutex); + return getInfoForLogUnlocked(segment_lock); } -String FileSegment::getInfoForLogImpl(std::lock_guard & segment_lock) const +String FileSegment::getInfoForLogUnlocked(std::unique_lock & segment_lock) const { WriteBufferFromOwnString info; info << "File segment: " << range().toString() << ", "; @@ -601,15 +635,18 @@ String FileSegment::getInfoForLogImpl(std::lock_guard & segment_lock info << "downloaded size: " << getDownloadedSizeUnlocked(segment_lock) << ", "; info << "reserved size: " << reserved_size << ", "; info << "downloader id: " << (downloader_id.empty() ? "None" : downloader_id) << ", "; + info << "current write offset: " << getCurrentWriteOffsetUnlocked(segment_lock) << ", "; + info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", "; info << "caller id: " << getCallerId() << ", "; + info << "detached: " << is_detached << ", "; info << "persistent: " << is_persistent; return info.str(); } -void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard & segment_lock) const +void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock & segment_lock) const { - e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogImpl(segment_lock))); + e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogUnlocked(segment_lock))); } String FileSegment::stateToString(FileSegment::State state) @@ -634,63 +671,65 @@ String FileSegment::stateToString(FileSegment::State state) void FileSegment::assertCorrectness() const { - std::lock_guard segment_lock(mutex); - assertCorrectnessImpl(segment_lock); + std::unique_lock segment_lock(mutex); + assertCorrectnessUnlocked(segment_lock); } -void FileSegment::assertCorrectnessImpl(std::lock_guard & /* segment_lock */) const +void FileSegment::assertCorrectnessUnlocked(std::unique_lock & segment_lock) const { - assert(downloader_id.empty() == (download_state != FileSegment::State::DOWNLOADING)); - assert(!downloader_id.empty() == (download_state == FileSegment::State::DOWNLOADING)); - assert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0); + // auto current_downloader = getDownloaderUnlocked(false, segment_lock); + LOG_TEST(log, "Checking correctness: {}", getInfoForLogUnlocked(segment_lock)); + // chassert(current_downloader.empty() == (download_state != FileSegment::State::DOWNLOADING)); + // chassert(!current_downloader.empty() == (download_state == FileSegment::State::DOWNLOADING)); + // chassert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0); } -void FileSegment::throwIfDetached() const -{ - std::lock_guard segment_lock(mutex); - throwIfDetachedUnlocked(segment_lock); -} - -void FileSegment::throwIfDetachedUnlocked(std::lock_guard & segment_lock) const +void FileSegment::throwIfDetachedUnlocked(std::unique_lock & segment_lock) const { throw Exception( ErrorCodes::LOGICAL_ERROR, "Cache file segment is in detached state, operation not allowed. " "It can happen when cache was concurrently dropped with SYSTEM DROP FILESYSTEM CACHE FORCE. " - "Please, retry. File segment info: {}", getInfoForLogImpl(segment_lock)); + "Please, retry. File segment info: {}", getInfoForLogUnlocked(segment_lock)); } +void FileSegment::assertNotDetached() const +{ + std::unique_lock segment_lock(mutex); + assertNotDetachedUnlocked(segment_lock); +} -void FileSegment::assertNotDetached(std::lock_guard & segment_lock) const +void FileSegment::assertNotDetachedUnlocked(std::unique_lock & segment_lock) const { if (is_detached) throwIfDetachedUnlocked(segment_lock); } -void FileSegment::assertDetachedStatus(std::lock_guard & segment_lock) const +void FileSegment::assertDetachedStatus(std::unique_lock & segment_lock) const { /// Detached file segment is allowed to have only a certain subset of states. /// It should be either EMPTY or one of the finalized states. - if (download_state != State::EMPTY && !hasFinalizedState()) + if (download_state != State::EMPTY && !hasFinalizedStateUnlocked(segment_lock)) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Detached file segment has incorrect state: {}", - getInfoForLogImpl(segment_lock)); + getInfoForLogUnlocked(segment_lock)); } } FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & /* cache_lock */) { - std::lock_guard segment_lock(file_segment->mutex); + std::unique_lock segment_lock(file_segment->mutex); auto snapshot = std::make_shared( file_segment->offset(), file_segment->range().size(), file_segment->key(), nullptr, - State::EMPTY); + State::EMPTY, + CreateFileSegmentSettings{}); snapshot->hits_count = file_segment->getHitsCount(); snapshot->ref_count = file_segment.use_count(); @@ -701,41 +740,40 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std return snapshot; } -bool FileSegment::hasFinalizedState() const +bool FileSegment::hasFinalizedStateUnlocked(std::unique_lock & /* segment_lock */) const { return download_state == State::DOWNLOADED || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION || download_state == State::SKIP_CACHE; } -void FileSegment::detach( - std::lock_guard & /* cache_lock */, - std::lock_guard & segment_lock) +bool FileSegment::isDetached() const +{ + std::unique_lock segment_lock(mutex); + return is_detached; +} + +void FileSegment::detach(std::lock_guard & /* cache_lock */, std::unique_lock & segment_lock) { - /// Now detached status can be in 2 cases, which do not do any complex logic: - /// 1. there is only 1 remaining file segment holder - /// && it does not need this segment anymore - /// && this file segment was in cache and needs to be removed - /// 2. in read_from_cache_if_exists_otherwise_bypass_cache case if (is_detached) return; - markAsDetached(segment_lock); - download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; - downloader_id.clear(); + resetDownloaderUnlocked(segment_lock); + setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - LOG_DEBUG(log, "Detached file segment: {}", getInfoForLogImpl(segment_lock)); + detachAssumeStateFinalized(segment_lock); } -void FileSegment::markAsDetached(std::lock_guard & /* segment_lock */) +void FileSegment::detachAssumeStateFinalized(std::unique_lock & segment_lock) { is_detached = true; CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments); + LOG_TEST(log, "Detached file segment: {}", getInfoForLogUnlocked(segment_lock)); } FileSegment::~FileSegment() { - std::lock_guard segment_lock(mutex); + std::unique_lock segment_lock(mutex); if (is_detached) CurrentMetrics::sub(CurrentMetrics::CacheDetachedFileSegments); } @@ -761,7 +799,7 @@ FileSegmentsHolder::~FileSegmentsHolder() bool is_detached = false; { - std::lock_guard segment_lock(file_segment->mutex); + std::unique_lock segment_lock(file_segment->mutex); is_detached = file_segment->isDetached(segment_lock); if (is_detached) file_segment->assertDetachedStatus(segment_lock); @@ -779,7 +817,7 @@ FileSegmentsHolder::~FileSegmentsHolder() /// under the same mutex, because complete() checks for segment pointers. std::lock_guard cache_lock(cache->mutex); - file_segment->completeWithoutState(cache_lock); + file_segment->completeWithoutStateUnlocked(cache_lock); file_segment_it = file_segments.erase(current_file_segment_it); } @@ -822,10 +860,16 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset std::lock_guard cache_lock(cache->mutex); + CreateFileSegmentSettings create_settings + { + .is_persistent = is_persistent, + }; + /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. auto file_segment = cache->createFileSegmentForDownload( - key, offset, cache->max_file_segment_size, is_persistent, cache_lock); + key, offset, cache->max_file_segment_size, create_settings, cache_lock); + return file_segments_holder.add(std::move(file_segment)); } @@ -859,10 +903,7 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) file_segment.reserved_size = current_downloaded_size; } - { - std::lock_guard cache_lock(cache->mutex); - file_segment.completeWithoutState(cache_lock); - } + file_segment.completeWithoutState(); on_complete_file_segment_func(file_segment); } @@ -893,18 +934,20 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset offset, current_file_segment_write_offset); } + size_t current_write_offset = (*current_file_segment_it)->getCurrentWriteOffset(); + auto current_file_segment = *current_file_segment_it; if (current_file_segment->getRemainingSizeToDownload() == 0) { completeFileSegment(*current_file_segment); current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); } - else if (current_file_segment->getDownloadOffset() != offset) + else if (current_write_offset != offset) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Cannot file segment download offset {} does not match current write offset {}", - current_file_segment->getDownloadOffset(), offset); + current_write_offset, offset); } } @@ -915,7 +958,10 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set a downloader. ({})", file_segment->getInfoForLog()); SCOPE_EXIT({ - file_segment->resetDownloader(); + if (file_segment->isDownloader()) + { + file_segment->completePartAndResetDownloader(); + } }); bool reserved = file_segment->reserve(size); @@ -932,7 +978,17 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset return false; } - (*current_file_segment_it)->write(data, size, offset); + try + { + file_segment->write(data, size, offset); + } + catch (...) + { + file_segment->completePartAndResetDownloader(); + throw; + } + + file_segment->completePartAndResetDownloader(); current_file_segment_write_offset += size; return true; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index f3fb367792a..9d9ddc3923a 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -1,11 +1,15 @@ #pragma once -#include +#include +#include #include #include +#include +#include +#include #include -#include +#include namespace Poco { class Logger; } @@ -26,17 +30,25 @@ using FileSegmentPtr = std::shared_ptr; using FileSegments = std::list; -class FileSegment : boost::noncopyable +struct CreateFileSegmentSettings +{ + bool is_persistent = false; +}; + +class FileSegment : private boost::noncopyable, public std::enable_shared_from_this { friend class FileCache; friend struct FileSegmentsHolder; friend class FileSegmentRangeWriter; +friend class StorageSystemFilesystemCache; public: using Key = FileCacheKey; using RemoteFileReaderPtr = std::shared_ptr; using LocalCacheWriterPtr = std::unique_ptr; + using Downloader = std::string; + using DownloaderId = std::string; enum class State { @@ -78,7 +90,7 @@ public: const Key & key_, FileCache * cache_, State download_state_, - bool is_persistent_ = false); + const CreateFileSegmentSettings & create_settings); ~FileSegment(); @@ -101,6 +113,14 @@ public: String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); } }; + static String getCallerId(); + + String getInfoForLog() const; + + /** + * ========== Methods to get file segment's constant state ================== + */ + const Range & range() const { return segment_range; } const Key & key() const { return file_key; } @@ -109,11 +129,85 @@ public: bool isPersistent() const { return is_persistent; } + using UniqueId = std::pair; + UniqueId getUniqueId() const { return std::pair(key(), offset()); } + + String getPathInLocalCache() const; + + /** + * ========== Methods for _any_ file segment's owner ======================== + */ + + String getOrSetDownloader(); + + bool isDownloader() const; + + DownloaderId getDownloader() const; + + /// Wait for the change of state from DOWNLOADING to any other. State wait(); - bool reserve(size_t size); + bool isDownloaded() const; - void write(const char * from, size_t size, size_t offset_); + size_t getHitsCount() const { return hits_count; } + + size_t getRefCount() const { return ref_count; } + + void incrementHitsCount() { ++hits_count; } + + size_t getCurrentWriteOffset() const; + + size_t getFirstNonDownloadedOffset() const; + + size_t getDownloadedSize() const; + + /// Now detached status can be used in the following cases: + /// 1. there is only 1 remaining file segment holder + /// && it does not need this segment anymore + /// && this file segment was in cache and needs to be removed + /// 2. in read_from_cache_if_exists_otherwise_bypass_cache case to create NOOP file segments. + /// 3. removeIfExists - method which removes file segments from cache even though + /// it might be used at the moment. + + /// If file segment is detached it means the following: + /// 1. It is not present in FileCache, e.g. will not be visible to any cache user apart from + /// those who acquired shared pointer to this file segment before it was detached. + /// 2. Detached file segment can still be hold by some cache users, but it's state became + /// immutable at the point it was detached, any non-const / stateful method will throw an + /// exception. + void detach(std::lock_guard & cache_lock, std::unique_lock & segment_lock); + + static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & cache_lock); + + bool isDetached() const; + + void assertCorrectness() const; + + /** + * ========== Methods for _only_ file segment's `writer` ====================== + */ + + void synchronousWrite(const char * from, size_t size, size_t offset); + + /** + * ========== Methods for _only_ file segment's `downloader` ================== + */ + + /// Try to reserve exactly `size` bytes. + bool reserve(size_t size_to_reserve); + + /// Write data into reserved space. + void write(const char * from, size_t size, size_t offset); + + /// Complete file segment with a certain state. + void completeWithState(State state); + + void completeWithoutState(); + + /// Complete file segment's part which was last written. + void completePartAndResetDownloader(); + + void resetDownloader(); RemoteFileReaderPtr getRemoteFileReader(); @@ -123,91 +217,56 @@ public: void resetRemoteFileReader(); - String getOrSetDownloader(); - - String getDownloader() const; - - void resetDownloader(); - - bool isDownloader() const; - - bool isDownloaded() const; - - static String getCallerId(); - - size_t getDownloadOffset() const; - - size_t getDownloadedSize() const; - size_t getRemainingSizeToDownload() const; - void completeBatchAndResetDownloader(); - - void completeWithState(State state); - - String getInfoForLog() const; - - size_t getHitsCount() const { return hits_count; } - - size_t getRefCount() const { return ref_count; } - - void incrementHitsCount() { ++hits_count; } - - void assertCorrectness() const; - - static FileSegmentPtr getSnapshot( - const FileSegmentPtr & file_segment, - std::lock_guard & cache_lock); - - void detach( - std::lock_guard & cache_lock, - std::lock_guard & segment_lock); - - [[noreturn]] void throwIfDetached() const; - - bool isDetached() const; - - String getPathInLocalCache() const; - private: - size_t availableSize() const { return reserved_size - downloaded_size; } + size_t getFirstNonDownloadedOffsetUnlocked(std::unique_lock & segment_lock) const; + size_t getCurrentWriteOffsetUnlocked(std::unique_lock & segment_lock) const; + size_t getDownloadedSizeUnlocked(std::unique_lock & segment_lock) const; + size_t getAvailableSizeUnlocked(std::unique_lock & segment_lock) const; - size_t getDownloadedSizeUnlocked(std::lock_guard & segment_lock) const; - String getInfoForLogImpl(std::lock_guard & segment_lock) const; - void assertCorrectnessImpl(std::lock_guard & segment_lock) const; - bool hasFinalizedState() const; + String getInfoForLogUnlocked(std::unique_lock & segment_lock) const; - bool isDetached(std::lock_guard & /* segment_lock */) const { return is_detached; } - void markAsDetached(std::lock_guard & segment_lock); - [[noreturn]] void throwIfDetachedUnlocked(std::lock_guard & segment_lock) const; + String getDownloaderUnlocked(std::unique_lock & segment_lock) const; + void resetDownloaderUnlocked(std::unique_lock & segment_lock); + void resetDownloadingStateUnlocked(std::unique_lock & segment_lock); - void assertDetachedStatus(std::lock_guard & segment_lock) const; - void assertNotDetached(std::lock_guard & segment_lock) const; + void setDownloadState(State state); - void setDownloaded(std::lock_guard & segment_lock); - void setDownloadFailed(std::lock_guard & segment_lock); - bool isDownloaderImpl(std::lock_guard & segment_lock) const; + void setDownloadedUnlocked(std::unique_lock & segment_lock); + void setDownloadFailedUnlocked(std::unique_lock & segment_lock); - bool isDownloadedUnlocked(std::lock_guard & segment_lock) const; + bool hasFinalizedStateUnlocked(std::unique_lock & segment_lock) const; - void wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard & segment_lock) const; + bool isDownloaderUnlocked(std::unique_lock & segment_lock) const; - bool lastFileSegmentHolder() const; + bool isDetached(std::unique_lock & /* segment_lock */) const { return is_detached; } + void detachAssumeStateFinalized(std::unique_lock & segment_lock); + [[noreturn]] void throwIfDetachedUnlocked(std::unique_lock & segment_lock) const; + + void assertDetachedStatus(std::unique_lock & segment_lock) const; + void assertNotDetached() const; + void assertNotDetachedUnlocked(std::unique_lock & segment_lock) const; + void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const; + void assertCorrectnessUnlocked(std::unique_lock & segment_lock) const; /// complete() without any completion state is called from destructor of /// FileSegmentsHolder. complete() might check if the caller of the method /// is the last alive holder of the segment. Therefore, complete() and destruction /// of the file segment pointer must be done under the same cache mutex. - void completeBasedOnCurrentState(std::lock_guard & cache_lock, std::lock_guard & segment_lock); - void completeWithoutState(std::lock_guard & cache_lock); + void completeWithoutStateUnlocked(std::lock_guard & cache_lock); + void completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock); - void resetDownloaderImpl(std::lock_guard & segment_lock); + void completePartAndResetDownloaderUnlocked(std::unique_lock & segment_lock); + + void wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock & segment_lock) const; Range segment_range; State download_state; - String downloader_id; + /// The one who prepares the download + DownloaderId downloader_id; RemoteFileReaderPtr remote_file_reader; LocalCacheWriterPtr cache_writer; @@ -245,6 +304,7 @@ private: std::atomic ref_count = 0; /// Used for getting snapshot state bool is_persistent; + CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments}; }; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 6460eeef8c5..22150b9f656 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -3,9 +3,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -64,7 +64,7 @@ void download(DB::FileSegmentPtr file_segment) fs::create_directories(subdir); std::string data(size, '0'); - file_segment->write(data.data(), size, file_segment->getDownloadOffset()); + file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset()); } void prepareAndDownload(DB::FileSegmentPtr file_segment) @@ -89,6 +89,7 @@ TEST(FileCache, get) { if (fs::exists(cache_base_path)) fs::remove_all(cache_base_path); + fs::create_directories(cache_base_path); DB::ThreadStatus thread_status; @@ -109,7 +110,7 @@ TEST(FileCache, get) auto key = cache.hash("key1"); { - auto holder = cache.getOrSet(key, 0, 10, false); /// Add range [0, 9] + auto holder = cache.getOrSet(key, 0, 10, {}); /// Add range [0, 9] auto segments = fromHolder(holder); /// Range was not present in cache. It should be added in cache as one while file segment. ASSERT_EQ(segments.size(), 1); @@ -138,7 +139,7 @@ TEST(FileCache, get) { /// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache. - auto holder = cache.getOrSet(key, 5, 10, false); + auto holder = cache.getOrSet(key, 5, 10, {}); auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 2); @@ -158,14 +159,14 @@ TEST(FileCache, get) ASSERT_EQ(cache.getUsedCacheSize(), 15); { - auto holder = cache.getOrSet(key, 9, 1, false); /// Get [9, 9] + auto holder = cache.getOrSet(key, 9, 1, {}); /// Get [9, 9] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } { - auto holder = cache.getOrSet(key, 9, 2, false); /// Get [9, 10] + auto holder = cache.getOrSet(key, 9, 2, {}); /// Get [9, 10] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 2); assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); @@ -173,15 +174,15 @@ TEST(FileCache, get) } { - auto holder = cache.getOrSet(key, 10, 1, false); /// Get [10, 10] + auto holder = cache.getOrSet(key, 10, 1, {}); /// Get [10, 10] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } - complete(cache.getOrSet(key, 17, 4, false)); /// Get [17, 20] - complete(cache.getOrSet(key, 24, 3, false)); /// Get [24, 26] - /// complete(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] + complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] + complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] + /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] /// Current cache: [__________][_____] [____] [___][] /// ^ ^^ ^ ^ ^ ^ ^^^ @@ -191,7 +192,7 @@ TEST(FileCache, get) ASSERT_EQ(cache.getUsedCacheSize(), 22); { - auto holder = cache.getOrSet(key, 0, 26, false); /// Get [0, 25] + auto holder = cache.getOrSet(key, 0, 26, {}); /// Get [0, 25] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 6); @@ -225,14 +226,14 @@ TEST(FileCache, get) /// as max elements size is reached, next attempt to put something in cache should fail. /// This will also check that [27, 27] was indeed evicted. - auto holder1 = cache.getOrSet(key, 27, 1, false); + auto holder1 = cache.getOrSet(key, 27, 1, {}); auto segments_1 = fromHolder(holder1); /// Get [27, 27] ASSERT_EQ(segments_1.size(), 1); assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); } { - auto holder = cache.getOrSet(key, 12, 10, false); /// Get [12, 21] + auto holder = cache.getOrSet(key, 12, 10, {}); /// Get [12, 21] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 4); @@ -256,7 +257,7 @@ TEST(FileCache, get) ASSERT_EQ(cache.getFileSegmentsNum(), 5); { - auto holder = cache.getOrSet(key, 23, 5, false); /// Get [23, 28] + auto holder = cache.getOrSet(key, 23, 5, {}); /// Get [23, 28] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); @@ -277,12 +278,12 @@ TEST(FileCache, get) /// 17 21 2324 26 28 { - auto holder5 = cache.getOrSet(key, 2, 3,false); /// Get [2, 4] + auto holder5 = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] auto s5 = fromHolder(holder5); ASSERT_EQ(s5.size(), 1); assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY); - auto holder1 = cache.getOrSet(key, 30, 2, false); /// Get [30, 31] + auto holder1 = cache.getOrSet(key, 30, 2, {}); /// Get [30, 31] auto s1 = fromHolder(holder1); ASSERT_EQ(s1.size(), 1); assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY); @@ -298,20 +299,20 @@ TEST(FileCache, get) /// ^ ^ ^ ^ ^ ^ ^ ^ /// 2 4 23 24 26 27 30 31 - auto holder2 = cache.getOrSet(key, 23, 1, false); /// Get [23, 23] + auto holder2 = cache.getOrSet(key, 23, 1, {}); /// Get [23, 23] auto s2 = fromHolder(holder2); ASSERT_EQ(s2.size(), 1); - auto holder3 = cache.getOrSet(key, 24, 3, false); /// Get [24, 26] + auto holder3 = cache.getOrSet(key, 24, 3, {}); /// Get [24, 26] auto s3 = fromHolder(holder3); ASSERT_EQ(s3.size(), 1); - auto holder4 = cache.getOrSet(key, 27, 1, false); /// Get [27, 27] + auto holder4 = cache.getOrSet(key, 27, 1, {}); /// Get [27, 27] auto s4 = fromHolder(holder4); ASSERT_EQ(s4.size(), 1); /// All cache is now unreleasable because pointers are still hold - auto holder6 = cache.getOrSet(key, 0, 40, false); + auto holder6 = cache.getOrSet(key, 0, 40, {}); auto f = fromHolder(holder6); ASSERT_EQ(f.size(), 9); @@ -332,7 +333,7 @@ TEST(FileCache, get) } { - auto holder = cache.getOrSet(key, 2, 3, false); /// Get [2, 4] + auto holder = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); @@ -343,7 +344,7 @@ TEST(FileCache, get) /// 2 4 23 24 26 27 30 31 { - auto holder = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] + auto holder = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); @@ -367,7 +368,7 @@ TEST(FileCache, get) DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); thread_status_1.attachQueryContext(query_context_1); - auto holder_2 = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] once again. + auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again. auto segments_2 = fromHolder(holder_2); ASSERT_EQ(segments.size(), 3); @@ -406,11 +407,11 @@ TEST(FileCache, get) { /// Now let's check the similar case but getting ERROR state after segment->wait(), when - /// state is changed not manually via segment->complete(state) but from destructor of holder + /// state is changed not manually via segment->completeWithState(state) but from destructor of holder /// and notify_all() is also called from destructor of holder. std::optional holder; - holder.emplace(cache.getOrSet(key, 3, 23, false)); /// Get [3, 25] + holder.emplace(cache.getOrSet(key, 3, 23, {})); /// Get [3, 25] auto segments = fromHolder(*holder); ASSERT_EQ(segments.size(), 3); @@ -436,7 +437,7 @@ TEST(FileCache, get) DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); thread_status_1.attachQueryContext(query_context_1); - auto holder_2 = cache.getOrSet(key, 3, 23, false); /// Get [3, 25] once again + auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again auto segments_2 = fromHolder(*holder); ASSERT_EQ(segments_2.size(), 3); @@ -485,7 +486,7 @@ TEST(FileCache, get) cache2.initialize(); auto key = cache2.hash("key1"); - auto holder1 = cache2.getOrSet(key, 2, 28, false); /// Get [2, 29] + auto holder1 = cache2.getOrSet(key, 2, 28, {}); /// Get [2, 29] auto segments1 = fromHolder(holder1); ASSERT_EQ(segments1.size(), 5); @@ -506,7 +507,7 @@ TEST(FileCache, get) cache2.initialize(); auto key = cache2.hash("key1"); - auto holder1 = cache2.getOrSet(key, 0, 25, false); /// Get [0, 24] + auto holder1 = cache2.getOrSet(key, 0, 25, {}); /// Get [0, 24] auto segments1 = fromHolder(holder1); ASSERT_EQ(segments1.size(), 3); From 23d95e672353d8a8954c05e9fa90eaf1498194bd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Sep 2022 13:24:22 +0200 Subject: [PATCH 489/582] Fix parts removal in case of failures --- src/Storages/MergeTree/MergeTreeData.cpp | 30 ++++++++++++++++----- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index edc070a8acf..048874fb3a3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2135,13 +2135,8 @@ void MergeTreeData::dropAllData() auto lock = lockParts(); - LOG_TRACE(log, "dropAllData: removing data from memory."); - DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end()); - data_parts_indexes.clear(); - column_sizes.clear(); - { std::lock_guard wal_lock(write_ahead_log_mutex); if (write_ahead_log) @@ -2153,10 +2148,30 @@ void MergeTreeData::dropAllData() if (!getStorageID().hasUUID()) getContext()->dropCaches(); - LOG_TRACE(log, "dropAllData: removing data from filesystem."); /// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls. - clearPartsFromFilesystem(all_parts); + NameSet part_names_failed; + try + { + LOG_TRACE(log, "dropAllData: removing data parts (count {}) from filesystem.", all_parts.size()); + clearPartsFromFilesystem(all_parts, true, &part_names_failed); + + LOG_TRACE(log, "dropAllData: removing all data parts from memory."); + data_parts_indexes.clear(); + } + catch (...) + { + LOG_WARNING(log, "dropAllData: got exception removing parts from disk, removing successfully removed parts from memory."); + for (const auto & part : all_parts) + { + if (!part_names_failed.contains(part->name)) + data_parts_indexes.erase(part->info); + } + + throw; + } + + column_sizes.clear(); for (const auto & disk : getDisks()) { @@ -2165,6 +2180,7 @@ void MergeTreeData::dropAllData() try { + LOG_INFO(log, "dropAllData: removing table directory recursive to cleanup garbage"); disk->removeRecursive(relative_data_path); } catch (const fs::filesystem_error & e) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3491f6c9d4a..0cb18eeb1c7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7554,7 +7554,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedData(const IMer if (!part.data_part_storage || !part.data_part_storage->supportZeroCopyReplication()) { - LOG_TRACE(log, "Part {} is not stored on zero-copy replicaed disk, blobs can be removed", part.name); + LOG_TRACE(log, "Part {} is not stored on zero-copy replicated disk, blobs can be removed", part.name); return std::make_pair(true, NameSet{}); } From 01e745bbbf084fa14e3d992a718d85be71ebae26 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Sep 2022 13:26:13 +0200 Subject: [PATCH 490/582] Fix parts removal --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 048874fb3a3..7e2e4ab174b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2161,6 +2161,9 @@ void MergeTreeData::dropAllData() } catch (...) { + /// Removing from memory only successfully removed parts from disk + /// Parts removal process can be important and on the next try it's better to try to remove + /// them instead of remove recursive call. LOG_WARNING(log, "dropAllData: got exception removing parts from disk, removing successfully removed parts from memory."); for (const auto & part : all_parts) { From b33c1fa60f557ff0fda3f8ec11b6817b7cda04a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 9 Sep 2022 13:46:53 +0200 Subject: [PATCH 491/582] Fix query_views_log with Window views --- src/Interpreters/QueryViewsLog.cpp | 3 +- ..._query_views_log_window_function.reference | 40 +++++++++++++++++++ .../02125_query_views_log_window_function.sql | 38 ++++++++++++++++++ 3 files changed, 80 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02125_query_views_log_window_function.reference create mode 100644 tests/queries/0_stateless/02125_query_views_log_window_function.sql diff --git a/src/Interpreters/QueryViewsLog.cpp b/src/Interpreters/QueryViewsLog.cpp index c0703d77691..fc235d0d1cd 100644 --- a/src/Interpreters/QueryViewsLog.cpp +++ b/src/Interpreters/QueryViewsLog.cpp @@ -28,7 +28,8 @@ NamesAndTypesList QueryViewsLogElement::getNamesAndTypes() auto view_type_datatype = std::make_shared(DataTypeEnum8::Values{ {"Default", static_cast(ViewType::DEFAULT)}, {"Materialized", static_cast(ViewType::MATERIALIZED)}, - {"Live", static_cast(ViewType::LIVE)}}); + {"Live", static_cast(ViewType::LIVE)}, + {"Window", static_cast(ViewType::WINDOW)}}); return { {"event_date", std::make_shared()}, diff --git a/tests/queries/0_stateless/02125_query_views_log_window_function.reference b/tests/queries/0_stateless/02125_query_views_log_window_function.reference new file mode 100644 index 00000000000..09e62921fad --- /dev/null +++ b/tests/queries/0_stateless/02125_query_views_log_window_function.reference @@ -0,0 +1,40 @@ +-- { echo } + +WITH + ( + SELECT initial_query_id + FROM system.query_log + WHERE current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT INTO wv%' + LIMIT 1 + ) AS q_id +SELECT view_name, view_type, view_query, read_rows, read_bytes, written_rows, written_bytes +FROM system.query_views_log +WHERE initial_query_id = q_id FORMAT Vertical; +Row 1: +────── +view_name: default.wv +view_type: Window +view_query: SELECT count(id), tumbleStart(w_id) AS window_start FROM default.data GROUP BY windowID(timestamp, toIntervalSecond('10')) AS w_id +read_rows: 1 +read_bytes: 12 +written_rows: 0 +written_bytes: 0 +WITH + ( + SELECT initial_query_id + FROM system.query_log + WHERE current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT INTO wv%' + LIMIT 1 + ) AS q_id +SELECT views +FROM system.query_log +WHERE initial_query_id = q_id + AND type = 'QueryFinish' +FORMAT Vertical; +Row 1: +────── +views: ['default.wv'] diff --git a/tests/queries/0_stateless/02125_query_views_log_window_function.sql b/tests/queries/0_stateless/02125_query_views_log_window_function.sql new file mode 100644 index 00000000000..1de2cc95b14 --- /dev/null +++ b/tests/queries/0_stateless/02125_query_views_log_window_function.sql @@ -0,0 +1,38 @@ +set allow_experimental_window_view = 1; +CREATE TABLE data ( `id` UInt64, `timestamp` DateTime) ENGINE = Memory; +CREATE WINDOW VIEW wv Engine Memory as select count(id), tumbleStart(w_id) as window_start from data group by tumble(timestamp, INTERVAL '10' SECOND) as w_id; + +-- INSERT INTO wv +INSERT INTO data VALUES(1,now()); + +SYSTEM FLUSH LOGS; + +-- { echo } + +WITH + ( + SELECT initial_query_id + FROM system.query_log + WHERE current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT INTO wv%' + LIMIT 1 + ) AS q_id +SELECT view_name, view_type, view_query, read_rows, read_bytes, written_rows, written_bytes +FROM system.query_views_log +WHERE initial_query_id = q_id FORMAT Vertical; + +WITH + ( + SELECT initial_query_id + FROM system.query_log + WHERE current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT INTO wv%' + LIMIT 1 + ) AS q_id +SELECT views +FROM system.query_log +WHERE initial_query_id = q_id + AND type = 'QueryFinish' +FORMAT Vertical; From 2472b9c4047eb666755368c91d0ca62fd1002b3f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Sep 2022 12:24:26 +0000 Subject: [PATCH 492/582] Mark primary key test as long --- tests/queries/0_stateless/02419_keeper_map_primary_key.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02419_keeper_map_primary_key.sh b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh index d5ec3be58da..c43c5bb6408 100755 --- a/tests/queries/0_stateless/02419_keeper_map_primary_key.sh +++ b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-ordinary-database, no-fasttest +# Tags: no-ordinary-database, no-fasttest, long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 3df65b9cfbb6b690c63f234eb86fe75fe3cfc964 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 9 Sep 2022 15:25:52 +0300 Subject: [PATCH 493/582] Update 01172_transaction_counters.sql --- tests/queries/0_stateless/01172_transaction_counters.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index ee00029501a..ffe2d08e2d8 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -1,5 +1,4 @@ --- Tags: no-s3-storage, no-ordinary-database --- FIXME this test fails with S3 due to a bug in DiskCacheWrapper +-- Tags: no-ordinary-database drop table if exists txn_counters; From afe371776108a5dbf8c9ce13d546ef46921f3c72 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 9 Sep 2022 08:34:42 -0400 Subject: [PATCH 494/582] move title to frontmatter to allow inclusion in other docs --- docs/en/getting-started/install.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 92873cb5fbf..83561b07ade 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -4,10 +4,9 @@ sidebar_position: 1 keywords: [clickhouse, install, installation, docs] description: ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture. slug: /en/getting-started/install +title: Installation --- -# Installation - ## System Requirements {#system-requirements} ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture. From 5bb40e5856ead49d8ccbfc8177255da00f96f88d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 9 Sep 2022 15:04:32 +0200 Subject: [PATCH 495/582] Change cache setting do_not_evict_index_and_mark_files default to 0. (#41139) --- src/Interpreters/Cache/FileCacheSettings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index b08c80f20db..819eeaf4140 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -31,7 +31,7 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false); enable_cache_hits_threshold = config.getUInt64(config_prefix + ".enable_cache_hits_threshold", REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD); - do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", true); + do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", false); } } From 98509e62e8bccff6756eac181fb24848ba8d471c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 9 Sep 2022 16:20:24 +0300 Subject: [PATCH 496/582] Update src/Storages/Freeze.cpp --- src/Storages/Freeze.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index d94f25ebf9b..a2f0395b001 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -74,7 +74,7 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) auto metadata_str = metadata_storage->readFileToString(file_path); ReadBufferFromString buffer(metadata_str); readIntText(version, buffer); - if (version < 1 or version > 2) + if (version < 1 || version > 2) { LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown frozen metadata version: {}", version); return false; From dbfb44829017e8703620d75a9c215f8995ddbea6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 9 Sep 2022 14:39:06 +0000 Subject: [PATCH 497/582] Fix: integration test, standalone keeper mode There is logic regarding which keeper binary use to start keeper cluster in an integration test There 2 options: (1) standalone keeper binary (expected binary name clickhouse-keeper) (2) clickhouse binary with keeper inside Fixed: - option (1) didn't work since docker_compose_keeper.yaml didn't create target clickhouse-keeper at all - if clickhouse-keeper existed, option (1) was taken but clickhouse-keeper could be just a link to clickhouse binary (the link is created always during build if cmake option BUILD_STANDALONE_KEEPER is OFF) --- .../integration/runner/compose/docker_compose_keeper.yml | 9 +++++++++ tests/integration/helpers/cluster.py | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_keeper.yml b/docker/test/integration/runner/compose/docker_compose_keeper.yml index 811bbdd800d..8524823ed87 100644 --- a/docker/test/integration/runner/compose/docker_compose_keeper.yml +++ b/docker/test/integration/runner/compose/docker_compose_keeper.yml @@ -8,6 +8,9 @@ services: - type: bind source: ${keeper_binary:-} target: /usr/bin/clickhouse + - type: bind + source: ${keeper_binary:-} + target: /usr/bin/clickhouse-keeper - type: bind source: ${keeper_config_dir1:-} target: /etc/clickhouse-keeper @@ -38,6 +41,9 @@ services: - type: bind source: ${keeper_binary:-} target: /usr/bin/clickhouse + - type: bind + source: ${keeper_binary:-} + target: /usr/bin/clickhouse-keeper - type: bind source: ${keeper_config_dir2:-} target: /etc/clickhouse-keeper @@ -68,6 +74,9 @@ services: - type: bind source: ${keeper_binary:-} target: /usr/bin/clickhouse + - type: bind + source: ${keeper_binary:-} + target: /usr/bin/clickhouse-keeper - type: bind source: ${keeper_config_dir3:-} target: /etc/clickhouse-keeper diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ad1cefe6055..9cc515f98fc 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -792,7 +792,7 @@ class ClickHouseCluster: binary_dir = os.path.dirname(self.server_bin_path) # always prefer clickhouse-keeper standalone binary - if os.path.exists(os.path.join(binary_dir, "clickhouse-keeper")): + if os.path.exists(os.path.join(binary_dir, "clickhouse-keeper")) and not os.path.islink(os.path.join(binary_dir, "clickhouse-keeper")): binary_path = os.path.join(binary_dir, "clickhouse-keeper") keeper_cmd_prefix = "clickhouse-keeper" else: From e9048220516eebf07b59d0f0655115262cb6ab8f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Sep 2022 16:54:03 +0200 Subject: [PATCH 498/582] Bump minio version in tests --- docker/test/stateless/setup_minio.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index e4625bfba75..a1de7f2d6ed 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -22,8 +22,8 @@ ls -lha mkdir -p ./minio_data if [ ! -f ./minio ]; then - MINIO_SERVER_VERSION=${MINIO_SERVER_VERSION:-2022-01-03T18-22-58Z} - MINIO_CLIENT_VERSION=${MINIO_CLIENT_VERSION:-2022-01-05T23-52-51Z} + MINIO_SERVER_VERSION=${MINIO_SERVER_VERSION:-2022-09-07T22-25-02Z} + MINIO_CLIENT_VERSION=${MINIO_CLIENT_VERSION:-2022-08-28T20-08-11Z} case $(uname -m) in x86_64) BIN_ARCH=amd64 ;; aarch64) BIN_ARCH=arm64 ;; From c33aa54032983b8050e8400c3b7f4914fa631ea9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 9 Sep 2022 17:53:26 +0200 Subject: [PATCH 499/582] Fix --- src/Processors/Formats/ISchemaReader.cpp | 34 ++++++++++++------------ 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 648b0f47365..c418cbef41f 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -113,6 +113,11 @@ NamesAndTypesList IRowSchemaReader::readSchema() "Most likely setting input_format_max_rows_to_read_for_schema_inference is set to 0"); DataTypes data_types = readRowAndGetDataTypes(); + + /// Check that we read at list one column. + if (data_types.empty()) + throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Cannot read rows from the data"); + /// If column names weren't set, use default names 'c1', 'c2', ... if (column_names.empty()) { @@ -120,20 +125,19 @@ NamesAndTypesList IRowSchemaReader::readSchema() for (size_t i = 0; i != data_types.size(); ++i) column_names.push_back("c" + std::to_string(i + 1)); } - else if (!data_types.empty()) + /// If column names were set, check that the number of names match the number of types. + else if (column_names.size() != data_types.size()) { - /// If column names were set, check that the number of names match the number of types. - if (column_names.size() != data_types.size()) - throw Exception( - ErrorCodes::INCORRECT_DATA, - "The number of column names {} differs with the number of types {}", column_names.size(), data_types.size()); - - for (size_t i = 0; i != column_names.size(); ++i) - { - auto hint_it = hints.find(column_names[i]); - if (hint_it != hints.end()) - data_types[i] = hint_it->second; - } + throw Exception( + ErrorCodes::INCORRECT_DATA, + "The number of column names {} differs with the number of types {}", column_names.size(), data_types.size()); + } + + for (size_t i = 0; i != column_names.size(); ++i) + { + auto hint_it = hints.find(column_names[i]); + if (hint_it != hints.end()) + data_types[i] = hint_it->second; } for (rows_read = 1; rows_read < max_rows_to_read; ++rows_read) @@ -158,10 +162,6 @@ NamesAndTypesList IRowSchemaReader::readSchema() } } - /// Check that we read at list one column. - if (data_types.empty()) - throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Cannot read rows from the data"); - NamesAndTypesList result; for (size_t i = 0; i != data_types.size(); ++i) { From 877776e569a5f83d685a954ef62877ddbd0f6602 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Sep 2022 18:04:29 +0200 Subject: [PATCH 500/582] Update src/Functions/grouping.h --- src/Functions/grouping.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index 7a9df462b23..b9ef6ffc107 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -19,6 +19,8 @@ protected: static constexpr UInt64 ONE = 1; const ColumnNumbers arguments_indexes; + // Initial implementation of GROUPING function returned 1 if the argument is used as an aggregation key. + // This differs from the behavior described in the standard and other DBMS. const bool force_compatibility; static constexpr UInt64 COMPATIBLE_MODE[] = {1, 0}; From 9bcec2d49a005dbc4a1a75c40596061cf8b5e41b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 9 Sep 2022 18:27:19 +0200 Subject: [PATCH 501/582] stop background ops before converting db engine --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 51 +++++++++++++-------- src/Interpreters/InterpreterSystemQuery.h | 7 +++ src/Interpreters/loadMetadata.cpp | 21 +++++++++ 5 files changed, 61 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 91604c8cc82..e67d57628e8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2974,7 +2974,7 @@ const IHostContextPtr & Context::getHostContext() const } -std::shared_ptr Context::getActionLocksManager() +std::shared_ptr Context::getActionLocksManager() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2997fc370bf..77f7d6cbfdd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -936,7 +936,7 @@ public: bool applyDeletedMask() const { return apply_deleted_mask; } void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } - ActionLocksManagerPtr getActionLocksManager(); + ActionLocksManagerPtr getActionLocksManager() const; enum class ApplicationType { diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index a6d88c7d28b..bac3fa850f2 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -180,30 +180,41 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, { for (auto & elem : DatabaseCatalog::instance().getDatabases()) { - for (auto iterator = elem.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) - { - StoragePtr table = iterator->table(); - if (!table) - continue; - - if (!access->isGranted(required_access_type, elem.first, iterator->name())) - { - LOG_INFO(log, "Access {} denied, skipping {}.{}", toString(required_access_type), elem.first, iterator->name()); - continue; - } - - if (start) - { - manager->remove(table, action_type); - table->onActionLockRemove(action_type); - } - else - manager->add(table, action_type); - } + startStopActionInDatabase(action_type, start, elem.first, elem.second, getContext(), log); } } } +void InterpreterSystemQuery::startStopActionInDatabase(StorageActionBlockType action_type, bool start, + const String & database_name, const DatabasePtr & database, + const ContextPtr & local_context, Poco::Logger * log) +{ + auto manager = local_context->getActionLocksManager(); + auto access = local_context->getAccess(); + auto required_access_type = getRequiredAccessType(action_type); + + for (auto iterator = database->getTablesIterator(local_context); iterator->isValid(); iterator->next()) + { + StoragePtr table = iterator->table(); + if (!table) + continue; + + if (!access->isGranted(required_access_type, database_name, iterator->name())) + { + LOG_INFO(log, "Access {} denied, skipping {}.{}", toString(required_access_type), database_name, iterator->name()); + continue; + } + + if (start) + { + manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } + else + manager->add(table, action_type); + } +} + InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery")) diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index f07f18489f0..af8734e8237 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -16,6 +16,9 @@ namespace DB class Context; class AccessRightsElements; class ASTSystemQuery; +class IDatabase; + +using DatabasePtr = std::shared_ptr; /** Implement various SYSTEM queries. @@ -37,6 +40,10 @@ public: BlockIO execute() override; + static void startStopActionInDatabase(StorageActionBlockType action_type, bool start, + const String & database_name, const DatabasePtr & database, + const ContextPtr & local_context, Poco::Logger * log); + private: ASTPtr query_ptr; Poco::Logger * log = nullptr; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 7d79e2b0224..2eec9d7fdf0 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include #include @@ -32,6 +33,14 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace ActionLocks +{ + extern StorageActionBlockType PartsMerge; + extern StorageActionBlockType PartsFetch; + extern StorageActionBlockType PartsSend; + extern StorageActionBlockType DistributedSend; +} + static void executeCreateQuery( const String & query, ContextMutablePtr context, @@ -327,9 +336,21 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons } auto local_context = Context::createCopy(context); + + /// We have to stop background operations that may lock table for share to avoid DEADLOCK_AVOIDED error + /// on moving tables from Ordinary database. Server has not started to accept connections yet, + /// so there are no user queries, only background operations + LOG_INFO(log, "Will stop background operations to be able to rename tables in Ordinary database {}", database_name); + for (const auto & action : {ActionLocks::PartsMerge, ActionLocks::PartsFetch, ActionLocks::PartsSend, ActionLocks::DistributedSend}) + InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ false, database_name, database, context, log); + local_context->setSetting("check_table_dependencies", false); convertOrdinaryDatabaseToAtomic(log, local_context, database, database_name, tmp_name); + LOG_INFO(log, "Will start background operations after renaming tables in database {}", database_name); + for (const auto & action : {ActionLocks::PartsMerge, ActionLocks::PartsFetch, ActionLocks::PartsSend, ActionLocks::DistributedSend}) + InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ true, database_name, database, context, log); + auto new_database = DatabaseCatalog::instance().getDatabase(database_name); UUID db_uuid = new_database->getUUID(); std::vector tables_uuids; From 57146c9361fd7ba87f7798f9a001db2a1b1d523d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 9 Sep 2022 19:43:14 +0200 Subject: [PATCH 502/582] Fix typos in SortedBlocksWriter Signed-off-by: Azat Khuzhin --- src/Interpreters/SortedBlocksWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 0acb056690f..20859e23ea7 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -87,7 +87,7 @@ void SortedBlocksWriter::insert(Block && block) { std::lock_guard lock{insert_mutex}; - /// insert bock into BlocksList undef lock + /// insert block into BlocksList under lock inserted_blocks.insert(std::move(block)); size_t total_row_count = inserted_blocks.row_count + row_count_in_flush; From 763bb18f98ac34521fe342158a44527e3318d58a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 9 Sep 2022 19:23:45 +0200 Subject: [PATCH 503/582] Fix SIGSEGV in SortedBlocksWriter in case of empty block CI found one issue [1]. Here is the stack trace for invalid read:
stack trace ``` 0: DB::TemporaryFileLazySource::TemporaryFileLazySource(std::__1::basic_string, std::__1::allocator > const&, DB::Block const&) [inlined] std::__1::basic_string, std::__1::allocator >::__is_long(this="") const at string:1445:22 1: DB::TemporaryFileLazySource::TemporaryFileLazySource(std::__1::basic_string, std::__1::allocator > const&, DB::Block const&) [inlined] std::__1::basic_string, std::__1::allocator >::basic_string(this="", __str="") at string:1927 2: DB::TemporaryFileLazySource::TemporaryFileLazySource(this=0x00007f3aec105f58, path_="", header_=0x00007f38ffd93b40) at TemporaryFileLazySource.cpp:11 3: DB::SortedBlocksWriter::streamFromFile(std::__1::unique_ptr > const&) const [inlined] DB::TemporaryFileLazySource* std::__1::construct_at, std::__1::allocator > const&, DB::Block, DB::TemporaryFileLazySource*>(__args=0x00007f38ffd91560) at construct_at.h:38:50 4: DB::SortedBlocksWriter::streamFromFile(std::__1::unique_ptr > const&) const [inlined] void std::__1::allocator_traits >::construct, std::__1::allocator > const&, DB::Block, void, void>(__args=0x00007f38ffd91560) at allocator_traits.h:298 5: DB::SortedBlocksWriter::streamFromFile(std::__1::unique_ptr > const&) const [inlined] std::__1::__shared_ptr_emplace >::__shared_ptr_emplace, std::__1::allocator > const&, DB::Block>(this=0x00007f3aec105f40, __args=0x00007f38ffd91560) at shared_ptr.h:293 6: DB::SortedBlocksWriter::streamFromFile(std::__1::unique_ptr > const&) const [inlined] std::__1::shared_ptr std::__1::allocate_shared, std::__1::basic_string, std::__1::allocator > const&, DB::Block, void>(__args=, __args=) at shared_ptr.h:954 7: DB::SortedBlocksWriter::streamFromFile(std::__1::unique_ptr > const&) const [inlined] std::__1::shared_ptr std::__1::make_shared, std::__1::allocator > const&, DB::Block, void>(__args=, __args=) at shared_ptr.h:963 8: DB::SortedBlocksWriter::streamFromFile(this=, file=) const at SortedBlocksWriter.cpp:238 9: DB::SortedBlocksWriter::premerge(this=) at SortedBlocksWriter.cpp:209:32 ```
[1]: https://s3.amazonaws.com/clickhouse-test-reports/41046/adea92f847373d1fcfd733d8979c63024f9b80bf/stress_test__asan_.html So the problem here is that there was empty unique_ptr<> reference to temporary file, because of empty block that accepted by SortedBlocksWriter::insert(), but insert() is not a problem the problem is premerge() that steals blocks from insert() and do not have check that there are some rows. However this check exists in SortedBlocksWriter::flush(), and in that case temporary file is not created. Signed-off-by: Azat Khuzhin --- src/Interpreters/SortedBlocksWriter.cpp | 10 +++++++++- .../0_stateless/02070_join_on_disk.reference | 0 .../0_stateless/02070_join_on_disk.sql | 19 +++++++++++++++++++ 3 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02070_join_on_disk.reference create mode 100644 tests/queries/0_stateless/02070_join_on_disk.sql diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 20859e23ea7..755c43df635 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -28,6 +28,11 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { @@ -84,6 +89,9 @@ void SortedBlocksWriter::insert(Block && block) size_t bytes = 0; size_t flush_no = 0; + if (!block.rows()) + return; + { std::lock_guard lock{insert_mutex}; @@ -145,7 +153,7 @@ SortedBlocksWriter::TmpFilePtr SortedBlocksWriter::flush(const BlocksList & bloc pipes.emplace_back(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), num_rows))); if (pipes.empty()) - return {}; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty block"); QueryPipelineBuilder pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); diff --git a/tests/queries/0_stateless/02070_join_on_disk.reference b/tests/queries/0_stateless/02070_join_on_disk.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02070_join_on_disk.sql b/tests/queries/0_stateless/02070_join_on_disk.sql new file mode 100644 index 00000000000..c25a7a1ffac --- /dev/null +++ b/tests/queries/0_stateless/02070_join_on_disk.sql @@ -0,0 +1,19 @@ +-- Regression test when Join stores data on disk and receive empty block. +-- Because of this it does not create empty file, while expect it. + +SET max_threads = 1; +SET join_algorithm = 'auto'; +SET max_rows_in_join = 1000; +SET optimize_aggregation_in_order = 1; +SET max_block_size = 1000; + +DROP TABLE IF EXISTS join_on_disk; + +SYSTEM STOP MERGES join_on_disk; + +CREATE TABLE join_on_disk (id Int) Engine=MergeTree() ORDER BY id; + +INSERT INTO join_on_disk SELECT number as id FROM numbers_mt(50000); +INSERT INTO join_on_disk SELECT number as id FROM numbers_mt(1000); + +SELECT id FROM join_on_disk lhs LEFT JOIN (SELECT id FROM join_on_disk GROUP BY id) rhs USING (id) FORMAT Null; From 62a105f216c40233c851dc9c6581a9719555f569 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 8 Sep 2022 20:16:46 +0200 Subject: [PATCH 504/582] Update capnproto to v0.10.2 (to address issue with ASan in clang-15) This should address issue with ASan: - CI report - https://s3.amazonaws.com/clickhouse-test-reports/41046/490a2c75610c4bc3191d55226f8454b3c3d3919a/stateless_tests__asan__[1/2].html 2022-09-08 19:39:40 kj/exception.c++:977: failed: expected offset < 65536 && offset > -65536; ExceptionCallback must be allocated on the stack. - Discussion in ML - https://www.mail-archive.com/capnproto@googlegroups.com/msg01451.html - Fix capnproto/capnproto@c4eef80a13e8575d "Fix ASAN problems under Clang 15. It appears ASAN now by default tries to detect stack-use-after-return. This breaks our assumptions in requireOnStack() and totally breaks fibers. For requireOnStack() we can just skip the check in this case. For fibers, we need to implement the ASAN hints to tell it when we're switching fibers." Signed-off-by: Azat Khuzhin --- contrib/capnproto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/capnproto b/contrib/capnproto index c8189ec3c27..2e88221d3dd 160000 --- a/contrib/capnproto +++ b/contrib/capnproto @@ -1 +1 @@ -Subproject commit c8189ec3c27dacbd4a3288e682473010e377f593 +Subproject commit 2e88221d3dde22266bfccf40eaee6ff9b40d113d From be90cecbd2aa37bb69827a3d7a0a224d8818ab63 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 9 Sep 2022 14:56:38 -0300 Subject: [PATCH 505/582] Revert "Revert "Fix trivial count optimization with array join"" This reverts commit 17de7b28768e77d842bc89b26abad44f96331c7a. --- src/Interpreters/TreeRewriter.cpp | 2 +- ...2367_optimize_trivial_count_with_array_join.reference | 1 + .../02367_optimize_trivial_count_with_array_join.sql | 9 +++++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference create mode 100644 tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 73410a39ffd..5b10f7e53c3 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1025,7 +1025,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select has_explicit_columns = !required.empty(); if (is_select && !has_explicit_columns) { - optimize_trivial_count = true; + optimize_trivial_count = !columns_context.has_array_join; /// You need to read at least one column to find the number of rows. /// We will find a column with minimum . diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql new file mode 100644 index 00000000000..8d812c56ffb --- /dev/null +++ b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql @@ -0,0 +1,9 @@ +drop table if exists t; + +create table t(id UInt32) engine MergeTree order by id; + +insert into t values (1); + +select count() from t array join range(2) as a settings optimize_trivial_count_query = 1; + +drop table t; From ac96f29ae10cc8cdb2dd666fc9a06abc0db83596 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 9 Sep 2022 15:00:14 -0300 Subject: [PATCH 506/582] make the test to trick fuzzer --- .../02367_optimize_trivial_count_with_array_join.sql | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql index 8d812c56ffb..a3fb46e9eda 100644 --- a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql +++ b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql @@ -1,9 +1,11 @@ drop table if exists t; +drop table if exists t1; -create table t(id UInt32) engine MergeTree order by id; +create table t(id UInt32) engine MergeTree order by id as select 1; -insert into t values (1); +create table t1(a Array(UInt32)) ENGINE = MergeTree ORDER BY tuple() as select [1,2]; -select count() from t array join range(2) as a settings optimize_trivial_count_query = 1; +select count() from t array join (select a from t1) AS _a settings optimize_trivial_count_query=1; drop table t; +drop table t1; From 8c5583d7a54ac98dea980eea1b4a0a6b0e2f1a07 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 9 Sep 2022 14:48:32 +0200 Subject: [PATCH 507/582] Fix stack-use-after-return in GetPriorityForLoadBalancing::getPriorityFunc() clang-15 reports [1]:
ASan report ``` ==1==ERROR: AddressSanitizer: stack-use-after-return on address 0x7f1d04c4eb20 at pc 0x000031c4803c bp 0x7f1d05e19a00 sp 0x7f1d05e199f8 READ of size 8 at 0x7f1d04c4eb20 thread T200 (QueryPullPipeEx) #0 0x31c4803b in DB::GetPriorityForLoadBalancing::getPriorityFunc(DB::LoadBalancing, unsigned long, unsigned long) const::$_3::operator()(unsigned long) const build_docker/../src/Common/GetPriorityForLoadBalancing.cpp:42:40 #1 0x31c4803b in decltype(static_cast(fp)(static_cast(fp0))) std::__1::__invoke(DB::GetPriorityForLoadBalancing::getPriorityFunc(DB::LoadBalancing, unsigned long, unsigned long) const::$_3&, unsigned long&&) build_docker/../contrib/libcxx/include/type_traits:3640:23 #2 0x31c4803b in unsigned long std::__1::__invoke_void_return_wrapper::__call(DB::GetPriorityForLoadBalancing::getPriorityFunc(DB::LoadBalancing, unsigned long, unsigned long) const::$_3&, unsigned long&&) build_docker/../contrib/libcxx/include/__functional/invoke.h:30:16 #3 0x31c4803b in std::__1::__function::__default_alloc_func::operator()(unsigned long&&) build_docker/../contrib/libcxx/include/__functional/function.h:230:12 #4 0x31c4803b in unsigned long std::__1::__function::__policy_invoker::__call_impl>(std::__1::__function::__policy_storage const*, unsigned long) build_docker/../contrib/libcxx/include/__functional/function.h:711:16 #5 0x31c38b07 in std::__1::__function::__policy_func::operator()(unsigned long&&) const build_docker/../contrib/libcxx/include/__functional/function.h:843:16 #6 0x31c38b07 in std::__1::function::operator()(unsigned long) const build_docker/../contrib/libcxx/include/__functional/function.h:1184:12 #7 0x31c38b07 in PoolWithFailoverBase::getShuffledPools(unsigned long, std::__1::function const&) build_docker/../src/Common/PoolWithFailoverBase.h:174:39 This frame has 2 object(s): [32, 40) 'pool_size.addr' <== Memory access at offset 32 is inside this variable [64, 88) 'ref.tmp' (line 18) ```
[1]: https://s3.amazonaws.com/clickhouse-test-reports/41046/adea92f847373d1fcfd733d8979c63024f9b80bf/integration_tests__asan__[1/3].html Signed-off-by: Azat Khuzhin --- src/Common/GetPriorityForLoadBalancing.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/GetPriorityForLoadBalancing.cpp b/src/Common/GetPriorityForLoadBalancing.cpp index d8e7566e891..5da60fb1bae 100644 --- a/src/Common/GetPriorityForLoadBalancing.cpp +++ b/src/Common/GetPriorityForLoadBalancing.cpp @@ -16,7 +16,7 @@ std::function GetPriorityForLoadBalancing::getPriorityFunc case LoadBalancing::NEAREST_HOSTNAME: if (hostname_differences.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_differences is not initialized"); - get_priority = [&](size_t i) { return hostname_differences[i]; }; + get_priority = [this](size_t i) { return hostname_differences[i]; }; break; case LoadBalancing::IN_ORDER: get_priority = [](size_t i) { return i; }; @@ -36,7 +36,7 @@ std::function GetPriorityForLoadBalancing::getPriorityFunc * last_used = 3 -> get_priority: 4 3 0 1 2 * ... * */ - get_priority = [&](size_t i) + get_priority = [this, pool_size](size_t i) { ++i; return i < last_used ? pool_size - i : i - last_used; From 50789126a83096f36c81df10fed9ecf5dbee30d0 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Fri, 9 Sep 2022 12:15:59 -0600 Subject: [PATCH 508/582] Update docs/en/sql-reference/statements/delete.md Co-authored-by: Ivan Blinkov --- docs/en/sql-reference/statements/delete.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 982e8a0e2da..b89be52606f 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -10,7 +10,7 @@ sidebar_label: DELETE DELETE FROM [db.]table [WHERE expr] ``` -For MergeTree tables, `DELETE FROM` performs a lightweight delete on the given table, which means that the deleted rows are marked as deleted immediately and deleted rows will be filtered out of all subsequent queries. The underlying data is permanently deleted whenever merges occur. +`DELETE FROM` removes rows from table `[db.]table` that match expression `expr`. The deleted rows are marked as deleted immediately and will be automatically filtered out of all subsequent queries. Cleanup of data happens asynchronously in background. This feature is only available for MergeTree table engine family. For example, the following query deletes all rows from the `hits` table where the `Title` column contains the text `hello`: From 9870957621dcaeeece4297d4db3f4bcb3675a996 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Fri, 9 Sep 2022 12:16:54 -0600 Subject: [PATCH 509/582] Update docs/en/sql-reference/statements/delete.md Co-authored-by: Ivan Blinkov --- docs/en/sql-reference/statements/delete.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index b89be52606f..07e70d1fd5d 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -28,7 +28,7 @@ SET allow_experimental_lightweight_delete = true; ::: -The [traditional way to delete rows](./alter/delete.md) in ClickHouse was to use `ALTER TABLE ... DELETE`, which is still a valid method for deleting rows. However, in most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. +An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. :::info Lightweight deletes are asynchronous by default. Set `mutations_sync` equal to 1 to wait for one replica to process the statement, and set `mutations_sync` to 2 to wait for all replicas. From 9d717d62e141e4db2a33a161c41548e79c2b26cd Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 9 Sep 2022 14:56:25 -0400 Subject: [PATCH 510/582] fix slug --- docs/zh/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/development/tests.md b/docs/zh/development/tests.md index e6d5cf66de9..6f1118e5e63 100644 --- a/docs/zh/development/tests.md +++ b/docs/zh/development/tests.md @@ -1,5 +1,5 @@ --- -slug: /en/development/tests +slug: /zh/development/tests sidebar_position: 70 sidebar_label: Testing title: ClickHouse Testing From b2b5091ffd04164ffcfee7986b0973c31e2fbc3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Sep 2022 22:01:52 +0300 Subject: [PATCH 511/582] Update 02070_join_on_disk.sql --- tests/queries/0_stateless/02070_join_on_disk.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02070_join_on_disk.sql b/tests/queries/0_stateless/02070_join_on_disk.sql index c25a7a1ffac..eabf31df25f 100644 --- a/tests/queries/0_stateless/02070_join_on_disk.sql +++ b/tests/queries/0_stateless/02070_join_on_disk.sql @@ -17,3 +17,5 @@ INSERT INTO join_on_disk SELECT number as id FROM numbers_mt(50000); INSERT INTO join_on_disk SELECT number as id FROM numbers_mt(1000); SELECT id FROM join_on_disk lhs LEFT JOIN (SELECT id FROM join_on_disk GROUP BY id) rhs USING (id) FORMAT Null; + +DROP TABLE join_on_disk; From d8cbd51333e0d3fef87497a1c683c8441ebdbe32 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 9 Sep 2022 19:26:14 +0000 Subject: [PATCH 512/582] Support relative path in Location header after http redirect --- src/IO/ReadWriteBufferFromHTTP.h | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 63496054e85..60885da6da3 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -26,6 +26,7 @@ #include #include +#include namespace ProfileEvents { @@ -346,13 +347,29 @@ namespace detail non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); } + Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) + { + auto location = response.get("Location"); + auto location_uri = Poco::URI(location); + if (!location_uri.isRelative()) + return location_uri; + /// Location header contains relative path. So we need to concatenate it + /// with path from the original URI and normalize it. + auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location); + location_uri = prev_uri; + location_uri.setPath(path); + return location_uri; + } + void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { call(response, method_, throw_on_all_errors); + Poco::URI prev_uri = uri; while (isRedirect(response.getStatus())) { - Poco::URI uri_redirect(response.get("Location")); + Poco::URI uri_redirect = getUriAfterRedirect(prev_uri, response); + prev_uri = uri_redirect; if (remote_host_filter) remote_host_filter->checkURL(uri_redirect); @@ -408,7 +425,7 @@ namespace detail while (isRedirect(response.getStatus())) { - Poco::URI uri_redirect(response.get("Location")); + Poco::URI uri_redirect = getUriAfterRedirect(saved_uri_redirect.value_or(uri), response); if (remote_host_filter) remote_host_filter->checkURL(uri_redirect); From 4af246a2e055772dab215187063131aa10ee37ec Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 9 Sep 2022 13:59:21 -0600 Subject: [PATCH 513/582] Feedback --- docs/en/sql-reference/statements/alter/delete.md | 5 +---- docs/en/sql-reference/statements/delete.md | 5 +++-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index 1cd6d466788..ba5d01d9b4d 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -12,12 +12,9 @@ ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr Deletes data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). -:::info -The `ALTER TABLE` command is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](../delete.md), which performs a lightweight delete and can be considerably faster. -::: :::note -The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. +The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. `ALTER TABLE` is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](../delete.md), which performs a lightweight delete and can be considerably faster. ::: The `filter_expr` must be of type `UInt8`. The query deletes rows in the table for which this expression takes a non-zero value. diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 07e70d1fd5d..492c59f525d 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -18,6 +18,7 @@ For example, the following query deletes all rows from the `hits` table where th DELETE FROM hits WHERE Title LIKE '%hello%'; ``` +Lightweight deletes are asynchronous by default. Set `mutations_sync` equal to 1 to wait for one replica to process the statement, and set `mutations_sync` to 2 to wait for all replicas. :::note This feature is experimental and requires you to set `allow_experimental_lightweight_delete` to true: @@ -30,7 +31,7 @@ SET allow_experimental_lightweight_delete = true; An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. -:::info -Lightweight deletes are asynchronous by default. Set `mutations_sync` equal to 1 to wait for one replica to process the statement, and set `mutations_sync` to 2 to wait for all replicas. +:::warn +Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on OLTP system. Ligthweight deletes are currently efficient for wide parts, but for compact parts they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. ::: From 79ccce23e9190546a44593b4a794c40473527c54 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Sep 2022 20:09:54 +0000 Subject: [PATCH 514/582] Automatic style fix --- tests/integration/helpers/cluster.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9cc515f98fc..e2cad8436a8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -792,7 +792,9 @@ class ClickHouseCluster: binary_dir = os.path.dirname(self.server_bin_path) # always prefer clickhouse-keeper standalone binary - if os.path.exists(os.path.join(binary_dir, "clickhouse-keeper")) and not os.path.islink(os.path.join(binary_dir, "clickhouse-keeper")): + if os.path.exists( + os.path.join(binary_dir, "clickhouse-keeper") + ) and not os.path.islink(os.path.join(binary_dir, "clickhouse-keeper")): binary_path = os.path.join(binary_dir, "clickhouse-keeper") keeper_cmd_prefix = "clickhouse-keeper" else: From 726639484ac653d029420cee757e72805a8bbfb1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Fri, 9 Sep 2022 22:15:38 +0200 Subject: [PATCH 515/582] Revert "Query plan optimization setting: read in window order" --- src/Interpreters/IInterpreterUnionOrSelectQuery.h | 2 -- src/Interpreters/InterpreterExplainQuery.cpp | 4 ++-- src/Processors/QueryPlan/Optimizations/Optimizations.h | 2 +- .../Optimizations/QueryPlanOptimizationSettings.cpp | 1 - .../Optimizations/QueryPlanOptimizationSettings.h | 3 --- .../reuseStorageOrderingForWindowFunctions.cpp | 7 ++++++- 6 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 6f893d4703e..a1c86f9de85 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -58,8 +58,6 @@ public: /// Add limits from external query. void addStorageLimits(const StorageLimitsList & limits); - ContextPtr getContext() const { return context; } - protected: ASTPtr query_ptr; ContextMutablePtr context; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 746d382198d..4799970b6a1 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -316,7 +316,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() interpreter.buildQueryPlan(plan); if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(interpreter.getContext())); + plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); if (settings.json) { @@ -326,7 +326,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto plan_array = std::make_unique(); plan_array->add(std::move(plan_map)); - auto format_settings = getFormatSettings(interpreter.getContext()); + auto format_settings = getFormatSettings(getContext()); format_settings.json.quote_64bit_integers = false; JSONBuilder::FormatSettings json_format_settings{.settings = format_settings}; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index f45200f3026..904f30e84b0 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -63,7 +63,7 @@ inline const auto & getOptimizations() {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::optimize_plan}, {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan}, - {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_read_in_window_order} + {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan} }}; return optimizations; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index f9707b973e4..1472fb87a89 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -11,7 +11,6 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.optimize_plan = from.query_plan_enable_optimizations; settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply; settings.filter_push_down = from.query_plan_filter_push_down; - settings.optimize_read_in_window_order = from.optimize_read_in_window_order; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 99e52b60a73..b5a37bf69d6 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -21,9 +21,6 @@ struct QueryPlanOptimizationSettings /// If filter push down optimization is enabled. bool filter_push_down = true; - /// window functions read in order optimization - bool optimize_read_in_window_order = true; - static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 8377b62c947..401774b390e 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -61,7 +61,12 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, return 0; } - const auto context = read_from_merge_tree->getContext(); + auto context = read_from_merge_tree->getContext(); + if (!context->getSettings().optimize_read_in_window_order) + { + return 0; + } + const auto & query_info = read_from_merge_tree->getQueryInfo(); const auto * select_query = query_info.query->as(); From b9e9d776f0b91291e00d37ab4a4ecb096c4a6dbf Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Fri, 9 Sep 2022 15:55:57 -0600 Subject: [PATCH 516/582] Update delete.md --- docs/en/sql-reference/statements/delete.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 492c59f525d..487dfc87f9a 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -31,7 +31,7 @@ SET allow_experimental_lightweight_delete = true; An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. -:::warn +:::warning Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on OLTP system. Ligthweight deletes are currently efficient for wide parts, but for compact parts they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. ::: From 4301b362abda3f8e6ebca8c3371cceaa8c7e120b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 02:34:37 +0200 Subject: [PATCH 517/582] Fix strange code in DatabaseReplicated --- src/Databases/DatabaseReplicated.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1e2c21ecfbd..c6ef3867b63 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -811,7 +811,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep /// Also we have to commit metadata transaction, because it's not committed by default for inner tables of MVs. /// Yep, I hate inner tables of materialized views. auto mv_drop_inner_table_context = make_query_context(); - table->dropInnerTableIfAny(sync, mv_drop_inner_table_context); + table->dropInnerTableIfAny(/* sync */ true, mv_drop_inner_table_context); mv_drop_inner_table_context->getZooKeeperMetadataTransaction()->commit(); } From daff20156692f3e4285279575ce32ccd6b463226 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 03:28:37 +0200 Subject: [PATCH 518/582] Fix something terribly wrong in LowCardinality --- src/Columns/ColumnLowCardinality.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 4fa6a0be4ed..7cd226c4c11 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -68,9 +68,9 @@ public: UInt64 get64(size_t n) const override { return getDictionary().get64(getIndexes().getUInt(n)); } UInt64 getUInt(size_t n) const override { return getDictionary().getUInt(getIndexes().getUInt(n)); } Int64 getInt(size_t n) const override { return getDictionary().getInt(getIndexes().getUInt(n)); } - Float64 getFloat64(size_t n) const override { return getDictionary().getInt(getIndexes().getFloat64(n)); } - Float32 getFloat32(size_t n) const override { return getDictionary().getInt(getIndexes().getFloat32(n)); } - bool getBool(size_t n) const override { return getDictionary().getInt(getIndexes().getBool(n)); } + Float64 getFloat64(size_t n) const override { return getDictionary().getFloat64(getIndexes().getUInt(n)); } + Float32 getFloat32(size_t n) const override { return getDictionary().getFloat32(getIndexes().getUInt(n)); } + bool getBool(size_t n) const override { return getDictionary().getBool(getIndexes().getUInt(n)); } bool isNullAt(size_t n) const override { return getDictionary().isNullAt(getIndexes().getUInt(n)); } ColumnPtr cut(size_t start, size_t length) const override { From fa62c7e98243e052061a2458713918aba7548f09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 04:07:51 +0200 Subject: [PATCH 519/582] Fix half of trash --- base/base/JSON.cpp | 35 +++------- base/base/StringRef.h | 6 +- base/base/getThreadId.cpp | 2 +- base/base/shift10.cpp | 68 +++++++++---------- base/base/shift10.h | 4 +- base/base/terminalColors.cpp | 10 +-- base/base/wide_integer_impl.h | 6 +- programs/obfuscator/Obfuscator.cpp | 4 +- src/AggregateFunctions/QuantileTDigest.h | 7 +- src/AggregateFunctions/QuantileTiming.h | 4 +- src/Client/QueryFuzzer.cpp | 2 +- src/Columns/ColumnAggregateFunction.cpp | 2 +- src/Columns/IColumnImpl.h | 2 +- src/Columns/MaskOperations.cpp | 3 +- src/Common/BitHelpers.h | 2 +- src/Common/DateLUTImpl.h | 46 +++++++------ src/Common/ErrorCodes.cpp | 2 +- src/Common/Exception.h | 2 +- src/Common/MemoryTracker.cpp | 6 +- src/Common/SLRUCachePolicy.h | 2 +- src/Common/SipHash.h | 2 +- src/Common/StudentTTest.h | 2 +- src/Common/ThreadFuzzer.cpp | 18 ++--- src/Common/ThreadProfileEvents.cpp | 4 +- src/Common/Throttler.cpp | 2 +- src/Common/UnicodeBar.cpp | 7 +- src/Common/VersionNumber.cpp | 14 ++-- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeperArgs.h | 4 +- src/Common/checkStackSize.cpp | 2 +- src/Common/getNumberOfPhysicalCPUCores.cpp | 8 +-- src/Common/mysqlxx/mysqlxx/Value.h | 1 - src/Core/Field.h | 9 ++- src/Core/MySQL/MySQLClient.cpp | 8 +-- src/Daemon/GraphiteWriter.h | 2 +- .../Serializations/SerializationString.cpp | 4 +- src/Functions/FunctionsLogical.cpp | 2 +- src/IO/MySQLPacketPayloadWriteBuffer.cpp | 2 +- src/IO/readFloatText.h | 8 +-- src/Interpreters/ProcessList.cpp | 2 +- src/Interpreters/SortedBlocksWriter.cpp | 4 +- .../Formats/Impl/AvroRowInputFormat.cpp | 26 +++---- src/Storages/MergeTree/LevelMergeSelector.cpp | 4 +- .../MergeTree/SimpleMergeSelector.cpp | 4 +- 44 files changed, 164 insertions(+), 192 deletions(-) diff --git a/base/base/JSON.cpp b/base/base/JSON.cpp index 92350ea0e18..315bcce38da 100644 --- a/base/base/JSON.cpp +++ b/base/base/JSON.cpp @@ -22,7 +22,7 @@ POCO_IMPLEMENT_EXCEPTION(JSONException, Poco::Exception, "JSONException") // NOL #endif -/// Прочитать беззнаковое целое в простом формате из не-0-terminated строки. +/// Read unsigned integer in a simple form from a non-0-terminated string. static UInt64 readUIntText(const char * buf, const char * end) { UInt64 x = 0; @@ -59,7 +59,7 @@ static UInt64 readUIntText(const char * buf, const char * end) } -/// Прочитать знаковое целое в простом формате из не-0-terminated строки. +/// Read signed integer in a simple form from a non-0-terminated string. static Int64 readIntText(const char * buf, const char * end) { bool negative = false; @@ -102,7 +102,7 @@ static Int64 readIntText(const char * buf, const char * end) } -/// Прочитать число с плавающей запятой в простом формате, с грубым округлением, из не-0-terminated строки. +/// Read floating point number in simple format, imprecisely, from a non-0-terminated string. static double readFloatText(const char * buf, const char * end) { bool negative = false; @@ -151,8 +151,8 @@ static double readFloatText(const char * buf, const char * end) case 'E': { ++buf; - Int32 exponent = readIntText(buf, end); - x *= preciseExp10(exponent); + auto exponent = readIntText(buf, end); + x *= preciseExp10(static_cast(exponent)); run = false; break; @@ -207,7 +207,7 @@ JSON::ElementType JSON::getType() const return TYPE_NUMBER; case '"': { - /// Проверим - это просто строка или name-value pair + /// Is it a string or a name-value pair? Pos after_string = skipString(); if (after_string < ptr_end && *after_string == ':') return TYPE_NAME_VALUE_PAIR; @@ -229,15 +229,13 @@ void JSON::checkPos(Pos pos) const JSON::Pos JSON::skipString() const { - //std::cerr << "skipString()\t" << data() << std::endl; - Pos pos = ptr_begin; checkPos(pos); if (*pos != '"') throw JSONException(std::string("JSON: expected \", got ") + *pos); ++pos; - /// fast path: находим следующую двойную кавычку. Если перед ней нет бэкслеша - значит это конец строки (при допущении корректности JSON). + /// fast path: find next double quote. If it is not escaped by backslash - then it's an end of string (assuming JSON is valid). Pos closing_quote = reinterpret_cast(memchr(reinterpret_cast(pos), '\"', ptr_end - pos)); if (nullptr != closing_quote && closing_quote[-1] != '\\') return closing_quote + 1; @@ -269,8 +267,6 @@ JSON::Pos JSON::skipString() const JSON::Pos JSON::skipNumber() const { - //std::cerr << "skipNumber()\t" << data() << std::endl; - Pos pos = ptr_begin; checkPos(pos); @@ -296,8 +292,6 @@ JSON::Pos JSON::skipNumber() const JSON::Pos JSON::skipBool() const { - //std::cerr << "skipBool()\t" << data() << std::endl; - Pos pos = ptr_begin; checkPos(pos); @@ -314,16 +308,12 @@ JSON::Pos JSON::skipBool() const JSON::Pos JSON::skipNull() const { - //std::cerr << "skipNull()\t" << data() << std::endl; - return ptr_begin + 4; } JSON::Pos JSON::skipNameValuePair() const { - //std::cerr << "skipNameValuePair()\t" << data() << std::endl; - Pos pos = skipString(); checkPos(pos); @@ -338,8 +328,6 @@ JSON::Pos JSON::skipNameValuePair() const JSON::Pos JSON::skipArray() const { - //std::cerr << "skipArray()\t" << data() << std::endl; - if (!isArray()) throw JSONException("JSON: expected ["); Pos pos = ptr_begin; @@ -370,8 +358,6 @@ JSON::Pos JSON::skipArray() const JSON::Pos JSON::skipObject() const { - //std::cerr << "skipObject()\t" << data() << std::endl; - if (!isObject()) throw JSONException("JSON: expected {"); Pos pos = ptr_begin; @@ -402,8 +388,6 @@ JSON::Pos JSON::skipObject() const JSON::Pos JSON::skipElement() const { - //std::cerr << "skipElement()\t" << data() << std::endl; - ElementType type = getType(); switch (type) @@ -640,7 +624,7 @@ std::string JSON::getString() const { throw JSONException("JSON: incorrect syntax: incorrect HEX code."); } - buf.resize(buf.size() + 6); /// максимальный размер UTF8 многобайтовой последовательности + buf.resize(buf.size() + 6); /// Max size of UTF-8 sequence, including pre-standard mapping of UCS-4 to UTF-8. int res = utf8.convert(unicode, reinterpret_cast(const_cast(buf.data())) + buf.size() - 6, 6); if (!res) @@ -754,8 +738,6 @@ JSON::iterator JSON::iterator::begin() const if (type != TYPE_ARRAY && type != TYPE_OBJECT) throw JSONException("JSON: not array or object when calling begin() method."); - //std::cerr << "begin()\t" << data() << std::endl; - Pos pos = ptr_begin + 1; checkPos(pos); if (*pos == '}' || *pos == ']') @@ -846,4 +828,3 @@ bool JSON::isType() const { return isBool(); } - diff --git a/base/base/StringRef.h b/base/base/StringRef.h index 14ca2530a33..9a368f21e7d 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -227,7 +227,7 @@ inline UInt64 shiftMix(UInt64 val) return val ^ (val >> 47); } -inline UInt64 rotateByAtLeast1(UInt64 val, int shift) +inline UInt64 rotateByAtLeast1(UInt64 val, UInt8 shift) { return (val >> shift) | (val << (64 - shift)); } @@ -249,7 +249,7 @@ inline size_t hashLessThan8(const char * data, size_t size) uint8_t b = data[size >> 1]; uint8_t c = data[size - 1]; uint32_t y = static_cast(a) + (static_cast(b) << 8); - uint32_t z = size + (static_cast(c) << 2); + uint32_t z = static_cast(size) + (static_cast(c) << 2); return shiftMix(y * k2 ^ z * k3) * k2; } @@ -262,7 +262,7 @@ inline size_t hashLessThan16(const char * data, size_t size) { UInt64 a = unalignedLoad(data); UInt64 b = unalignedLoad(data + size - 8); - return hashLen16(a, rotateByAtLeast1(b + size, size)) ^ b; + return hashLen16(a, rotateByAtLeast1(b + size, static_cast(size))) ^ b; } return hashLessThan8(data, size); diff --git a/base/base/getThreadId.cpp b/base/base/getThreadId.cpp index 21167b19d32..b6c22bb8856 100644 --- a/base/base/getThreadId.cpp +++ b/base/base/getThreadId.cpp @@ -22,7 +22,7 @@ uint64_t getThreadId() #if defined(OS_ANDROID) current_tid = gettid(); #elif defined(OS_LINUX) - current_tid = syscall(SYS_gettid); /// This call is always successful. - man gettid + current_tid = static_cast(syscall(SYS_gettid)); /// This call is always successful. - man gettid #elif defined(OS_FREEBSD) current_tid = pthread_getthreadid_np(); #elif defined(OS_SUNOS) diff --git a/base/base/shift10.cpp b/base/base/shift10.cpp index db2f4a6963e..286c3be85f0 100644 --- a/base/base/shift10.cpp +++ b/base/base/shift10.cpp @@ -14,37 +14,37 @@ static T shift10Impl(T x, int exponent) static const long double powers10[] = { 1e-323L, 1e-322L, 1e-321L, 1e-320L, 1e-319L, 1e-318L, 1e-317L, 1e-316L, 1e-315L, 1e-314L, 1e-313L, 1e-312L, 1e-311L, - 1e-310L,1e-309L,1e-308L,1e-307L,1e-306L,1e-305L,1e-304L,1e-303L,1e-302L,1e-301L,1e-300L,1e-299L,1e-298L,1e-297L,1e-296L,1e-295L,1e-294L,1e-293L,1e-292L,1e-291L, - 1e-290L,1e-289L,1e-288L,1e-287L,1e-286L,1e-285L,1e-284L,1e-283L,1e-282L,1e-281L,1e-280L,1e-279L,1e-278L,1e-277L,1e-276L,1e-275L,1e-274L,1e-273L,1e-272L,1e-271L, - 1e-270L,1e-269L,1e-268L,1e-267L,1e-266L,1e-265L,1e-264L,1e-263L,1e-262L,1e-261L,1e-260L,1e-259L,1e-258L,1e-257L,1e-256L,1e-255L,1e-254L,1e-253L,1e-252L,1e-251L, - 1e-250L,1e-249L,1e-248L,1e-247L,1e-246L,1e-245L,1e-244L,1e-243L,1e-242L,1e-241L,1e-240L,1e-239L,1e-238L,1e-237L,1e-236L,1e-235L,1e-234L,1e-233L,1e-232L,1e-231L, - 1e-230L,1e-229L,1e-228L,1e-227L,1e-226L,1e-225L,1e-224L,1e-223L,1e-222L,1e-221L,1e-220L,1e-219L,1e-218L,1e-217L,1e-216L,1e-215L,1e-214L,1e-213L,1e-212L,1e-211L, - 1e-210L,1e-209L,1e-208L,1e-207L,1e-206L,1e-205L,1e-204L,1e-203L,1e-202L,1e-201L,1e-200L,1e-199L,1e-198L,1e-197L,1e-196L,1e-195L,1e-194L,1e-193L,1e-192L,1e-191L, - 1e-190L,1e-189L,1e-188L,1e-187L,1e-186L,1e-185L,1e-184L,1e-183L,1e-182L,1e-181L,1e-180L,1e-179L,1e-178L,1e-177L,1e-176L,1e-175L,1e-174L,1e-173L,1e-172L,1e-171L, - 1e-170L,1e-169L,1e-168L,1e-167L,1e-166L,1e-165L,1e-164L,1e-163L,1e-162L,1e-161L,1e-160L,1e-159L,1e-158L,1e-157L,1e-156L,1e-155L,1e-154L,1e-153L,1e-152L,1e-151L, - 1e-150L,1e-149L,1e-148L,1e-147L,1e-146L,1e-145L,1e-144L,1e-143L,1e-142L,1e-141L,1e-140L,1e-139L,1e-138L,1e-137L,1e-136L,1e-135L,1e-134L,1e-133L,1e-132L,1e-131L, - 1e-130L,1e-129L,1e-128L,1e-127L,1e-126L,1e-125L,1e-124L,1e-123L,1e-122L,1e-121L,1e-120L,1e-119L,1e-118L,1e-117L,1e-116L,1e-115L,1e-114L,1e-113L,1e-112L,1e-111L, - 1e-110L,1e-109L,1e-108L,1e-107L,1e-106L,1e-105L,1e-104L,1e-103L,1e-102L,1e-101L,1e-100L,1e-99L,1e-98L,1e-97L,1e-96L,1e-95L,1e-94L,1e-93L,1e-92L,1e-91L,1e-90L, - 1e-89L,1e-88L,1e-87L,1e-86L,1e-85L,1e-84L,1e-83L,1e-82L,1e-81L,1e-80L,1e-79L,1e-78L,1e-77L,1e-76L,1e-75L,1e-74L,1e-73L,1e-72L,1e-71L,1e-70, - 1e-69L,1e-68L,1e-67L,1e-66L,1e-65L,1e-64L,1e-63L,1e-62L,1e-61L,1e-60L,1e-59L,1e-58L,1e-57L,1e-56L,1e-55L,1e-54L,1e-53L,1e-52L,1e-51L,1e-50, - 1e-49L,1e-48L,1e-47L,1e-46L,1e-45L,1e-44L,1e-43L,1e-42L,1e-41L,1e-40L,1e-39L,1e-38L,1e-37L,1e-36L,1e-35L,1e-34L,1e-33L,1e-32L,1e-31L,1e-30, - 1e-29L,1e-28L,1e-27L,1e-26L,1e-25L,1e-24L,1e-23L,1e-22L,1e-21L,1e-20L,1e-19L,1e-18L,1e-17L,1e-16L,1e-15L,1e-14L,1e-13L,1e-12L,1e-11L,1e-10, - 1e-9L,1e-8L,1e-7L,1e-6L,1e-5L,1e-4L,1e-3L,1e-2L,1e-1L,1e0L,1e1L,1e2L,1e3L,1e4L,1e5L,1e6L,1e7L,1e8L,1e9L,1e10, - 1e11L,1e12L,1e13L,1e14L,1e15L,1e16L,1e17L,1e18L,1e19L,1e20L,1e21L,1e22L,1e23L,1e24L,1e25L,1e26L,1e27L,1e28L,1e29L,1e30, - 1e31L,1e32L,1e33L,1e34L,1e35L,1e36L,1e37L,1e38L,1e39L,1e40L,1e41L,1e42L,1e43L,1e44L,1e45L,1e46L,1e47L,1e48L,1e49L,1e50, - 1e51L,1e52L,1e53L,1e54L,1e55L,1e56L,1e57L,1e58L,1e59L,1e60L,1e61L,1e62L,1e63L,1e64L,1e65L,1e66L,1e67L,1e68L,1e69L,1e70, - 1e71L,1e72L,1e73L,1e74L,1e75L,1e76L,1e77L,1e78L,1e79L,1e80L,1e81L,1e82L,1e83L,1e84L,1e85L,1e86L,1e87L,1e88L,1e89L,1e90, - 1e91L,1e92L,1e93L,1e94L,1e95L,1e96L,1e97L,1e98L,1e99L,1e100L,1e101L,1e102L,1e103L,1e104L,1e105L,1e106L,1e107L,1e108L,1e109L,1e110, - 1e111L,1e112L,1e113L,1e114L,1e115L,1e116L,1e117L,1e118L,1e119L,1e120L,1e121L,1e122L,1e123L,1e124L,1e125L,1e126L,1e127L,1e128L,1e129L,1e130, - 1e131L,1e132L,1e133L,1e134L,1e135L,1e136L,1e137L,1e138L,1e139L,1e140L,1e141L,1e142L,1e143L,1e144L,1e145L,1e146L,1e147L,1e148L,1e149L,1e150, - 1e151L,1e152L,1e153L,1e154L,1e155L,1e156L,1e157L,1e158L,1e159L,1e160L,1e161L,1e162L,1e163L,1e164L,1e165L,1e166L,1e167L,1e168L,1e169L,1e170, - 1e171L,1e172L,1e173L,1e174L,1e175L,1e176L,1e177L,1e178L,1e179L,1e180L,1e181L,1e182L,1e183L,1e184L,1e185L,1e186L,1e187L,1e188L,1e189L,1e190, - 1e191L,1e192L,1e193L,1e194L,1e195L,1e196L,1e197L,1e198L,1e199L,1e200L,1e201L,1e202L,1e203L,1e204L,1e205L,1e206L,1e207L,1e208L,1e209L,1e210, - 1e211L,1e212L,1e213L,1e214L,1e215L,1e216L,1e217L,1e218L,1e219L,1e220L,1e221L,1e222L,1e223L,1e224L,1e225L,1e226L,1e227L,1e228L,1e229L,1e230, - 1e231L,1e232L,1e233L,1e234L,1e235L,1e236L,1e237L,1e238L,1e239L,1e240L,1e241L,1e242L,1e243L,1e244L,1e245L,1e246L,1e247L,1e248L,1e249L,1e250, - 1e251L,1e252L,1e253L,1e254L,1e255L,1e256L,1e257L,1e258L,1e259L,1e260L,1e261L,1e262L,1e263L,1e264L,1e265L,1e266L,1e267L,1e268L,1e269L,1e270, - 1e271L,1e272L,1e273L,1e274L,1e275L,1e276L,1e277L,1e278L,1e279L,1e280L,1e281L,1e282L,1e283L,1e284L,1e285L,1e286L,1e287L,1e288L,1e289L,1e290, - 1e291L,1e292L,1e293L,1e294L,1e295L,1e296L,1e297L,1e298L,1e299L,1e300L,1e301L,1e302L,1e303L,1e304L,1e305L,1e306L,1e307L,1e308L + 1e-310L, 1e-309L, 1e-308L, 1e-307L, 1e-306L, 1e-305L, 1e-304L, 1e-303L, 1e-302L, 1e-301L, 1e-300L, 1e-299L, 1e-298L, 1e-297L, 1e-296L, 1e-295L, 1e-294L, 1e-293L, 1e-292L, 1e-291L, + 1e-290L, 1e-289L, 1e-288L, 1e-287L, 1e-286L, 1e-285L, 1e-284L, 1e-283L, 1e-282L, 1e-281L, 1e-280L, 1e-279L, 1e-278L, 1e-277L, 1e-276L, 1e-275L, 1e-274L, 1e-273L, 1e-272L, 1e-271L, + 1e-270L, 1e-269L, 1e-268L, 1e-267L, 1e-266L, 1e-265L, 1e-264L, 1e-263L, 1e-262L, 1e-261L, 1e-260L, 1e-259L, 1e-258L, 1e-257L, 1e-256L, 1e-255L, 1e-254L, 1e-253L, 1e-252L, 1e-251L, + 1e-250L, 1e-249L, 1e-248L, 1e-247L, 1e-246L, 1e-245L, 1e-244L, 1e-243L, 1e-242L, 1e-241L, 1e-240L, 1e-239L, 1e-238L, 1e-237L, 1e-236L, 1e-235L, 1e-234L, 1e-233L, 1e-232L, 1e-231L, + 1e-230L, 1e-229L, 1e-228L, 1e-227L, 1e-226L, 1e-225L, 1e-224L, 1e-223L, 1e-222L, 1e-221L, 1e-220L, 1e-219L, 1e-218L, 1e-217L, 1e-216L, 1e-215L, 1e-214L, 1e-213L, 1e-212L, 1e-211L, + 1e-210L, 1e-209L, 1e-208L, 1e-207L, 1e-206L, 1e-205L, 1e-204L, 1e-203L, 1e-202L, 1e-201L, 1e-200L, 1e-199L, 1e-198L, 1e-197L, 1e-196L, 1e-195L, 1e-194L, 1e-193L, 1e-192L, 1e-191L, + 1e-190L, 1e-189L, 1e-188L, 1e-187L, 1e-186L, 1e-185L, 1e-184L, 1e-183L, 1e-182L, 1e-181L, 1e-180L, 1e-179L, 1e-178L, 1e-177L, 1e-176L, 1e-175L, 1e-174L, 1e-173L, 1e-172L, 1e-171L, + 1e-170L, 1e-169L, 1e-168L, 1e-167L, 1e-166L, 1e-165L, 1e-164L, 1e-163L, 1e-162L, 1e-161L, 1e-160L, 1e-159L, 1e-158L, 1e-157L, 1e-156L, 1e-155L, 1e-154L, 1e-153L, 1e-152L, 1e-151L, + 1e-150L, 1e-149L, 1e-148L, 1e-147L, 1e-146L, 1e-145L, 1e-144L, 1e-143L, 1e-142L, 1e-141L, 1e-140L, 1e-139L, 1e-138L, 1e-137L, 1e-136L, 1e-135L, 1e-134L, 1e-133L, 1e-132L, 1e-131L, + 1e-130L, 1e-129L, 1e-128L, 1e-127L, 1e-126L, 1e-125L, 1e-124L, 1e-123L, 1e-122L, 1e-121L, 1e-120L, 1e-119L, 1e-118L, 1e-117L, 1e-116L, 1e-115L, 1e-114L, 1e-113L, 1e-112L, 1e-111L, + 1e-110L, 1e-109L, 1e-108L, 1e-107L, 1e-106L, 1e-105L, 1e-104L, 1e-103L, 1e-102L, 1e-101L, 1e-100L, 1e-99L, 1e-98L, 1e-97L, 1e-96L, 1e-95L, 1e-94L, 1e-93L, 1e-92L, 1e-91L, 1e-90L, + 1e-89L, 1e-88L, 1e-87L, 1e-86L, 1e-85L, 1e-84L, 1e-83L, 1e-82L, 1e-81L, 1e-80L, 1e-79L, 1e-78L, 1e-77L, 1e-76L, 1e-75L, 1e-74L, 1e-73L, 1e-72L, 1e-71L, 1e-70, + 1e-69L, 1e-68L, 1e-67L, 1e-66L, 1e-65L, 1e-64L, 1e-63L, 1e-62L, 1e-61L, 1e-60L, 1e-59L, 1e-58L, 1e-57L, 1e-56L, 1e-55L, 1e-54L, 1e-53L, 1e-52L, 1e-51L, 1e-50, + 1e-49L, 1e-48L, 1e-47L, 1e-46L, 1e-45L, 1e-44L, 1e-43L, 1e-42L, 1e-41L, 1e-40L, 1e-39L, 1e-38L, 1e-37L, 1e-36L, 1e-35L, 1e-34L, 1e-33L, 1e-32L, 1e-31L, 1e-30, + 1e-29L, 1e-28L, 1e-27L, 1e-26L, 1e-25L, 1e-24L, 1e-23L, 1e-22L, 1e-21L, 1e-20L, 1e-19L, 1e-18L, 1e-17L, 1e-16L, 1e-15L, 1e-14L, 1e-13L, 1e-12L, 1e-11L, 1e-10, + 1e-9L, 1e-8L, 1e-7L, 1e-6L, 1e-5L, 1e-4L, 1e-3L, 1e-2L, 1e-1L, 1e0L, 1e1L, 1e2L, 1e3L, 1e4L, 1e5L, 1e6L, 1e7L, 1e8L, 1e9L, 1e10, + 1e11L, 1e12L, 1e13L, 1e14L, 1e15L, 1e16L, 1e17L, 1e18L, 1e19L, 1e20L, 1e21L, 1e22L, 1e23L, 1e24L, 1e25L, 1e26L, 1e27L, 1e28L, 1e29L, 1e30, + 1e31L, 1e32L, 1e33L, 1e34L, 1e35L, 1e36L, 1e37L, 1e38L, 1e39L, 1e40L, 1e41L, 1e42L, 1e43L, 1e44L, 1e45L, 1e46L, 1e47L, 1e48L, 1e49L, 1e50, + 1e51L, 1e52L, 1e53L, 1e54L, 1e55L, 1e56L, 1e57L, 1e58L, 1e59L, 1e60L, 1e61L, 1e62L, 1e63L, 1e64L, 1e65L, 1e66L, 1e67L, 1e68L, 1e69L, 1e70, + 1e71L, 1e72L, 1e73L, 1e74L, 1e75L, 1e76L, 1e77L, 1e78L, 1e79L, 1e80L, 1e81L, 1e82L, 1e83L, 1e84L, 1e85L, 1e86L, 1e87L, 1e88L, 1e89L, 1e90, + 1e91L, 1e92L, 1e93L, 1e94L, 1e95L, 1e96L, 1e97L, 1e98L, 1e99L, 1e100L, 1e101L, 1e102L, 1e103L, 1e104L, 1e105L, 1e106L, 1e107L, 1e108L, 1e109L, 1e110, + 1e111L, 1e112L, 1e113L, 1e114L, 1e115L, 1e116L, 1e117L, 1e118L, 1e119L, 1e120L, 1e121L, 1e122L, 1e123L, 1e124L, 1e125L, 1e126L, 1e127L, 1e128L, 1e129L, 1e130, + 1e131L, 1e132L, 1e133L, 1e134L, 1e135L, 1e136L, 1e137L, 1e138L, 1e139L, 1e140L, 1e141L, 1e142L, 1e143L, 1e144L, 1e145L, 1e146L, 1e147L, 1e148L, 1e149L, 1e150, + 1e151L, 1e152L, 1e153L, 1e154L, 1e155L, 1e156L, 1e157L, 1e158L, 1e159L, 1e160L, 1e161L, 1e162L, 1e163L, 1e164L, 1e165L, 1e166L, 1e167L, 1e168L, 1e169L, 1e170, + 1e171L, 1e172L, 1e173L, 1e174L, 1e175L, 1e176L, 1e177L, 1e178L, 1e179L, 1e180L, 1e181L, 1e182L, 1e183L, 1e184L, 1e185L, 1e186L, 1e187L, 1e188L, 1e189L, 1e190, + 1e191L, 1e192L, 1e193L, 1e194L, 1e195L, 1e196L, 1e197L, 1e198L, 1e199L, 1e200L, 1e201L, 1e202L, 1e203L, 1e204L, 1e205L, 1e206L, 1e207L, 1e208L, 1e209L, 1e210, + 1e211L, 1e212L, 1e213L, 1e214L, 1e215L, 1e216L, 1e217L, 1e218L, 1e219L, 1e220L, 1e221L, 1e222L, 1e223L, 1e224L, 1e225L, 1e226L, 1e227L, 1e228L, 1e229L, 1e230, + 1e231L, 1e232L, 1e233L, 1e234L, 1e235L, 1e236L, 1e237L, 1e238L, 1e239L, 1e240L, 1e241L, 1e242L, 1e243L, 1e244L, 1e245L, 1e246L, 1e247L, 1e248L, 1e249L, 1e250, + 1e251L, 1e252L, 1e253L, 1e254L, 1e255L, 1e256L, 1e257L, 1e258L, 1e259L, 1e260L, 1e261L, 1e262L, 1e263L, 1e264L, 1e265L, 1e266L, 1e267L, 1e268L, 1e269L, 1e270, + 1e271L, 1e272L, 1e273L, 1e274L, 1e275L, 1e276L, 1e277L, 1e278L, 1e279L, 1e280L, 1e281L, 1e282L, 1e283L, 1e284L, 1e285L, 1e286L, 1e287L, 1e288L, 1e289L, 1e290, + 1e291L, 1e292L, 1e293L, 1e294L, 1e295L, 1e296L, 1e297L, 1e298L, 1e299L, 1e300L, 1e301L, 1e302L, 1e303L, 1e304L, 1e305L, 1e306L, 1e307L, 1e308L }; if (unlikely(exponent < min_exponent)) /// Note: there are some values below MIN_EXPONENT that is greater than zero. @@ -52,7 +52,7 @@ static T shift10Impl(T x, int exponent) else if (unlikely(exponent > max_exponent)) x *= std::numeric_limits::infinity(); /// Multiplying to keep the sign of infinity. else - x *= powers10[exponent - min_exponent]; + x *= static_cast(powers10[exponent - min_exponent]); return x; } @@ -68,12 +68,12 @@ float shift10(float x, int exponent) return shift10Impl(x, exponent); } -double shift10(UInt64 x, int exponent) +long double shift10(UInt64 x, int exponent) { return shift10Impl(static_cast(x), exponent); } -double shift10(Int64 x, int exponent) +long double shift10(Int64 x, int exponent) { return shift10Impl(static_cast(x), exponent); } diff --git a/base/base/shift10.h b/base/base/shift10.h index 0f54d1f2988..2332194591b 100644 --- a/base/base/shift10.h +++ b/base/base/shift10.h @@ -12,5 +12,5 @@ double shift10(double x, int exponent); float shift10(float x, int exponent); -double shift10(UInt64 x, int exponent); -double shift10(Int64 x, int exponent); +long double shift10(UInt64 x, int exponent); +long double shift10(Int64 x, int exponent); diff --git a/base/base/terminalColors.cpp b/base/base/terminalColors.cpp index 4c2013c6499..fbb0e28a193 100644 --- a/base/base/terminalColors.cpp +++ b/base/base/terminalColors.cpp @@ -11,12 +11,12 @@ std::string setColor(UInt64 hash) /// It still looks awesome. UInt8 y = 128; - UInt8 cb = hash % 256; - UInt8 cr = hash / 256 % 256; + UInt8 cb = static_cast(hash % 256); + UInt8 cr = static_cast(hash / 256 % 256); - UInt8 r = std::max(0.0, std::min(255.0, y + 1.402 * (cr - 128))); - UInt8 g = std::max(0.0, std::min(255.0, y - 0.344136 * (cb - 128) - 0.714136 * (cr - 128))); - UInt8 b = std::max(0.0, std::min(255.0, y + 1.772 * (cb - 128))); + UInt8 r = static_cast(std::max(0.0, std::min(255.0, y + 1.402 * (cr - 128)))); + UInt8 g = static_cast(std::max(0.0, std::min(255.0, y - 0.344136 * (cb - 128) - 0.714136 * (cr - 128)))); + UInt8 b = static_cast(std::max(0.0, std::min(255.0, y + 1.772 * (cb - 128)))); /// ANSI escape sequence to set 24-bit foreground font color in terminal. return "\033[38;2;" + std::to_string(r) + ";" + std::to_string(g) + ";" + std::to_string(b) + "m"; diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 5e32b286871..eb2edcb98ff 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -453,7 +453,7 @@ private: if constexpr (sizeof(T) <= sizeof(base_type)) { if (0 == idx) - return x; + return static_cast(x); } else if (idx * sizeof(base_type) < sizeof(T)) return x >> (idx * base_bits); // & std::numeric_limits::max() @@ -1239,13 +1239,13 @@ constexpr integer::operator long double() const noexcept template constexpr integer::operator double() const noexcept { - return static_cast(*this); + return static_cast(static_cast(*this)); } template constexpr integer::operator float() const noexcept { - return static_cast(*this); + return static_cast(static_cast(*this)); } // Unary operators diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 95bf89b0255..bdf26c9e730 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -833,7 +833,7 @@ public: } } - if (params.frequency_desaturate) + if (params.frequency_desaturate > 0.0) { for (auto & elem : table) { @@ -846,7 +846,7 @@ public: UInt64 new_total = 0; for (auto & bucket : histogram.buckets) { - bucket.second = bucket.second * (1.0 - params.frequency_desaturate) + average * params.frequency_desaturate; + bucket.second = static_cast(bucket.second * (1.0 - params.frequency_desaturate) + average * params.frequency_desaturate); new_total += bucket.second; } diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index b5f32bad247..b61e4874d00 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -81,7 +81,7 @@ class QuantileTDigest */ struct Params { - Value epsilon = 0.01; + Value epsilon = 0.01f; size_t max_centroids = 2048; size_t max_unmerged = 2048; }; @@ -99,12 +99,11 @@ class QuantileTDigest BetterFloat count = 0; size_t unmerged = 0; - /** Linear interpolation at the point x on the line (x1, y1)..(x2, y2) - */ + /// Linear interpolation at the point x on the line (x1, y1)..(x2, y2) static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2) { /// Symmetric interpolation for better results with infinities. - double k = (x - x1) / (x2 - x1); + Value k = (x - x1) / (x2 - x1); return (1 - k) * y1 + k * y2; } diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index c89d1b66f5b..092e2d76951 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -178,7 +178,7 @@ namespace detail if (!elems.empty()) { size_t n = level < 1 - ? level * elems.size() + ? static_cast(level * elems.size()) : (elems.size() - 1); /// Sorting an array will not be considered a violation of constancy. @@ -201,7 +201,7 @@ namespace detail auto level = levels[level_index]; size_t n = level < 1 - ? level * elems.size() + ? static_cast(level * elems.size()) : (elems.size() - 1); ::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 787fad5990a..9b404e7c5b7 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -54,7 +54,7 @@ Field QueryFuzzer::getRandomField(int type) } case 1: { - static constexpr float values[] + static constexpr double values[] = {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999, 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20, FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 55f851cb7c6..61c5ded0594 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -607,7 +607,7 @@ MutableColumns ColumnAggregateFunction::scatter(IColumn::ColumnIndex num_columns size_t num_rows = size(); { - size_t reserve_size = static_cast(num_rows) / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule. + size_t reserve_size = static_cast(static_cast(num_rows) / num_columns * 1.1); /// 1.1 is just a guess. Better to use n-sigma rule. if (reserve_size > 1) for (auto & column : columns) diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index e90503cbad2..642f242abfa 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -54,7 +54,7 @@ std::vector IColumn::scatterImpl(ColumnIndex num_columns, column = cloneEmpty(); { - size_t reserve_size = num_rows * 1.1 / num_columns; /// 1.1 is just a guess. Better to use n-sigma rule. + size_t reserve_size = static_cast(num_rows * 1.1 / num_columns); /// 1.1 is just a guess. Better to use n-sigma rule. if (reserve_size > 1) for (auto & column : columns) diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 9e2d02253be..3120828921f 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -119,7 +119,7 @@ size_t extractMaskNumericImpl( (*nulls)[i] = 1; } else - value = !!data[index]; + value = static_cast(data[index]); if constexpr (inverted) value = !value; @@ -335,4 +335,3 @@ void copyMask(const PaddedPODArray & from, PaddedPODArray & to) } } - diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index 0e9e39cb1b2..6a3efb58c42 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -31,7 +31,7 @@ inline size_t roundUpToPowerOfTwoOrZero(size_t n) template -inline size_t getLeadingZeroBitsUnsafe(T x) +inline uint32_t getLeadingZeroBitsUnsafe(T x) { assert(x != 0); diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index f9361e809a9..f5504749684 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -577,7 +577,7 @@ public: /// also make the special timezones with no whole hour offset such as 'Australia/Lord_Howe' been taken into account. LUTIndex index = findIndex(t); - UInt32 time = t - lut[index].date; + UInt32 time = static_cast(t - lut[index].date); if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); @@ -618,33 +618,33 @@ public: } template - inline unsigned toMonth(DateOrTime v) const { return lut[toLUTIndex(v)].month; } + inline UInt8 toMonth(DateOrTime v) const { return lut[toLUTIndex(v)].month; } template - inline unsigned toQuarter(DateOrTime v) const { return (lut[toLUTIndex(v)].month - 1) / 3 + 1; } + inline UInt8 toQuarter(DateOrTime v) const { return (lut[toLUTIndex(v)].month - 1) / 3 + 1; } template inline Int16 toYear(DateOrTime v) const { return lut[toLUTIndex(v)].year; } template - inline unsigned toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; } + inline UInt8 toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; } template - inline unsigned toDayOfMonth(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_month; } + inline UInt8 toDayOfMonth(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_month; } template - inline unsigned toDayOfYear(DateOrTime v) const + inline UInt16 toDayOfYear(DateOrTime v) const { // TODO: different overload for ExtendedDayNum const LUTIndex i = toLUTIndex(v); - return i + 1 - toFirstDayNumOfYearIndex(i); + return static_cast(i + 1 - toFirstDayNumOfYearIndex(i)); } /// Number of week from some fixed moment in the past. Week begins at monday. /// (round down to monday and divide DayNum by 7; we made an assumption, /// that in domain of the function there was no weeks with any other number of days than 7) template - inline unsigned toRelativeWeekNum(DateOrTime v) const + inline Int32 toRelativeWeekNum(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); /// We add 8 to avoid underflow at beginning of unix epoch. @@ -653,7 +653,7 @@ public: /// Get year that contains most of the current week. Week begins at monday. template - inline unsigned toISOYear(DateOrTime v) const + inline Int16 toISOYear(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); /// That's effectively the year of thursday of current week. @@ -694,7 +694,7 @@ public: /// ISO 8601 week number. Week begins at monday. /// The week number 1 is the first week in year that contains 4 or more days (that's more than half). template - inline unsigned toISOWeek(DateOrTime v) const + inline UInt8 toISOWeek(DateOrTime v) const { return 1 + (toFirstDayNumOfWeek(v) - toDayNum(toFirstDayNumOfISOYearIndex(v))) / 7; } @@ -751,38 +751,40 @@ public: YearWeek yw(toYear(i), 0); UInt16 days = 0; - const auto daynr = makeDayNum(yw.first, toMonth(i), toDayOfMonth(i)); - auto first_daynr = makeDayNum(yw.first, 1, 1); + const auto day_number = makeDayNum(yw.first, toMonth(i), toDayOfMonth(i)); + auto first_day_number = makeDayNum(yw.first, 1, 1); // 0 for monday, 1 for tuesday ... // get weekday from first day in year. - UInt16 weekday = calc_weekday(first_daynr, !monday_first_mode); + UInt8 weekday = calc_weekday(first_day_number, !monday_first_mode); if (toMonth(i) == 1 && toDayOfMonth(i) <= static_cast(7 - weekday)) { if (!week_year_mode && ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4))) return yw; week_year_mode = true; - (yw.first)--; - first_daynr -= (days = calc_days_in_year(yw.first)); + --yw.first; + days = calc_days_in_year(yw.first); + first_day_number -= days; weekday = (weekday + 53 * 7 - days) % 7; } if ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4)) - days = daynr - (first_daynr + (7 - weekday)); + days = day_number - (first_day_number + (7 - weekday)); else - days = daynr - (first_daynr - weekday); + days = day_number - (first_day_number - weekday); if (week_year_mode && days >= 52 * 7) { weekday = (weekday + calc_days_in_year(yw.first)) % 7; if ((!first_weekday_mode && weekday < 4) || (first_weekday_mode && weekday == 0)) { - (yw.first)++; + ++yw.first; yw.second = 1; return yw; } } + yw.second = days / 7 + 1; return yw; } @@ -853,7 +855,7 @@ public: * Returns 0 for monday, 1 for tuesday... */ template - inline unsigned calc_weekday(DateOrTime v, bool sunday_first_day_of_week) const /// NOLINT + inline UInt8 calc_weekday(DateOrTime v, bool sunday_first_day_of_week) const /// NOLINT { const LUTIndex i = toLUTIndex(v); if (!sunday_first_day_of_week) @@ -863,21 +865,21 @@ public: } /// Calculate days in one year. - inline unsigned calc_days_in_year(Int32 year) const /// NOLINT + inline UInt16 calc_days_in_year(Int32 year) const /// NOLINT { return ((year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)) ? 366 : 365); } /// Number of month from some fixed moment in the past (year * 12 + month) template - inline unsigned toRelativeMonthNum(DateOrTime v) const + inline Int32 toRelativeMonthNum(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); return lut[i].year * 12 + lut[i].month; } template - inline unsigned toRelativeQuarterNum(DateOrTime v) const + inline Int32 toRelativeQuarterNum(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); return lut[i].year * 4 + (lut[i].month - 1) / 3; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f65711a8521..8f46cea25a0 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -674,7 +674,7 @@ namespace ErrorCodes ErrorCode getErrorCodeByName(std::string_view error_name) { - for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) + for (int i = 0, end = ErrorCodes::end(); i < end; ++i) { std::string_view name = ErrorCodes::getName(i); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 9d59fb4d7b5..c5259d157b2 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -127,7 +127,7 @@ public: #endif ; - int getLineNumber() const { return line_number; } + ssize_t getLineNumber() const { return line_number; } void setLineNumber(int line_number_) { line_number = line_number_;} String getFileName() const { return file_name; } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 29dbcdd28b0..cebec219ff2 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -178,7 +178,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size); @@ -186,7 +186,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution fault(fault_probability); - if (unlikely(fault_probability && fault(thread_local_rng))) + if (unlikely(fault_probability > 0.0 && fault(thread_local_rng))) { if (memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded) { @@ -318,7 +318,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size); diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 8d4709c66a7..e1d72aa630a 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -35,7 +35,7 @@ public: */ /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). SLRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) - : max_protected_size(max_size_ * std::min(1.0, size_ratio)) + : max_protected_size(static_cast(max_size_ * std::min(1.0, size_ratio))) , max_size(max_size_) , max_elements_size(max_elements_size_) { diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 1f26cba31fb..6162de48143 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -55,7 +55,7 @@ private: ALWAYS_INLINE void finalize() { /// In the last free byte, we write the remainder of the division by 256. - current_bytes[7] = cnt; + current_bytes[7] = static_cast(cnt); v3 ^= current_word; SIPROUND; diff --git a/src/Common/StudentTTest.h b/src/Common/StudentTTest.h index b09190050b5..26c44ebc8c8 100644 --- a/src/Common/StudentTTest.h +++ b/src/Common/StudentTTest.h @@ -31,7 +31,7 @@ private: double avg() const { - return sum / size; + return sum / static_cast(size); } double var() const diff --git a/src/Common/ThreadFuzzer.cpp b/src/Common/ThreadFuzzer.cpp index cbfbfc5d690..16b51c8bbce 100644 --- a/src/Common/ThreadFuzzer.cpp +++ b/src/Common/ThreadFuzzer.cpp @@ -157,22 +157,22 @@ bool ThreadFuzzer::isEffective() const #if THREAD_FUZZER_WRAP_PTHREAD # define CHECK_WRAPPER_PARAMS(RET, NAME, ...) \ - if (NAME##_before_yield_probability.load(std::memory_order_relaxed)) \ + if (NAME##_before_yield_probability.load(std::memory_order_relaxed) > 0.0) \ return true; \ - if (NAME##_before_migrate_probability.load(std::memory_order_relaxed)) \ + if (NAME##_before_migrate_probability.load(std::memory_order_relaxed) > 0.0) \ return true; \ - if (NAME##_before_sleep_probability.load(std::memory_order_relaxed)) \ + if (NAME##_before_sleep_probability.load(std::memory_order_relaxed) > 0.0) \ return true; \ - if (NAME##_before_sleep_time_us.load(std::memory_order_relaxed)) \ + if (NAME##_before_sleep_time_us.load(std::memory_order_relaxed) > 0.0) \ return true; \ \ - if (NAME##_after_yield_probability.load(std::memory_order_relaxed)) \ + if (NAME##_after_yield_probability.load(std::memory_order_relaxed) > 0.0) \ return true; \ - if (NAME##_after_migrate_probability.load(std::memory_order_relaxed)) \ + if (NAME##_after_migrate_probability.load(std::memory_order_relaxed) > 0.0) \ return true; \ - if (NAME##_after_sleep_probability.load(std::memory_order_relaxed)) \ + if (NAME##_after_sleep_probability.load(std::memory_order_relaxed) > 0.0) \ return true; \ - if (NAME##_after_sleep_time_us.load(std::memory_order_relaxed)) \ + if (NAME##_after_sleep_time_us.load(std::memory_order_relaxed) > 0.0) \ return true; FOR_EACH_WRAPPED_FUNCTION(CHECK_WRAPPER_PARAMS) @@ -239,7 +239,7 @@ static void injection( && sleep_time_us > 0 && std::bernoulli_distribution(sleep_probability)(thread_local_rng)) { - sleepForNanoseconds(sleep_time_us * 1000); + sleepForNanoseconds(static_cast(sleep_time_us * 1000)); } } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index dba65a138c3..82b03f6ed91 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -558,8 +558,8 @@ void PerfEventsCounters::finalizeProfileEvents(ProfileEvents::Counters & profile // deltas from old values. const auto enabled = current_value.time_enabled - previous_value.time_enabled; const auto running = current_value.time_running - previous_value.time_running; - const UInt64 delta = (current_value.value - previous_value.value) - * enabled / std::max(1.f, float(running)); + const UInt64 delta = static_cast( + (current_value.value - previous_value.value) * enabled / std::max(1.f, float(running))); if (min_enabled_time > enabled) { diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 95baf40f2c0..2c9279e21e1 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -66,7 +66,7 @@ void Throttler::add(size_t amount) if (max_speed && current_speed > max_speed) { /// If we was too fast then we have to sleep until our smoothed speed became <= max_speed - int64_t sleep_time = -window_ns * std::log2(max_speed / current_speed); + int64_t sleep_time = static_cast(-window_ns * std::log2(max_speed / current_speed)); if (sleep_time > 0) { diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp index 9cb3069aebb..efc85ad35e4 100644 --- a/src/Common/UnicodeBar.cpp +++ b/src/Common/UnicodeBar.cpp @@ -28,12 +28,12 @@ namespace UnicodeBar size_t getWidthInBytes(double width) { - return ceil(width - 1.0 / 8) * UNICODE_BAR_CHAR_SIZE; + return static_cast(ceil(width - 1.0 / 8) * UNICODE_BAR_CHAR_SIZE); } void render(double width, char * dst) { - size_t floor_width = floor(width); + size_t floor_width = static_cast(floor(width)); for (size_t i = 0; i < floor_width; ++i) { @@ -41,7 +41,7 @@ namespace UnicodeBar dst += UNICODE_BAR_CHAR_SIZE; } - size_t remainder = floor((width - floor_width) * 8); + size_t remainder = static_cast(floor((width - floor_width) * 8)); if (remainder) { @@ -59,4 +59,3 @@ namespace UnicodeBar return res; } } - diff --git a/src/Common/VersionNumber.cpp b/src/Common/VersionNumber.cpp index b25fd994947..c8332132d2b 100644 --- a/src/Common/VersionNumber.cpp +++ b/src/Common/VersionNumber.cpp @@ -40,23 +40,17 @@ int VersionNumber::compare(const VersionNumber & rhs) const size_t min = std::min(components.size(), rhs.components.size()); for (size_t i = 0; i < min; ++i) { - if (int d = components[i] - rhs.components[i]) - return d; + if (auto d = components[i] - rhs.components[i]) + return d > 0 ? 1 : -1; } if (components.size() > min) { - if (components[min] != 0) - return components[min]; - else - return 1; + return components[min] >= 0 ? 1 : -1; } else if (rhs.components.size() > min) { - if (rhs.components[min] != 0) - return -rhs.components[min]; - else - return -1; + return -rhs.components[min] > 0 ? 1 : -1; } return 0; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 12aa4471ad7..1c7ba7f1d9c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -319,7 +319,7 @@ public: void setZooKeeperLog(std::shared_ptr zk_log_); - UInt32 getSessionUptime() const { return session_uptime.elapsedSeconds(); } + UInt32 getSessionUptime() const { return static_cast(session_uptime.elapsedSeconds()); } private: friend class EphemeralNodeHolder; diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index b5c7b293506..48c8e4386f8 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -28,8 +28,8 @@ struct ZooKeeperArgs int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; - float send_fault_probability = 0; - float recv_fault_probability = 0; + double send_fault_probability = 0.0; + double recv_fault_probability = 0.0; DB::GetPriorityForLoadBalancing get_priority_load_balancing; }; diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index d6edb8596b7..9d76452df43 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -101,7 +101,7 @@ __attribute__((__weak__)) void checkStackSize() throw Exception("Logical error: frame address is greater than stack begin address", ErrorCodes::LOGICAL_ERROR); size_t stack_size = int_stack_address + max_stack_size - int_frame_address; - size_t max_stack_size_allowed = max_stack_size * STACK_SIZE_FREE_RATIO; + size_t max_stack_size_allowed = static_cast(max_stack_size * STACK_SIZE_FREE_RATIO); /// Just check if we have eat more than a STACK_SIZE_FREE_RATIO of stack size already. if (stack_size > max_stack_size_allowed) diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 0df7a83cd53..88542b3e705 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -9,7 +9,7 @@ #include #if defined(OS_LINUX) -static int readFrom(const char * filename, int default_value) +static int32_t readFrom(const char * filename, int default_value) { std::ifstream infile(filename); if (!infile.is_open()) @@ -22,15 +22,15 @@ static int readFrom(const char * filename, int default_value) } /// Try to look at cgroups limit if it is available. -static unsigned getCGroupLimitedCPUCores(unsigned default_cpu_count) +static uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count) { - unsigned quota_count = default_cpu_count; + uint32_t quota_count = default_cpu_count; /// Return the number of milliseconds per period process is guaranteed to run. /// -1 for no quota int cgroup_quota = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", -1); int cgroup_period = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_period_us", -1); if (cgroup_quota > -1 && cgroup_period > 0) - quota_count = ceil(static_cast(cgroup_quota) / static_cast(cgroup_period)); + quota_count = static_cast(ceil(static_cast(cgroup_quota) / static_cast(cgroup_period))); return std::min(default_cpu_count, quota_count); } diff --git a/src/Common/mysqlxx/mysqlxx/Value.h b/src/Common/mysqlxx/mysqlxx/Value.h index 2e6471692a2..46fcee0753f 100644 --- a/src/Common/mysqlxx/mysqlxx/Value.h +++ b/src/Common/mysqlxx/mysqlxx/Value.h @@ -248,7 +248,6 @@ template <> inline long Value::get() cons template <> inline unsigned long Value::get() const { return getUInt(); } /// NOLINT template <> inline long long Value::get() const { return getInt(); } /// NOLINT template <> inline unsigned long long Value::get() const { return getUInt(); } /// NOLINT -template <> inline float Value::get() const { return getDouble(); } template <> inline double Value::get() const { return getDouble(); } template <> inline std::string Value::get() const { return getString(); } template <> inline LocalDate Value::get() const { return getDate(); } diff --git a/src/Core/Field.h b/src/Core/Field.h index f60b7e4902e..17809c90fd1 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -885,25 +885,25 @@ inline char & Field::reinterpret() template T get(const Field & field) { - return field.template get(); + return static_cast(field.template get()); } template T get(Field & field) { - return field.template get(); + return static_cast(field.template get()); } template T safeGet(const Field & field) { - return field.template safeGet(); + return static_cast(field.template safeGet()); } template T safeGet(Field & field) { - return field.template safeGet(); + return static_cast(field.template safeGet()); } template @@ -1036,4 +1036,3 @@ struct fmt::formatter return format_to(ctx.out(), "{}", toString(x)); } }; - diff --git a/src/Core/MySQL/MySQLClient.cpp b/src/Core/MySQL/MySQLClient.cpp index b83acfe5e98..2f2d6e11a8b 100644 --- a/src/Core/MySQL/MySQLClient.cpp +++ b/src/Core/MySQL/MySQLClient.cpp @@ -42,9 +42,9 @@ void MySQLClient::connect() disconnect(); } - const Poco::Timespan connection_timeout(10 * 1e9); - const Poco::Timespan receive_timeout(5 * 1e9); - const Poco::Timespan send_timeout(5 * 1e9); + const Poco::Timespan connection_timeout(10'000'000'000); + const Poco::Timespan receive_timeout(5'000'000'000); + const Poco::Timespan send_timeout(5'000'000'000); socket = std::make_unique(); address = DNSResolver::instance().resolveAddress(host, port); @@ -152,7 +152,7 @@ void MySQLClient::startBinlogDumpGTID(UInt32 slave_id, String replicate_db, std: setBinlogChecksum(binlog_checksum); /// Set heartbeat 1s. - UInt64 period_ns = (1 * 1e9); + UInt64 period_ns = 1'000'000'000; writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns)); // Register slave. diff --git a/src/Daemon/GraphiteWriter.h b/src/Daemon/GraphiteWriter.h index 268c1a37b83..903385863e8 100644 --- a/src/Daemon/GraphiteWriter.h +++ b/src/Daemon/GraphiteWriter.h @@ -43,7 +43,7 @@ private: { Poco::Net::SocketAddress socket_address(host, port); Poco::Net::StreamSocket socket(socket_address); - socket.setSendTimeout(Poco::Timespan(timeout * 1000000)); + socket.setSendTimeout(Poco::Timespan(static_cast(timeout * 1000000))); Poco::Net::SocketStream str(socket); out(str, data, timestamp, custom_root_path); diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index e07fd4f26cf..2d2ae59574c 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -165,7 +165,7 @@ void SerializationString::deserializeBinaryBulk(IColumn & column, ReadBuffer & i double avg_chars_size = 1; /// By default reserve only for empty strings. - if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0])) + if (avg_value_size_hint > 0.0 && avg_value_size_hint > sizeof(offsets[0])) { /// Randomly selected. constexpr auto avg_value_size_hint_reserve_multiplier = 1.2; @@ -173,7 +173,7 @@ void SerializationString::deserializeBinaryBulk(IColumn & column, ReadBuffer & i avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier; } - size_t size_to_reserve = data.size() + std::ceil(limit * avg_chars_size); + size_t size_to_reserve = data.size() + static_cast(std::ceil(limit * avg_chars_size)); /// Never reserve for too big size. if (size_to_reserve < 256 * 1024 * 1024) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 63a2818a0c0..2ac7688737f 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -386,7 +386,7 @@ struct TypedExecutorInvoker std::transform( x.getData().cbegin(), x.getData().cend(), column->getData().cbegin(), result.begin(), - [](const auto a, const auto b) { return Op::apply(!!a, !!b); }); + [](const auto a, const auto b) { return Op::apply(static_cast(a), static_cast(b)); }); else TypedExecutorInvoker::template apply(x, y, result); } diff --git a/src/IO/MySQLPacketPayloadWriteBuffer.cpp b/src/IO/MySQLPacketPayloadWriteBuffer.cpp index 3d803dfa070..25ce9a9fcdf 100644 --- a/src/IO/MySQLPacketPayloadWriteBuffer.cpp +++ b/src/IO/MySQLPacketPayloadWriteBuffer.cpp @@ -44,7 +44,7 @@ void MySQLPacketPayloadWriteBuffer::setWorkingBuffer() void MySQLPacketPayloadWriteBuffer::nextImpl() { - const int written = pos - working_buffer.begin(); + size_t written = pos - working_buffer.begin(); if (eof) throw Exception("Cannot write after end of buffer.", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index b6be7adbbee..b8d0c1ba3c0 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -354,7 +354,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) if (unlikely(read_digits > significant_digits)) { int before_point_additional_exponent = read_digits - significant_digits; - x = shift10(before_point, before_point_additional_exponent); + x = static_cast(shift10(before_point, before_point_additional_exponent)); } else { @@ -411,10 +411,10 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) } if (after_point) - x += shift10(after_point, after_point_exponent); + x += static_cast(shift10(after_point, after_point_exponent)); if (exponent) - x = shift10(x, exponent); + x = static_cast(shift10(x, exponent)); if (negative) x = -x; @@ -486,7 +486,7 @@ ReturnType readFloatTextSimpleImpl(T & x, ReadBuffer & buf) bool negative = false; x = 0; bool after_point = false; - double power_of_ten = 1; + T power_of_ten = 1; if (buf.eof()) throwReadAfterEOF(); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d0ffeda49e4..e21d3814086 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -224,7 +224,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as } thread_group->memory_tracker.setDescription("(for query)"); - if (settings.memory_tracker_fault_probability) + if (settings.memory_tracker_fault_probability > 0.0) thread_group->memory_tracker.setFaultProbability(settings.memory_tracker_fault_probability); thread_group->memory_tracker.setOvercommitWaitingTime(settings.memory_usage_overcommit_max_wait_microseconds); diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 0acb056690f..82d501451a6 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -262,7 +262,7 @@ Pipe SortedBlocksWriter::streamFromFile(const TmpFilePtr & file) const Block SortedBlocksBuffer::exchange(Block && block) { - static constexpr const float reserve_coef = 1.2; + static constexpr const double reserve_coefficient = 1.2; Blocks out_blocks; Block empty_out = block.cloneEmpty(); @@ -282,7 +282,7 @@ Block SortedBlocksBuffer::exchange(Block && block) /// Not saved. Return buffered. out_blocks.swap(buffer); - buffer.reserve(out_blocks.size() * reserve_coef); + buffer.reserve(static_cast(out_blocks.size() * reserve_coefficient)); current_bytes = 0; } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 9ec39223484..a4e68ea530d 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -105,45 +105,45 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) switch (type.idx) { case TypeIndex::UInt8: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Date: [[fallthrough]]; case TypeIndex::UInt16: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::DateTime: [[fallthrough]]; case TypeIndex::UInt32: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::UInt64: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Int8: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Int16: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Int32: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Int64: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Float32: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Float64: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Decimal32: - assert_cast &>(column).insertValue(value); + assert_cast &>(column).insertValue(static_cast(value)); break; case TypeIndex::Decimal64: - assert_cast &>(column).insertValue(value); + assert_cast &>(column).insertValue(static_cast(value)); break; case TypeIndex::DateTime64: - assert_cast &>(column).insertValue(value); + assert_cast &>(column).insertValue(static_cast(value)); break; default: throw Exception("Type is not compatible with Avro", ErrorCodes::ILLEGAL_COLUMN); diff --git a/src/Storages/MergeTree/LevelMergeSelector.cpp b/src/Storages/MergeTree/LevelMergeSelector.cpp index 16947277463..67495c1372d 100644 --- a/src/Storages/MergeTree/LevelMergeSelector.cpp +++ b/src/Storages/MergeTree/LevelMergeSelector.cpp @@ -20,7 +20,7 @@ struct Estimator { double current_score = sum_size; - if (!min_score || current_score < min_score) + if (min_score == 0.0 || current_score < min_score) { min_score = current_score; best_begin = begin; @@ -33,7 +33,7 @@ struct Estimator return LevelMergeSelector::PartsRange(best_begin, best_end); } - double min_score = 0; + double min_score = 0.0; Iterator best_begin {}; Iterator best_end {}; }; diff --git a/src/Storages/MergeTree/SimpleMergeSelector.cpp b/src/Storages/MergeTree/SimpleMergeSelector.cpp index 434d44022df..3b71e2720c8 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.cpp +++ b/src/Storages/MergeTree/SimpleMergeSelector.cpp @@ -36,7 +36,7 @@ struct Estimator while (end >= begin + 3 && (end - 1)->size < settings.heuristic_to_remove_small_parts_at_right_max_ratio * sum_size) --end; - if (!min_score || current_score < min_score) + if (min_score == 0.0 || current_score < min_score) { min_score = current_score; best_begin = begin; @@ -69,7 +69,7 @@ struct Estimator return (sum_size + sum_size_fixed_cost * count) / (count - 1.9); } - double min_score = 0; + double min_score = 0.0; Iterator best_begin {}; Iterator best_end {}; }; From fd235919aae0f0c5ddc652ce65762141b2692007 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 05:04:40 +0200 Subject: [PATCH 520/582] Remove some methods --- .../AggregateFunctionSumMap.h | 4 +-- src/Columns/ColumnArray.cpp | 4 +-- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnMap.cpp | 4 +-- src/Columns/ColumnString.h | 2 +- src/Columns/ColumnTuple.cpp | 4 +-- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnVector.h | 2 +- src/Common/FieldVisitorSum.cpp | 3 +-- src/Common/FieldVisitorSum.h | 3 +-- src/Core/Field.h | 25 ------------------- .../Serializations/JSONDataParser.cpp | 4 +-- .../SerializationAggregateFunction.cpp | 4 +-- .../Serializations/SerializationArray.cpp | 4 +-- .../SerializationDecimalBase.cpp | 2 +- .../SerializationFixedString.cpp | 4 +-- .../Serializations/SerializationMap.cpp | 2 +- .../Serializations/SerializationNumber.cpp | 2 +- .../Serializations/SerializationString.cpp | 4 +-- .../Serializations/SerializationTuple.cpp | 4 +-- .../Serializations/SerializationUUID.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 10 ++++---- src/Dictionaries/MongoDBDictionarySource.cpp | 4 +-- src/Dictionaries/RedisDictionarySource.cpp | 2 +- src/Functions/array/arrayElement.cpp | 20 +++++++-------- src/Interpreters/ActionsVisitor.cpp | 6 ++--- .../InterpreterKillQueryQuery.cpp | 2 +- .../RewriteFunctionToSubcolumnVisitor.cpp | 4 +-- .../tests/gtest_comparison_graph.cpp | 4 +-- src/Parsers/ParserShowTablesQuery.cpp | 3 +-- .../Transforms/FillingTransform.cpp | 14 +++++------ src/Storages/LiveView/StorageLiveView.cpp | 2 +- .../MergeTreeIndexConditionBloomFilter.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 6 ++--- src/Storages/WindowView/StorageWindowView.cpp | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 4 +-- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 38 files changed, 74 insertions(+), 102 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index faac2f14e6c..90c73b826b2 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -519,7 +519,7 @@ private: template bool compareImpl(FieldType & x) const { - auto val = get(rhs); + auto val = rhs.get(); if (val > x) { x = val; @@ -554,7 +554,7 @@ private: template bool compareImpl(FieldType & x) const { - auto val = get(rhs); + auto val = rhs.get(); if (val < x) { x = val; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index fabdc03ace9..7bddfc14707 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const size, max_array_size_as_field); res = Array(); - Array & res_arr = DB::get(res); + Array & res_arr = res.get(); res_arr.reserve(size); for (size_t i = 0; i < size; ++i) @@ -296,7 +296,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const void ColumnArray::insert(const Field & x) { - const Array & array = DB::get(x); + const Array & array = x.get(); size_t size = array.size(); for (size_t i = 0; i < size; ++i) getData().insert(array[i]); diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index afae2cd641a..5634b9064bf 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -63,7 +63,7 @@ public: { data.resize_fill(data.size() + length); } - void insert(const Field & x) override { data.push_back(DB::get(x)); } + void insert(const Field & x) override { data.push_back(x.get()); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void popBack(size_t n) override diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 158d154f3f8..7038579d436 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const void ColumnFixedString::insert(const Field & x) { - const String & s = DB::get(x); + const String & s = x.get(); if (s.size() > n) throw Exception("Too large string '" + s + "' for FixedString column", ErrorCodes::TOO_LARGE_STRING_SIZE); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 06851f992ac..9956ba19455 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -74,7 +74,7 @@ void ColumnMap::get(size_t n, Field & res) const size_t size = offsets[n] - offsets[n - 1]; res = Map(); - auto & map = DB::get(res); + auto & map = res.get(); map.reserve(size); for (size_t i = 0; i < size; ++i) @@ -98,7 +98,7 @@ void ColumnMap::insertData(const char *, size_t) void ColumnMap::insert(const Field & x) { - const auto & map = DB::get(x); + const auto & map = x.get(); nested->insert(Array(map.begin(), map.end())); } diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 905b8a7ebef..361b792df55 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -128,7 +128,7 @@ public: void insert(const Field & x) override { - const String & s = DB::get(x); + const String & s = x.get(); const size_t old_size = chars.size(); const size_t size_to_append = s.size() + 1; const size_t new_size = old_size + size_to_append; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 6763c11bb9a..f023798e589 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -109,7 +109,7 @@ void ColumnTuple::get(size_t n, Field & res) const const size_t tuple_size = columns.size(); res = Tuple(); - Tuple & res_tuple = DB::get(res); + Tuple & res_tuple = res.get(); res_tuple.reserve(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -137,7 +137,7 @@ void ColumnTuple::insertData(const char *, size_t) void ColumnTuple::insert(const Field & x) { - const auto & tuple = DB::get(x); + const auto & tuple = x.get(); const size_t tuple_size = columns.size(); if (tuple.size() != tuple_size) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index d89917fc6c2..2db7fad2a50 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -918,7 +918,7 @@ ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, c auto res = this->create(); auto & res_data = res->getData(); - T default_value = safeGet(default_field); + T default_value = default_field.safeGet(); res_data.resize_fill(total_rows, default_value); for (size_t i = 0; i < offsets.size(); ++i) res_data[offsets[i]] = data[i + shift]; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 88e953891cc..ce480448713 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -301,7 +301,7 @@ public: void insert(const Field & x) override { - data.push_back(DB::get(x)); + data.push_back(x.get()); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index a8cb694d930..bc996ae2298 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const return x != 0; } -bool FieldVisitorSum::operator() (Float64 & x) const { x += get(rhs); return x != 0; } +bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get(); return x != 0; } bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); } bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } @@ -37,4 +37,3 @@ bool FieldVisitorSum::operator() (AggregateFunctionStateData &) const bool FieldVisitorSum::operator() (bool &) const { throw Exception("Cannot sum Bools", ErrorCodes::LOGICAL_ERROR); } } - diff --git a/src/Common/FieldVisitorSum.h b/src/Common/FieldVisitorSum.h index 247ef5b43da..cd8777e7bfb 100644 --- a/src/Common/FieldVisitorSum.h +++ b/src/Common/FieldVisitorSum.h @@ -33,7 +33,7 @@ public: template bool operator() (DecimalField & x) const { - x += get>(rhs); + x += rhs.get>(); return x.getValue() != T(0); } @@ -47,4 +47,3 @@ public: }; } - diff --git a/src/Core/Field.h b/src/Core/Field.h index f60b7e4902e..a0945b8315a 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -882,30 +882,6 @@ inline char & Field::reinterpret() return *reinterpret_cast(&storage); } -template -T get(const Field & field) -{ - return field.template get(); -} - -template -T get(Field & field) -{ - return field.template get(); -} - -template -T safeGet(const Field & field) -{ - return field.template safeGet(); -} - -template -T safeGet(Field & field) -{ - return field.template safeGet(); -} - template Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t) //-V730 { @@ -1036,4 +1012,3 @@ struct fmt::formatter return format_to(ctx.out(), "{}", toString(x)); } }; - diff --git a/src/DataTypes/Serializations/JSONDataParser.cpp b/src/DataTypes/Serializations/JSONDataParser.cpp index 598ba475697..e84197534a0 100644 --- a/src/DataTypes/Serializations/JSONDataParser.cpp +++ b/src/DataTypes/Serializations/JSONDataParser.cpp @@ -131,7 +131,7 @@ void JSONDataParser::traverseArrayElement(const Element & element, P auto nested_key = getNameOfNested(paths[i], values[i]); if (!nested_key.empty()) { - size_t array_size = get(values[i]).size(); + size_t array_size = values[i].template get().size(); auto & current_nested_sizes = ctx.nested_sizes_by_key[nested_key]; if (current_nested_sizes.size() == ctx.current_size) @@ -154,7 +154,7 @@ void JSONDataParser::traverseArrayElement(const Element & element, P auto nested_key = getNameOfNested(paths[i], values[i]); if (!nested_key.empty()) { - size_t array_size = get(values[i]).size(); + size_t array_size = values[i].template get().size(); auto & current_nested_sizes = ctx.nested_sizes_by_key[nested_key]; if (current_nested_sizes.empty()) diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 442df47a773..c8db1a56ed0 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -19,14 +19,14 @@ namespace DB void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const AggregateFunctionStateData & state = get(field); + const AggregateFunctionStateData & state = field.get(); writeBinary(state.data, ostr); } void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const { field = AggregateFunctionStateData(); - AggregateFunctionStateData & s = get(field); + AggregateFunctionStateData & s = field.get(); readBinary(s.data, istr); s.name = type_name; } diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 48980febd9d..abd99038e98 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const Array & a = get(field); + const Array & a = field.get(); writeVarUInt(a.size(), ostr); for (size_t i = 0; i < a.size(); ++i) { @@ -38,7 +38,7 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) con size_t size; readVarUInt(size, istr); field = Array(); - Array & arr = get(field); + Array & arr = field.get(); arr.reserve(size); for (size_t i = 0; i < size; ++i) nested->deserializeBinary(arr.emplace_back(), istr); diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index 8557c9ff719..00ffd607664 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -14,7 +14,7 @@ namespace DB template void SerializationDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr) const { - FieldType x = get>(field); + FieldType x = field.get>(); writeBinary(x, ostr); } diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index 972313a564f..9baaf95cb52 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const String & s = get(field); + const String & s = field.get(); ostr.write(s.data(), std::min(s.size(), n)); if (s.size() < n) for (size_t i = s.size(); i < n; ++i) @@ -37,7 +37,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr) const { field = String(); - String & s = get(field); + String & s = field.get(); s.resize(n); istr.readStrict(s.data(), n); } diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index e46bb480d14..958e33fbaf4 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -38,7 +38,7 @@ static IColumn & extractNestedColumn(IColumn & column) void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const auto & map = get(field); + const auto & map = field.get(); writeVarUInt(map.size(), ostr); for (const auto & elem : map) { diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 14c53dd5956..f1845fe9c80 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -105,7 +105,7 @@ template void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & ostr) const { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 - typename ColumnVector::ValueType x = get(field); + typename ColumnVector::ValueType x = field.get(); writeBinary(x, ostr); } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index e07fd4f26cf..61fcff35c92 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -22,7 +22,7 @@ namespace DB void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const String & s = get(field); + const String & s = field.get(); writeVarUInt(s.size(), ostr); writeString(s, ostr); } @@ -33,7 +33,7 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) co UInt64 size; readVarUInt(size, istr); field = String(); - String & s = get(field); + String & s = field.get(); s.resize(size); istr.readStrict(s.data(), size); } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 5663ff86dd6..c2c40cbb507 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -31,7 +31,7 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_ void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const auto & tuple = get(field); + const auto & tuple = field.get(); for (size_t element_index = 0; element_index < elems.size(); ++element_index) { const auto & serialization = elems[element_index]; @@ -44,7 +44,7 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) con const size_t size = elems.size(); field = Tuple(); - Tuple & tuple = get(field); + Tuple & tuple = field.get(); tuple.reserve(size); for (size_t i = 0; i < size; ++i) elems[i]->deserializeBinary(tuple.emplace_back(), istr); diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 36a060f2048..1fa9219a643 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -84,7 +84,7 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr) const { - UUID x = get(field); + UUID x = field.get(); writeBinary(x, ostr); } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 230b158b231..29f86a05016 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -633,7 +633,7 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = DB::get(rows_data[index]); + const Tuple & row_data = rows_data[index].get(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) @@ -673,7 +673,7 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = DB::get(rows_data[index]); + const Tuple & row_data = rows_data[index].get(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) @@ -695,7 +695,7 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = DB::get(rows_data[index]); + const Tuple & row_data = rows_data[index].get(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) @@ -709,7 +709,7 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = DB::get(rows_data[index]); + const Tuple & row_data = rows_data[index].get(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) @@ -761,7 +761,7 @@ static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t { writeable_rows_mask[index + 1] = true; writeable_rows_mask[index] = differenceSortingKeys( - DB::get(rows_data[index]), DB::get(rows_data[index + 1]), sorting_columns_index); + rows_data[index].get(), rows_data[index + 1].get(), sorting_columns_index); } for (size_t column = 0; column < buffer.columns() - 2; ++column) diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index a735f426ec7..0a8d632efcb 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -232,7 +232,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con } case AttributeUnderlyingType::String: { - String loaded_str(get((*key_columns[attribute_index])[row_idx])); + String loaded_str((*key_columns[attribute_index])[row_idx].get()); /// Convert string to ObjectID if (key_attribute.is_object_id) { @@ -259,7 +259,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con std::string MongoDBDictionarySource::toString() const { - return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + DB::toString(port); + return fmt::format("MongoDB: {}.{},{}{}:{}", db, collection, (user.empty() ? " " : " " + user + '@'), host, port); } } diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index ab5c758848a..b7aa72b253a 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -220,7 +220,7 @@ namespace DB if (isInteger(type)) key << DB::toString(key_columns[i]->get64(row)); else if (isString(type)) - key << get((*key_columns[i])[row]); + key << (*key_columns[i])[row].get(); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary"); } diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 7ffe71a7e5a..acd8f89ffe5 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -476,10 +476,10 @@ ColumnPtr FunctionArrayElement::executeNumberConst( auto col_res = ColumnVector::create(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && get(index) >= 0)) + || (index.getType() == Field::Types::Int64 && index.get() >= 0)) { ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), get(index) - 1, col_res->getData(), builder); + col_nested->getData(), col_array->getOffsets(), index.get() - 1, col_res->getData(), builder); } else if (index.getType() == Field::Types::Int64) { @@ -493,7 +493,7 @@ ColumnPtr FunctionArrayElement::executeNumberConst( /// arr[-2] is the element at offset 1 from the last and so on. ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), -(static_cast(safeGet(index)) + 1), col_res->getData(), builder); + col_nested->getData(), col_array->getOffsets(), -(static_cast(index.safeGet()) + 1), col_res->getData(), builder); } else throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); @@ -539,12 +539,12 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument auto col_res = ColumnString::create(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && get(index) >= 0)) + || (index.getType() == Field::Types::Int64 && index.get() >= 0)) ArrayElementStringImpl::vectorConst( col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - get(index) - 1, + index.get() - 1, col_res->getChars(), col_res->getOffsets(), builder); @@ -553,7 +553,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - -(UInt64(get(index)) + 1), + -(UInt64(index.get()) + 1), col_res->getChars(), col_res->getOffsets(), builder); @@ -603,12 +603,12 @@ ColumnPtr FunctionArrayElement::executeGenericConst( auto col_res = col_nested.cloneEmpty(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && get(index) >= 0)) + || (index.getType() == Field::Types::Int64 && index.get() >= 0)) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), get(index) - 1, *col_res, builder); + col_nested, col_array->getOffsets(), index.get() - 1, *col_res, builder); else if (index.getType() == Field::Types::Int64) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), -(static_cast(get(index) + 1)), *col_res, builder); + col_nested, col_array->getOffsets(), -(static_cast(index.get() + 1)), *col_res, builder); else throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); @@ -877,7 +877,7 @@ bool FunctionArrayElement::matchKeyToIndexStringConst( using DataColumn = std::decay_t; if (index.getType() != Field::Types::String) return false; - MatcherStringConst matcher{data_column, get(index)}; + MatcherStringConst matcher{data_column, index.get()}; executeMatchKeyToIndex(offsets, matched_idxs, matcher); return true; }); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 6c9e54a966d..f10510fadae 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -114,7 +114,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data throw Exception("Invalid type in set. Expected tuple, got " + String(value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET); - const auto & tuple = DB::get(value); + const auto & tuple = value.template get(); size_t tuple_size = tuple.size(); if (tuple_size != columns_num) @@ -306,9 +306,9 @@ Block createBlockForSet( { auto type_index = right_arg_type->getTypeId(); if (type_index == TypeIndex::Tuple) - block = createBlockFromCollection(DB::get(right_arg_value), set_element_types, tranform_null_in); + block = createBlockFromCollection(right_arg_value.get(), set_element_types, tranform_null_in); else if (type_index == TypeIndex::Array) - block = createBlockFromCollection(DB::get(right_arg_value), set_element_types, tranform_null_in); + block = createBlockFromCollection(right_arg_value.get(), set_element_types, tranform_null_in); else throw_unsupported_type(right_arg_type); } diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index f279244d769..1b4364351df 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -327,7 +327,7 @@ BlockIO InterpreterKillQueryQuery::execute() for (size_t i = 0; i < moves_block.rows(); ++i) { table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()}; - auto task_uuid = get(task_uuid_col[i]); + auto task_uuid = task_uuid_col[i].get(); CancellationCode code = CancellationCode::Unknown; diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp index a34c81d52e2..506fa13b7ba 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp @@ -111,11 +111,11 @@ void RewriteFunctionToSubcolumnData::visit(ASTFunction & function, ASTPtr & ast) if (value_type == Field::Types::UInt64) { const auto & type_tuple = assert_cast(*column_type); - auto index = get(literal->value); + auto index = literal->value.get(); subcolumn_name = type_tuple.getNameByPosition(index); } else if (value_type == Field::Types::String) - subcolumn_name = get(literal->value); + subcolumn_name = literal->value.get(); else return; diff --git a/src/Interpreters/tests/gtest_comparison_graph.cpp b/src/Interpreters/tests/gtest_comparison_graph.cpp index 6348bd13b8e..72e72b4b802 100644 --- a/src/Interpreters/tests/gtest_comparison_graph.cpp +++ b/src/Interpreters/tests/gtest_comparison_graph.cpp @@ -29,7 +29,7 @@ TEST(ComparisonGraph, Bounds) const auto & [lower, strict] = *res; - ASSERT_EQ(get(lower), 3); + ASSERT_EQ(lower.get(), 3); ASSERT_TRUE(strict); } @@ -39,7 +39,7 @@ TEST(ComparisonGraph, Bounds) const auto & [upper, strict] = *res; - ASSERT_EQ(get(upper), 7); + ASSERT_EQ(upper.get(), 7); ASSERT_TRUE(strict); } diff --git a/src/Parsers/ParserShowTablesQuery.cpp b/src/Parsers/ParserShowTablesQuery.cpp index 46c783b8271..858aafce847 100644 --- a/src/Parsers/ParserShowTablesQuery.cpp +++ b/src/Parsers/ParserShowTablesQuery.cpp @@ -177,12 +177,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec tryGetIdentifierNameInto(database, query->from); if (like) - query->like = safeGet(like->as().value); + query->like = like->as().value.safeGet(); node = query; return true; } - } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 311c88e46e8..786b28dc02e 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -40,7 +40,7 @@ static FillColumnDescription::StepFunction getStepFunction( { #define DECLARE_CASE(NAME) \ case IntervalKind::NAME: \ - return [step, scale, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(get(field), step, date_lut, scale); }; + return [step, scale, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(static_cast(field.get()), step, date_lut, scale); }; FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE @@ -106,21 +106,21 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & { if (which.isDate() || which.isDate32()) { - Int64 avg_seconds = get(descr.fill_step) * descr.step_kind->toAvgSeconds(); + Int64 avg_seconds = descr.fill_step.get() * descr.step_kind->toAvgSeconds(); if (std::abs(avg_seconds) < 86400) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds)); } if (which.isDate()) - descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), DateLUT::instance()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), DateLUT::instance()); else if (which.isDate32()) - descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), DateLUT::instance()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), DateLUT::instance()); else if (const auto * date_time = checkAndGetDataType(type.get())) - descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), date_time->getTimeZone()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), date_time->getTimeZone()); else if (const auto * date_time64 = checkAndGetDataType(type.get())) { - const auto & step_dec = get &>(descr.fill_step); + const auto & step_dec = descr.fill_step.get &>(); Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); switch (*descr.step_kind) @@ -129,7 +129,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & case IntervalKind::NAME: \ descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ { \ - auto field_decimal = get>(field); \ + auto field_decimal = field.get>(); \ auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, field_decimal.getScale()); \ field = DecimalField(res, field_decimal.getScale()); \ }; \ diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index b37dec6cbf1..4f52267a7fe 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -582,7 +582,7 @@ Pipe StorageLiveView::watch( if (query.limit_length) { has_limit = true; - limit = safeGet(typeid_cast(*query.limit_length).value); + limit = typeid_cast(*query.limit_length).value.safeGet(); } if (query.is_watch_events) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index a80f7093775..9d1a075a63f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -613,7 +613,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( if (which.isTuple() && function->name == "tuple") { - const Tuple & tuple = get(value_field); + const Tuple & tuple = value_field.get(); const auto * value_tuple_data_type = typeid_cast(value_type.get()); const ASTs & arguments = typeid_cast(*function->arguments).children; diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e52a0fed674..0e27f108cec 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -333,7 +333,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// Get path and name from engine arguments ast_zk_path = engine_args[arg_num]->as(); if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) - zookeeper_path = safeGet(ast_zk_path->value); + zookeeper_path = ast_zk_path->value.safeGet(); else throw Exception( "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), @@ -342,7 +342,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ast_replica_name = engine_args[arg_num]->as(); if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) - replica_name = safeGet(ast_replica_name->value); + replica_name = ast_replica_name->value.safeGet(); else throw Exception( "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); @@ -654,7 +654,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) - storage_settings->index_granularity = safeGet(ast->value); + storage_settings->index_granularity = ast->value.safeGet(); else throw Exception( "Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def), diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 76833b3e19a..7e0194ffd30 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1127,7 +1127,7 @@ Pipe StorageWindowView::watch( if (query.limit_length) { has_limit = true; - limit = safeGet(typeid_cast(*query.limit_length).value); + limit = typeid_cast(*query.limit_length).value.safeGet(); } auto reader = std::make_shared( diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 8acd7434d51..3d104ada0b6 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -196,8 +196,8 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( size_t size = name_col.size(); for (size_t i = 0; i < size; ++i) { - auto name = get(name_col[i]); - auto type_name = get(type_col[i]); + auto name = name_col[i].get(); + auto type_name = type_col[i].get(); auto storage_column = storage_columns.tryGetPhysical(name); if (storage_column && isObject(storage_column->type)) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 098756bcd7c..097a239ccae 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -107,7 +107,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (lit->value.getType() != Field::Types::String) return false; - res = safeGet(lit->value); + res = lit->value.safeGet(); return true; }; From f2227e125ff69b7f70b052cecaf61642b96e5a66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 05:11:09 +0200 Subject: [PATCH 521/582] Fix test --- tests/integration/test_settings_profile/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index a90273e7a7a..49ed0bd1aca 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -374,17 +374,17 @@ def test_show_profiles(): assert instance.query("SHOW CREATE PROFILE xyz") == "CREATE SETTINGS PROFILE xyz\n" assert ( instance.query("SHOW CREATE SETTINGS PROFILE default") - == "CREATE SETTINGS PROFILE default SETTINGS load_balancing = \\'random\\'\n" + == "CREATE SETTINGS PROFILE default\n" ) assert ( instance.query("SHOW CREATE PROFILES") - == "CREATE SETTINGS PROFILE default SETTINGS load_balancing = \\'random\\'\n" + == "CREATE SETTINGS PROFILE default\n" "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" "CREATE SETTINGS PROFILE xyz\n" ) expected_access = ( - "CREATE SETTINGS PROFILE default SETTINGS load_balancing = \\'random\\'\n" + "CREATE SETTINGS PROFILE default\n" "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" "CREATE SETTINGS PROFILE xyz\n" ) From 5bc2845e4997ef6f625d567d5590524e1fffe6c2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 10 Sep 2022 03:17:34 +0000 Subject: [PATCH 522/582] Automatic style fix --- tests/integration/test_settings_profile/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 49ed0bd1aca..13af1cf650d 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -377,8 +377,7 @@ def test_show_profiles(): == "CREATE SETTINGS PROFILE default\n" ) assert ( - instance.query("SHOW CREATE PROFILES") - == "CREATE SETTINGS PROFILE default\n" + instance.query("SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default\n" "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" "CREATE SETTINGS PROFILE xyz\n" ) From f3a16ec08fc9c358a9c46450bf0f76056bb90f35 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 05:17:55 +0200 Subject: [PATCH 523/582] Fix Chinese changelog --- docs/zh/whats-new/changelog/2017.md | 264 -- docs/zh/whats-new/changelog/2018.md | 1059 -------- docs/zh/whats-new/changelog/2019.md | 1855 -------------- docs/zh/whats-new/changelog/2020.md | 3529 --------------------------- 4 files changed, 6707 deletions(-) delete mode 100644 docs/zh/whats-new/changelog/2017.md delete mode 100644 docs/zh/whats-new/changelog/2018.md delete mode 100644 docs/zh/whats-new/changelog/2019.md delete mode 100644 docs/zh/whats-new/changelog/2020.md diff --git a/docs/zh/whats-new/changelog/2017.md b/docs/zh/whats-new/changelog/2017.md deleted file mode 100644 index 762a7f107e3..00000000000 --- a/docs/zh/whats-new/changelog/2017.md +++ /dev/null @@ -1,264 +0,0 @@ ---- -slug: /zh/whats-new/changelog/2017 ---- -### ClickHouse 版本 1.1.54327, 2017-12-21 {#clickhouse-release-1-1-54327-2017-12-21} - -此版本包含先前版本 1.1.54318 的错误修复: - -- 修复了可能导致数据丢失的复制中可能出现的竞争条件的错误. 此问题影响1.1.54310和1.1.54318版本. 如果将这些版本的任意一个与 Replicated 表一起使用,则强烈建议进行更新. 此问题显示在日志中的警告消息中,例如 `Part ... from own log does not exist.` 即使您没有在日志中看到这些消息,该问题也是相关的. - -### ClickHouse 版本 1.1.54318, 2017-11-30 {#clickhouse-release-1-1-54318-2017-11-30} - -此版本包含先前版本 1.1.54310 的错误修复: - -- 修复了在 SummingMergeTree 引擎中合并期间不正确的行删除. -- 修复了未复制的 MergeTree 引擎中的内存泄漏. -- 修复了在 MergeTree 引擎中频繁插入导致性能下降的问题. -- 修复了导致复制队列停止运行的问题. -- 修复了服务器日志的轮换和归档. - -### ClickHouse 版本 1.1.54310, 2017-11-01 {#clickhouse-release-1-1-54310-2017-11-01} - -#### 新特征: {#new-features} - -- MergeTree 系列表引擎的自定义分区键. -- [Kafka](https://clickhouse.com/docs/en/operations/table_engines/kafka/) 表引擎. -- 增加了对加载[CatBoost](https://catboost.yandex/)模型的支持, 并将它们应用到存储在ClickHouse中的数据. -- 添加了对 UTC 非整数偏移时区的支持 -- 添加了对具有时间间隔的算术运算的支持. -- Date 和 DateTime 类型的值范围扩展到 2105 年. -- 添加了 `CREATE MATERIALIZED VIEW x TO y` 查询(指定用于存储物化视图数据的现有表). -- 添加了不带参数的`ATTACH TABLE` 查询. -- SummingMergeTree 表中名称以 -Map 结尾的嵌套列的处理逻辑被提取到 sumMap 聚合函数中. 您现在可以明确指定此类列. -- IP 树字典的最大大小增加到 128M 条目. -- 添加了 getSizeOfEnumType 函数. -- 添加了 sumWithOverflow 聚合函数. -- 添加了对 Cap'n Proto 输入格式的支持. -- 您现在可以在使用 zstd 算法时自定义压缩级别. - -#### 向后不兼容的变化: {#backward-incompatible-changes} - -- 不允许使用内存以外的引擎创建临时表. -- 不允许使用 View 或 MaterializedView 引擎显式创建表. -- 在表创建期间,新的检查验证采样键表达式是否包含在主键中. - -#### Bug 修复: {#bug-fixes} - -- 修复了同步插入分布式表时的挂断问题. -- 修复了复制表中部件的非原子添加和删除. -- 插入物化视图的数据不会进行不必要的重复数据删除. -- 对本地副本滞后且远程副本不可用的分布式表执行查询不再导致错误. -- 用户不再需要访问 `default` 数据库的权限来创建临时表. -- 修复了指定不带参数的 Array 类型时崩溃的问题. -- 修复了包含服务器日志的磁盘卷已满时的挂断问题. -- 修复了 Unix 纪元第一周 toRelativeWeekNum 函数中的溢出问题. - -#### Build 改进: {#build-improvements} - -- 更新了多个第三方库(尤其是 Poco)并转换为 git 子模块. - -### ClickHouse 版本 1.1.54304, 2017-10-19 {#clickhouse-release-1-1-54304-2017-10-19} - -#### 新特征: {#new-features-1} - -- 本机协议中的 TLS 支持(要启用,请在 `config.xml` 中设置 `tcp_ssl_port`). - -#### Bug 修复: {#bug-fixes-1} - -- 复制表的`ALTER` 现在尝试尽快开始运行. -- 修复了使用设置 `preferred_block_size_bytes=0.` 读取数据时崩溃的问题. -- 修复了按下 `Page Down` 时 `clickhouse-client` 崩溃的问题. -- 使用 `GLOBAL IN` 和 `UNION ALL` 正确解释某些复杂的查询. -- `FREEZE PARTITION` 现在总是以原子方式工作. -- 空 POST 请求现在返回代码为 411 的响应. -- 修正了像 `CAST(1 AS Nullable(UInt8)).` 这样的表达式的解释错误. -- 修正了从 `MergeTree` 表中读取 `Array(Nullable(String))` 列时的错误. -- 修复了在解析诸如 `SELECT dummy AS dummy, dummy AS b` 之类的查询时崩溃的问题. -- 用户使用无效的 `users.xml` 正确更新. -- 可执行字典返回非零响应代码时的正确处理. - -### ClickHouse 版本 1.1.54292, 2017-09-20 {#clickhouse-release-1-1-54292-2017-09-20} - -#### 新特征: {#new-features-2} - -- 添加了用于处理坐标平面上的坐标的 `pointInPolygon` 函数. -- 添加了用于计算数组总和的 `sumMap` 聚合函数, 类似于 `SummingMergeTree` . -- 添加了 `trunc` 功能. 改进了舍入函数(`round`、`floor`、`ceil`、`roundToExp2`)的性能并更正了它们工作方式的逻辑. 更改了分数和负数的 `roundToExp2` 函数的逻辑. -- ClickHouse 可执行文件现在较少依赖于 libc 版本. 同一个 ClickHouse 可执行文件可以在各种 Linux 系统上运行. 使用编译查询时仍然存在依赖性(使用设置 `compile = 1` , 默认情况下不使用). -- 减少动态编译查询所需的时间. - -#### Bug 修复: {#bug-fixes-2} - -- 修复了有时会产生 `part ... intersects previous part` 消息和削弱副本一致性的错误. -- 修复了关闭期间 ZooKeeper 不可用导致服务器锁定的错误. -- 恢复副本时删除了过多的日志记录. -- 修复了 UNION ALL 实现中的错误. -- 修复了如果块中的第一列具有 Array 类型时在 concat 函数中发生的错误. -- 进度现在在 system.merges 表中可以正确显示. - -### ClickHouse 版本 1.1.54289, 2017-09-13 {#clickhouse-release-1-1-54289-2017-09-13} - -#### 新特征: {#new-features-3} - -- 用于服务器管理的 `SYSTEM` 查询: `SYSTEM RELOAD DICTIONARY`, `SYSTEM RELOAD DICTIONARIES`, `SYSTEM DROP DNS CACHE`, `SYSTEM SHUTDOWN`, `SYSTEM KILL`. -- 添加了用于处理数组的函数: `concat`, `arraySlice`, `arrayPushBack`, `arrayPushFront`, `arrayPopBack`, `arrayPopFront`. -- 为 ZooKeeper 配置添加了 `root` 和 `identity` 参数. 这将允许您隔离同一 ZooKeeper 集群上的各个用户. -- 添加了聚合函数 `groupBitAnd` 、 `groupBitOr` 和 `groupBitXor` (为了兼容性,它们也可以在名称 `BIT_AND` 、`BIT_OR`和`BIT_XOR` 下使用). -- 可以通过在文件系统中指定套接字来从 MySQL 加载外部字典. -- 可以通过 SSL 从 MySQL 加载外部字典 (`ssl_cert`, `ssl_key`, `ssl_ca` 参数). -- 添加了 `max_network_bandwidth_for_user` 设置以限制每个用户查询的总体带宽使用. -- 支持临时表的 `DROP TABLE`. -- 支持从 `CSV` 和 `JSONEachRow` 格式读取 Unix 时间戳格式的 `DateTime` 值. -- 现在默认排除分布式查询中的滞后副本(默认阈值为 5 分钟). -- 在 ALTER 期间使用 FIFO 锁定:对于连续运行的查询,ALTER 查询不会无限期阻塞. -- 在配置文件中设置 `umask` 的选项. -- 使用 `DISTINCT` 提高查询的性能. - -#### Bug 修复: {#bug-fixes-3} - -- 改进了在 ZooKeeper 中删除旧节点的过程. 以前, 如果插入非常频繁, 旧节点有时不会被删除, 从而导致服务器关闭缓慢等. -- 修复了为 ZooKeeper 连接选择主机时的随机化问题. -- 如果副本是本地主机, 则修复了在分布式查询中排除滞后副本的问题. -- 修复了在 `嵌套` 结构中的元素上运行 `ALTER MODIFY` 后, `ReplicatedMergeTree` 表中的数据部分可能被破坏的错误. -- 修复了可能导致 SELECT 查询 `hang` 的错误. -- 分布式 DDL 查询的改进. -- 修复了查询 `CREATE TABLE ... AS `. -- 解决了对 Buffer 表的 `ALTER ... CLEAR COLUMN IN PARTITION` 查询中的死锁. -- 修复了使用 `JSONEachRow` 和 `TSKV` 格式时 `Enum` 的无效默认值 (0 而不是最小值). -- 解决了使用带有 `可执行` 源的字典时出现僵尸进程的问题. -- 修复了 HEAD 查询的段错误. - -#### 改进了开发和组装ClickHouse的工作流: {#improved-workflow-for-developing-and-assembling-clickhouse} - -- 您可以使用 `pbuilder` 来构建 ClickHouse. -- 你可以使用 `libc++` 代替 `libstdc++` 在 Linux 上构建. -- 添加了使用静态代码分析工具的说明: `Coverage`, `clang-tidy`, `cppcheck`. - -#### 升级时请注意: {#please-note-when-upgrading} - -- 现在有更高的 MergeTree 设置默认值 `max_bytes_to_merge_at_max_space_in_pool` (要合并的数据部分的最大总大小, 以字节为单位): 它已从 100 GiB 增加到 150 GiB. 这可能会导致在服务器升级后运行大型合并, 从而导致磁盘子系统负载增加. 如果服务器上的可用空间小于正在运行的合并总量的两倍, 这将导致所有其他合并停止运行, 包括小数据部分的合并. 因此, INSERT 查询将失败并显示消息"合并的处理速度明显慢于插入." , 使用 `SELECT * FROM system.merges` 查询来监控情况. 您还可以在 `system.metrics` 表或 Graphite 中检查 `DiskSpaceReservedForMerge` 指标. 您不需要做任何事情来解决这个问题, 因为一旦大型合并完成, 问题就会自行解决. 如果您发现这不可接受, 您可以恢复 `max_bytes_to_merge_at_max_space_in_pool` 设置的先前值. 为此, 请转到 config.xml 中的 `` 部分, 设置 ``` ``107374182400 ``` 并重新启动服务器. - -### ClickHouse 版本 1.1.54284, 2017-08-29 {#clickhouse-release-1-1-54284-2017-08-29} - -- 这是先前 1.1.54282 版本的错误修复版本. 它修复了 ZooKeeper 中部分目录中的泄漏. - -### ClickHouse 版本 1.1.54282, 2017-08-23 {#clickhouse-release-1-1-54282-2017-08-23} - -此版本包含先前版本 1.1.54276 的错误修复: - -- 修复了插入分布式表时的 `DB::Exception: Assertion violation: !_path.empty()`. -- 如果输入数据以 ';' 开头, 则在以 RowBinary 格式插入时固定解析. -- 某些聚合函数 (例如 `groupArray()` ) 的运行时编译期间的错误. - -### ClickHouse 版本 1.1.54276, 2017-08-16 {#clickhouse-release-1-1-54276-2017-08-16} - -#### 新特征: {#new-features-4} - -- SELECT 查询添加了一个可选的 WITH 部分. 示例查询:`WITH 1+1 AS a SELECT a, a*a` . -- NSERT 可以在分布式表中同步执行:只有在所有数据都保存在所有分片上后才返回 OK. 这是通过设置 `insert_distributed_sync=1` 激活的. -- 添加了用于处理 16 字节标识符的 UUID 数据类型. -- 添加了 CHAR、FLOAT 和其他类型的别名以与 Tableau 兼容. -- 新增 toYYYYMM, toYYYYMMDD, toYYYYMMDDhhmmss 时间转数字功能. -- 您可以使用 IP 地址 (与主机名一起) 来识别集群 DDL 查询的服务器. -- 在函数 `substring(str, pos, len)` 中添加了对非常量参数和负偏移量的支持. -- 为 `groupArray(max_size)(column)` 聚合函数增加了max_size参数, 并优化了其性能. - -#### Main Changes: {#main-changes} - -- 安全改进:所有服务器文件都使用 0640 权限创建(可以通过 `` 配置参数更改). -- 改进了语法无效查询的错误消息. -- 合并大段 MergeTree 数据时显着减少内存消耗并提高性能. -- 显着提高了 ReplacingMergeTree 引擎的数据合并性能. -- 通过组合多个源插入提高了从分布式表进行异步插入的性能. 要启用此功能, 请使用设置 `distributed_directory_monitor_batch_inserts=1` . - -#### Backward Incompatible Changes: {#backward-incompatible-changes-1} - -- 更改了数组 `groupArray(array_column)` 函数聚合状态的二进制格式. - -#### Complete List of Changes: {#complete-list-of-changes} - -- 添加了 `output_format_json_quote_denormals` 设置, 可以以 JSON 格式输出 nan 和 inf 值. -- 从分布式表读取时优化流分配. -- 如果值不变, 可以在只读模式下配置设置. -- 加了检索 MergeTree 引擎的非整数粒度的功能, 以满足对 `preferred_block_size_bytes` 设置中指定的块大小的限制. 目的是在处理来自大列的表的查询时减少RAM的消耗并增加缓存局部性. -- 有效地使用包含像 `toStartOfHour(x)` 这样的表达式的索引来处理像 `toStartOfHour(x) op сonstexpr` 这样的条件. -- 添加了 MergeTree 引擎的新设置(config.xml 中的 merge_tree 部分): - - `replicad_deduplication_window_seconds` 设置允许在复制表中删除重复插入的秒数. - - `cleanup_delay_period` 设置启动清理以删除过时数据的频率. - - `Replicationd_can_become_leader` 可以防止副本成为领导者(并分配合并). -- 加速清理以从 ZooKeeper 中删除过时的数据. -- 集群 DDL 查询的多项改进和修复. 特别有趣的是新设置 `distributed_ddl_task_timeout`, 它限制了等待集群中服务器响应的时间. 如果 ddl 请求没有在所有主机上执行,响应将包含超时错误并且请求将以异步模式执行. -- 改进了服务器日志中堆栈跟踪的显示. -- 为压缩方法添加了 "none" 值. -- 您可以在 config.xml 中使用多个dictionaries_config 部分. -- 可以通过文件系统中的套接字连接到 MySQL. -- `system.parts` 表有一个新列, 其中包含有关标记大小的信息(以字节为单位). - -#### Bug 修复: {#bug-fixes-4} - -- 使用 Merge 表的分布式表现在可以正确用于带有 `_table` 字段条件的 SELECT 查询. -- 修复了检查数据部分时 ReplicatedMergeTree 中罕见的竞争条件. -- 修复了启动服务器时 `leader election` 可能会冻结的问题. -- 使用数据源的本地副本时,将忽略 `max_replica_delay_for_distributed_queries` 设置. 这已被修复. -- 修复了尝试清理不存在的列时 `ALTER TABLE CLEAR COLUMN IN PARTITION` 的错误行为. -- 修复了 multiIf 函数中使用空数组或字符串时的异常. -- 修复了反序列化本机格式时过多的内存分配. -- 修复了 Trie 词典的错误自动更新. -- 修复了在使用 SAMPLE 时从合并表中使用 GROUP BY 子句运行查询时的异常. -- 修复了 `distributed_aggregation_memory_efficient=1` 时 GROUP BY 的崩溃. -- 现在可以在 IN 和 JOIN 右侧指定 `database.table`. -- 太多线程用于并行聚合. 这已被修复. -- 修复了 `if` 函数如何与 FixedString 参数一起工作. -- 对于权重为 0 的分片, SELECT 在分布式表中工作不正确. 这已得到修复. -- 运行 `CREATE VIEW IF EXISTS 不再导致崩溃` . -- 修复了设置 `input_format_skip_unknown_fields=1` 且存在负数时的错误行为. -- 修复了如果字典中有一些无效数据, `dictGetHierarchy()` 函数中的无限循环. -- 修复了使用 IN 或 JOIN 子句和合并表中的子查询运行分布式查询时的 `Syntax error: unexpected (...)` 错误. -- 修复了对字典表中 SELECT 查询的错误解释. -- 修复了在超过 20 亿元素的 IN 和 JOIN 子句中使用数组时的 "Cannot mremap" 错误. -- 修复了以 MySQL 为源的字典的故障转移. - -#### 改进了开发和组装ClickHouse的工作流: {#improved-workflow-for-developing-and-assembling-clickhouse-1} - -- 可以在 Arcadia 中组装 Builds. -- 可以使用 gcc 7 编译 ClickHouse. -- 使用 ccache+distcc 的并行构建现在更快了. - -### ClickHouse 版本 1.1.54245, 2017-07-04 {#clickhouse-release-1-1-54245-2017-07-04} - -#### 新特征: {#new-features-5} - -- 分布式 DDL(例如, `REATE TABLE ON CLUSTER`). -- 复制查询 `ALTER TABLE CLEAR COLUMN IN PARTITION.` . -- 字典表引擎 (以表的形式访问字典数据). -- 字典数据库引擎 (这种类型的数据库自动为所有连接的外部字典提供字典表). -- 您可以通过向源发送请求来检查字典的更新. -- 合格的列名. -- 使用双引号引用标识符. -- HTTP 接口中的会话. -- 复制表的 OPTIMIZE 查询不仅可以在领导者上运行. - -#### 向后不兼容的变化: {#backward-incompatible-changes-2} - -- 删除了 SET GLOBAL. - -#### 次要更改: {#minor-changes} - -- 目前在触发警报后,日志会打印完整的堆栈跟踪. -- 放宽了启动时损坏/额外数据部分数量的验证(误报太多). - -#### Bug 修复: {#bug-fixes-5} - -- 修复了插入分布式表时的错误连接"卡住"问题. -- GLOBAL IN 现在适用于来自查看分布式表的合并表的查询. -- 在 Google Compute Engine 虚拟机上检测到的内核数不正确. 这已被修复. -- 缓存外部字典的可执行源的工作方式发生了变化. -- 修正了包含空字符的字符串的比较. -- 修复了 Float32 主键字段与常量的比较. -- 以前,对字段大小的错误估计可能会导致分配过大. -- 修复了使用 ALTER 查询添加到表中的 Nullable 列时发生的崩溃. -- 修复了当行数小于 LIMIT 时按 Nullable 列排序时崩溃的问题. -- 修复了仅由常量值组成的 ORDER BY 子查询. -- 以前,复制表在 DROP TABLE 失败后可能保持无效状态. -- 结果为空的标量子查询的别名不再丢失. -- 现在,如果 .so 文件损坏,使用编译的查询不会因错误而失败. diff --git a/docs/zh/whats-new/changelog/2018.md b/docs/zh/whats-new/changelog/2018.md deleted file mode 100644 index 8b0c305af3d..00000000000 --- a/docs/zh/whats-new/changelog/2018.md +++ /dev/null @@ -1,1059 +0,0 @@ ---- -slug: /zh/whats-new/changelog/2018 ---- -## ClickHouse 版本 18.16 {#clickhouse-release-18-16} - -### ClickHouse 版本 18.16.1, 2018-12-21 {#clickhouse-release-18-16-1-2018-12-21} - -#### Bug 修复: {#bug-fixes} - -- 修复了导致使用 ODBC 源更新字典出现问题的错误. [#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) -- 聚合函数的 JIT 编译现在适用于 LowCardinality 列. [#3838](https://github.com/ClickHouse/ClickHouse/issues/3838) - -#### 改进: {#improvements} - -- 添加了 `low_cardinality_allow_in_native_format` 设置 (默认启用). 禁用时, LowCardinality 列将转换为用于 SELECT 查询的普通列, 而用于 INSERT 查询的普通列将被转换. [#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) - -#### Build 改进: {#build-improvements} - -- 修复了在 macOS 和 ARM 上的构建. - -### ClickHouse 版本 18.16.0, 2018-12-14 {#clickhouse-release-18-16-0-2018-12-14} - -#### 新特征: {#new-features} - -- 以半结构化输入格式 (`JSONEachRow` 、 `TSKV` ) 加载数据时, 会评估 `DEFAULT` 表达式的缺失字段. 该功能通过 `insert_sample_with_metadata` 设置启用. [#3555](https://github.com/ClickHouse/ClickHouse/pull/3555) -- `ALTER TABLE` 查询现在具有 `MODIFY ORDER BY` 操作, 用于在添加或删除表列时更改排序键. 这对于 `MergeTree` 系列中的表很有用, 这些表在基于此排序键合并时执行附加任务, 例如 `SummingMergeTree`、`AggregatingMergeTree` 等. [#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) [#3755](https://github.com/ClickHouse/ClickHouse/pull/3755) -- 对于`MergeTree` 系列中的表, 现在您可以指定不同的排序键 (`ORDER BY`) 和索引 (`PRIMARY KEY`) . 排序键可以比索引长. [#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) -- 添加了 `hdfs` 表功能和 `HDFS` 表引擎, 用于导入和导出数据到HDFS. [chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/3617) -- 添加了使用 base64 的函数: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alexander Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3350) -- 现在你可以使用一个参数来配置`uniqCombined`聚合函数的精度(选择HyperLogLog单元格的数量). [#3406](https://github.com/ClickHouse/ClickHouse/pull/3406) -- 添加了 `system.contributors` 表, 其中包含在 ClickHouse 中提交的每个人的姓名. [#3452](https://github.com/ClickHouse/ClickHouse/pull/3452) -- 添加了省略 `ALTER TABLE ... FREEZE` 查询的分区的功能,以便一次备份所有分区. [#3514](https://github.com/ClickHouse/ClickHouse/pull/3514) -- 添加了不需要指定返回值类型的 `dictGet` 和 `dictGetOrDefault` 函数. 类型是根据字典描述自动确定的. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3564) -- 现在您可以为表描述中的列指定注释并使用 `ALTER` 更改它. [#3377](https://github.com/ClickHouse/ClickHouse/pull/3377) -- 支持使用简单键读取 `Join` 类型的表. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3728) -- 现在您可以在创建 `Join` 类型表时指定选项 `join_use_nulls`、`max_rows_in_join`、`max_bytes_in_join` 和 `join_overflow_mode`. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3728) -- 加了 `joinGet` 函数, 允许您像使用字典一样使用 `Join` 类型的表. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3728) -- 将 `partition_key`、`sorting_key`、`primary_key` 和 `sampling_key` 列添加到 `system.tables` 表中, 以提供有关表键的信息. [#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) -- 向 `system.columns` 表添加了 `is_in_partition_key`、`is_in_sorting_key` 、`is_in_primary_key` 和 `is_in_sampling_key` 列. [#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) -- 将 `min_time` 和 `max_time` 列添加到 `system.parts` 表中. 当分区键是由 `DateTime`列组成的表达式时,将填充这些列. [Emmanuel Donin de Rosière](https://github.com/ClickHouse/ClickHouse/pull/3800) - -#### Bug 修复: {#bug-fixes-1} - -- `LowCardinality` 数据类型的修复和性能改进. `GROUP BY` 使用 `LowCardinality(Nullable(...))`. 获取 `extremes` 的值. 处理高阶函数. `LEFT ARRAY JOIN`. 分布式 `GROUP BY`. 返回 `Array` 的函数. 执行 `ORDER BY`. 写入 `Distributed` 表(nicelulu). 向后兼容来自实现 `Native` 协议的旧客户端的 `INSERT` 查询. 支持 `JOIN` 的 `LowCardinality`. 在单个流中工作时提高了性能. [#3823](https://github.com/ClickHouse/ClickHouse/pull/3823) [#3803](https://github.com/ClickHouse/ClickHouse/pull/3803) [#3799](https://github.com/ClickHouse/ClickHouse/pull/3799) [#3769](https://github.com/ClickHouse/ClickHouse/pull/3769) [#3744](https://github.com/ClickHouse/ClickHouse/pull/3744) [#3681](https://github.com/ClickHouse/ClickHouse/pull/3681) [#3651](https://github.com/ClickHouse/ClickHouse/pull/3651) [#3649](https://github.com/ClickHouse/ClickHouse/pull/3649) [#3641](https://github.com/ClickHouse/ClickHouse/pull/3641) [#3632](https://github.com/ClickHouse/ClickHouse/pull/3632) [#3568](https://github.com/ClickHouse/ClickHouse/pull/3568) [#3523](https://github.com/ClickHouse/ClickHouse/pull/3523) [#3518](https://github.com/ClickHouse/ClickHouse/pull/3518) -- 修复了 `select_sequential_consistency` 选项的工作方式. 以前, 启用此设置后, 有时会在开始写入新分区后返回不完整的结果. [#2863](https://github.com/ClickHouse/ClickHouse/pull/2863) -- 执行 DDL `ON CLUSTER` 查询和 `ALTER UPDATE/DELETE` 时正确指定数据库. [#3772](https://github.com/ClickHouse/ClickHouse/pull/3772) [#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) -- 为 VIEW 内的子查询正确指定了数据库. [#3521](https://github.com/ClickHouse/ClickHouse/pull/3521) -- 修复了 `PREWHERE` 中的 `FINAL` 错误, 用于 `VersionedCollapsingMergeTree` . [7167bfd7](https://github.com/ClickHouse/ClickHouse/commit/7167bfd7b365538f7a91c4307ad77e552ab4e8c1) -- 现在您可以使用 `KILL QUERY` 取消尚未启动的查询, 因为它们正在等待表被锁定. [#3517](https://github.com/ClickHouse/ClickHouse/pull/3517) -- 如果时钟在午夜后移, 则更正了日期和时间计算(这发生在伊朗, 1981 年至 1983 年发生在莫斯科). 以前, 这导致时间比必要的提前一天重置, 并且还导致文本格式的日期和时间格式不正确. [#3819](https://github.com/ClickHouse/ClickHouse/pull/3819) -- 修复了某些情况下 `VIEW` 和省略数据库的子查询的错误. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3521) -- 修复了由于未锁定内部 `MATERIALIZED VIEW` 而同时从 `MATERIALIZED VIEW` 中读取并删除 `MATERIALIZED VIEW` 时的竞争条件. [#3404](https://github.com/ClickHouse/ClickHouse/pull/3404) [#3694](https://github.com/ClickHouse/ClickHouse/pull/3694) -- 修复了错误`Lock handler cannot be nullptr`. [#3689](https://github.com/ClickHouse/ClickHouse/pull/3689) -- 修复了启用 `compile_expressions` 选项时的查询处理(默认情况下启用). 不再展开像 `now` 函数这样的非确定性常量表达式. [#3457](https://github.com/ClickHouse/ClickHouse/pull/3457) -- 修复了在 `toDecimal32/64/128` 函数中指定非常量比例参数时崩溃的问题s. -- 修复了尝试将包含 `Values` 格式的 `NULL` 元素的数组插入没有 `Nullable` 的 `Array` 类型的列时的错误(如果 `input_format_values_interpret_expressions` = 1). [#3487](https://github.com/ClickHouse/ClickHouse/pull/3487) [#3503](https://github.com/ClickHouse/ClickHouse/pull/3503) -- 如果 ZooKeeper 不可用, 则修复了在 `DDLWorker` 中连续错误记录的问题. [8f50c620](https://github.com/ClickHouse/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2) -- 修复了来自 `Date` 和 `DateTime` 类型参数的 `quantile*` 函数的返回类型. [#3580](https://github.com/ClickHouse/ClickHouse/pull/3580) -- 修复了 `WITH` 子句 如果它指定一个没有表达式的简单别名. [#3570](https://github.com/ClickHouse/ClickHouse/pull/3570) -- 修复了启用 `enable_optimize_predicate_expression` 时使用命名子查询和限定列名的查询的处理. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3588) -- 修复了使用物化视图时 `尝试附加到 nullptr 线程组` 的错误. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3623) -- 修复了将某些不正确的参数传递给 `arrayReverse` 函数时发生的崩溃. [73e3a7b6](https://github.com/ClickHouse/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) -- 修复了 `extractURLParameter` 函数中的缓冲区溢出问题. 改进性能. 添加了对包含零字节的字符串的正确处理. [141e9799](https://github.com/ClickHouse/ClickHouse/commit/141e9799e49201d84ea8e951d1bed4fb6d3dacb5) -- 修复了 `lowerUTF8` 和 `upperUTF8` 函数中的缓冲区溢出. 删除了通过 `FixedString` 类型参数执行这些函数的能力. [#3662](https://github.com/ClickHouse/ClickHouse/pull/3662) -- 修复了删除 `MergeTree` 表时罕见的竞争条件. [#3680](https://github.com/ClickHouse/ClickHouse/pull/3680) -- 修复了从 `Buffer` 表读取并同时在目标表上执行 `ALTER` 或 `DROP` 时的竞争条件. [#3719](https://github.com/ClickHouse/ClickHouse/pull/3719) -- 修复了超出 `max_temporary_non_const_columns` 限制时的段错误. [#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) - -#### 改进: {#improvements-1} - -- 服务器不会将处理后的配置文件写入`/etc/clickhouse-server/`目录. 相反, 它将它们保存在 `path` 内的 `preprocessed_configs` 目录中. 这意味着 `/etc/clickhouse-server/` 目录对 `clickhouse` 用户没有写权限, 提高了安全性. [#2443](https://github.com/ClickHouse/ClickHouse/pull/2443) -- `min_merge_bytes_to_use_direct_io` 选项默认设置为 10 GiB. 形成 MergeTree 系列中大部分表的合并将在 `O_DIRECT` 模式下执行,以防止过多的页面缓存逐出. [#3504](https://github.com/ClickHouse/ClickHouse/pull/3504) -- 当表数量非常多时,加速服务器启动. [#3398](https://github.com/ClickHouse/ClickHouse/pull/3398) -- 为副本之间的连接添加了连接池和 HTTP `Keep-Alive`. [#3594](https://github.com/ClickHouse/ClickHouse/pull/3594) -- 如果查询语法无效, `HTTP` 接口返回 `400 Bad Request` 代码(之前返回500). [31bc680a](https://github.com/ClickHouse/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) -- `join_default_strictness` 选项默认设置为 `ALL` 以实现兼容性. [120e2cbe](https://github.com/ClickHouse/ClickHouse/commit/120e2cbe2ff4fbad626c28042d9b28781c805afe) -- 对于无效或复杂的正则表达式,从 `re2` 库中删除了对 `stderr` 的日志记录. [#3723](https://github.com/ClickHouse/ClickHouse/pull/3723) -- 为 `Kafka` 表引擎添加: 在开始从 Kafka 读取之前检查订阅; 表的 kafka_max_block_size 设置. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3396) -- `cityHash64`、`farmHash64`、`metroHash64`、`sipHash64`、`halfMD5`、`murmurHash2_32`、`murmurHash2_64`、`murmurHash3_32` 和`murmurHash3_64` 函数现在可用于任意数量的参数和参数元组的形式. [#3451](https://github.com/ClickHouse/ClickHouse/pull/3451) [#3519](https://github.com/ClickHouse/ClickHouse/pull/3519) -- `arrayReverse` 函数现在适用于任何类型的数组. [73e3a7b6](https://github.com/ClickHouse/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) -- 添加了一个可选参数:`timeSlots` 函数的插槽大小. [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3724) -- 对于 `FULL` 和 `RIGHT JOIN`,`max_block_size` 设置用于来自右表的非连接数据流. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3699) -- 在 `clickhouse-benchmark` 和 `clickhouse-performance-test` 中添加了 `--secure` 命令行参数以启用 TLS. [#3688](https://github.com/ClickHouse/ClickHouse/pull/3688) [#3690](https://github.com/ClickHouse/ClickHouse/pull/3690) -- `Buffer` 类型表的结构与目标表的结构不匹配时的类型转换. [Vitaly Baranov](https://github.com/ClickHouse/ClickHouse/pull/3603) -- 添加了 `tcp_keep_alive_timeout` 选项以在指定时间间隔内不活动后启用保持活动数据包. [#3441](https://github.com/ClickHouse/ClickHouse/pull/3441) -- 如果`system.parts` 表包含单列, 则删除了不必要的对分区键值的引用. [#3652](https://github.com/ClickHouse/ClickHouse/pull/3652) -- 模函数适用于 `Date` 和 `DateTime` 数据类型. [#3385](https://github.com/ClickHouse/ClickHouse/pull/3385) -- 添加了`POWER`、`LN`、`LCASE`、`UCASE`、`REPLACE`、`LOCATE`、`SUBSTR` 和`MID` 函数的同义词. [#3774](https://github.com/ClickHouse/ClickHouse/pull/3774) [#3763](https://github.com/ClickHouse/ClickHouse/pull/3763) Some function names are case-insensitive for compatibility with the SQL standard. Added syntactic sugar `SUBSTRING(expr FROM start FOR length)` for compatibility with SQL. [#3804](https://github.com/ClickHouse/ClickHouse/pull/3804) -- 添加了对与 `clickhouse-server` 可执行代码对应的 `mlock` 内存页面的功能,以防止其被强制内存不足. 默认情况下禁用此功能. [#3553](https://github.com/ClickHouse/ClickHouse/pull/3553) -- 从`O_DIRECT` 读取时提高了性能(启用了`min_bytes_to_use_direct_io` 选项). [#3405](https://github.com/ClickHouse/ClickHouse/pull/3405) -- 改进了用于常量键参数和非常量默认参数的 `dictGet...OrDefault` 函数的性能. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3563) -- `firstSignificantSubdomain` 函数现在可以处理域 `gov`、`mil` 和 `edu`. [Igor Hatarist](https://github.com/ClickHouse/ClickHouse/pull/3601) Improved performance. [#3628](https://github.com/ClickHouse/ClickHouse/pull/3628) -- 能够通过在 `/etc/default/clickhouse` 中定义 `CLICKHOUSE_PROGRAM_ENV` 使用 `SYS-V init.d` 脚本来指定用于启动 `clickhouse-server` 的自定义环境变量. - [Pavlo Bashynskyi](https://github.com/ClickHouse/ClickHouse/pull/3612) -- clickhouse-server 初始化脚本的正确返回码. [#3516](https://github.com/ClickHouse/ClickHouse/pull/3516) -- `system.metrics` 表现在有 `VersionInteger` 指标, `system.build_options` 添加了行 `VERSION_INTEGER`, 其中包含 ClickHouse 版本的数字形式, 例如 `18016000`. [#3644](https://github.com/ClickHouse/ClickHouse/pull/3644) -- 删除了将 `Date` 类型与数字进行比较的功能,以避免诸如 `date = 2018-12-17` 之类的潜在错误, 其中错误地省略了日期周围的引号. [#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) -- 修复了像 `rowNumberInAllBlocks` 这样的有状态函数的行为。 由于在查询分析期间启动,他们以前输出的结果大一个数字. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3729) -- 如果无法删除 `force_restore_data` 文件, 则会显示错误消息. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3794) - -#### Build 改进: {#build-improvements-1} - -- 更新了 `jemalloc` 库, 修复了潜在的内存泄漏. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3557) -- 默认情况下启用使用 `jemalloc` 进行分析以便调试构建. [2cc82f5c](https://github.com/ClickHouse/ClickHouse/commit/2cc82f5cbe266421cd4c1165286c2c47e5ffcb15) -- 添加了仅在系统上安装了 `Docker` 时运行集成测试的能力. [#3650](https://github.com/ClickHouse/ClickHouse/pull/3650) -- 在 SELECT 查询中添加了模糊表达式测试. [#3442](https://github.com/ClickHouse/ClickHouse/pull/3442) -- 添加了针对提交的压力测试, 它以并行和随机顺序执行功能测试以检测更多竞争条件. [#3438](https://github.com/ClickHouse/ClickHouse/pull/3438) -- 改进了在 Docker 镜像中启动 clickhouse-server 的方法. [Elghazal Ahmed](https://github.com/ClickHouse/ClickHouse/pull/3663) -- 对于 Docker 映像, 添加了对使用 `/docker-entrypoint-initdb.d` 目录中的文件初始化数据库的支持. [Konstantin Lebedev](https://github.com/ClickHouse/ClickHouse/pull/3695) -- 修复了 ARM 上的构建. [#3709](https://github.com/ClickHouse/ClickHouse/pull/3709) - -#### 向后不兼容的更新: {#backward-incompatible-changes} - -- 删除了将 `Date` 类型与数字进行比较的能力. 而不是 `toDate('2018-12-18') = 17883`, 你必须使用显式类型转换 `= toDate(17883)`. [#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) - -## ClickHouse 版本 18.14 {#clickhouse-release-18-14} - -### ClickHouse 版本 18.14.19, 2018-12-19 {#clickhouse-release-18-14-19-2018-12-19} - -#### Bug 修复: {#bug-fixes-2} - -- 修复了导致使用 ODBC 源更新字典出现问题的错误. [#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) -- 执行 DDL `ON CLUSTER` 查询时正确指定数据库. [#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) -- 修复了超出 `max_temporary_non_const_columns` 限制时的段错误. [#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) - -#### Build 改进: {#build-improvements-2} - -- 修复了 ARM 上的构建. - -### ClickHouse 版本 18.14.18, 2018-12-04 {#clickhouse-release-18-14-18-2018-12-04} - -#### Bug 修复: {#bug-fixes-3} - -- 修复了 `dictGet...` 函数对于 `range` 类型的字典的错误,如果其中一个参数是常量而另一个不是. [#3751](https://github.com/ClickHouse/ClickHouse/pull/3751) -- 修复了导致消息 `netlink: '...': attribute type 1 has an invalid length` 打印在 Linux 内核日志中的错误, 这种错误只发生在足够新的 Linux 内核版本上. [#3749](https://github.com/ClickHouse/ClickHouse/pull/3749) -- 修复了函数 `empty` 中 `FixedString` 类型参数的段错误. [Daniel, Dao Quang Minh](https://github.com/ClickHouse/ClickHouse/pull/3703) -- 修复了在使用大值 `max_query_size` 设置时过多的内存分配(一次性预分配了 `max_query_size` 字节的内存块). [#3720](https://github.com/ClickHouse/ClickHouse/pull/3720) - -#### Build 更新: {#build-changes} - -- 使用 OS 包中版本 7 的 LLVM/Clang 库修复构建 (这些库用于运行时查询编译). [#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) - -### ClickHouse 版本 18.14.17, 2018-11-30 {#clickhouse-release-18-14-17-2018-11-30} - -#### Bug 修复: {#bug-fixes-4} - -- 修复了 ODBC 桥进程未随主服务器进程终止的情况. [#3642](https://github.com/ClickHouse/ClickHouse/pull/3642) -- 修复了同步插入到`Distributed` 表中的问题, 其中的列列表与远程表的列列表不同. [#3673](https://github.com/ClickHouse/ClickHouse/pull/3673) -- 修复了在删除 MergeTree 表时可能导致崩溃的罕见竞争条件. [#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) -- 修复了查询线程创建失败并出现 `资源暂时不可用` 错误时的查询死锁. [#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) -- 修复了当使用 `CREATE AS table` 语法并且在 `AS table` 之前指定 `ENGINE` 子句时对 `ENGINE` 子句的解析 (错误导致忽略指定的引擎). [#3692](https://github.com/ClickHouse/ClickHouse/pull/3692) - -### ClickHouse 版本 18.14.15, 2018-11-21 {#clickhouse-release-18-14-15-2018-11-21} - -#### Bug 修复: {#bug-fixes-5} - -- 反序列化 `Array(String)` 类型的列时高估了内存块的大小, 导致 `Memory limit exceeded` 错误. 该问题出现在版本 18.12.13. [#3589](https://github.com/ClickHouse/ClickHouse/issues/3589) - -### ClickHouse 版本 18.14.14, 2018-11-20 {#clickhouse-release-18-14-14-2018-11-20} - -#### Bug 修复: {#bug-fixes-6} - -- 修复了集群配置为安全时的 `ON CLUSTER` 查询 (标志 `` ). [#3599](https://github.com/ClickHouse/ClickHouse/pull/3599) - -#### Build Changes: {#build-changes-1} - -- 修复的问题(来自系统的 llvm-7, macos). [#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) - -### ClickHouse 版本 18.14.13, 2018-11-08 {#clickhouse-release-18-14-13-2018-11-08} - -#### Bug 修复: {#bug-fixes-7} - -- 修复了 `MergingSorted 流中的块结构不匹配` 错误. [#3162](https://github.com/ClickHouse/ClickHouse/issues/3162) -- 修复了 `ON CLUSTER` 查询, 以防在集群配置中打开安全连接 (`` 标志). [#3465](https://github.com/ClickHouse/ClickHouse/pull/3465) -- 修复了使用 `SAMPLE`、`PREWHERE` 和 alias 列的查询中的错误. [#3543](https://github.com/ClickHouse/ClickHouse/pull/3543) -- 修复了启用 `min_bytes_to_use_direct_io` 设置时罕见的 `unknown compression method` 错误. [3544](https://github.com/ClickHouse/ClickHouse/pull/3544) - -#### 性能改进: {#performance-improvements} - -- 修复了在 AMD EPYC 处理器上执行时使用 UInt16 或 Date 类型列的 `GROUP BY` 查询的性能回归. [Igor Lapko](https://github.com/ClickHouse/ClickHouse/pull/3512) -- 修复了处理长字符串的查询的性能回归. [#3530](https://github.com/ClickHouse/ClickHouse/pull/3530) - -#### Build 改进: {#build-improvements-3} - -- 简化 Arcadia 构建的改进. [#3475](https://github.com/ClickHouse/ClickHouse/pull/3475), [#3535](https://github.com/ClickHouse/ClickHouse/pull/3535) - -### ClickHouse 版本 18.14.12, 2018-11-02 {#clickhouse-release-18-14-12-2018-11-02} - -#### Bug 修复: {#bug-fixes-8} - -- 修复了加入两个未命名子查询时崩溃的问题. [#3505](https://github.com/ClickHouse/ClickHouse/pull/3505) -- 修复了在查询外部数据库时生成不正确的查询 (带有空的 `WHERE` 子句). [hotid](https://github.com/ClickHouse/ClickHouse/pull/3477) -- 修复了在 ODBC 字典中使用错误超时值的问题. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3511) - -### ClickHouse 版本 18.14.11, 2018-10-29 {#clickhouse-release-18-14-11-2018-10-29} - -#### Bug 修复: {#bug-fixes-9} - -- 修复了错误 `Block structure mismatch in UNION stream: different number of columns`. [#2156](https://github.com/ClickHouse/ClickHouse/issues/2156) -- 修复了在包含嵌套结构内的数组的表中合并数据时出现的错误. [#3397](https://github.com/ClickHouse/ClickHouse/pull/3397) -- 修复了如果禁用 `merge_tree_uniform_read_distribution` 设置 (默认启用) 时查询结果不正确的问题. [#3429](https://github.com/ClickHouse/ClickHouse/pull/3429) -- 修复了以本机格式插入分布式表的错误. [#3411](https://github.com/ClickHouse/ClickHouse/issues/3411) - -### ClickHouse 版本 18.14.10, 2018-10-23 {#clickhouse-release-18-14-10-2018-10-23} - -- `compile_expressions` 设置 (表达式的 JIT 编译) 默认是禁用的. [#3410](https://github.com/ClickHouse/ClickHouse/pull/3410) -- `enable_optimize_predicate_expression` 设置默认是禁用的. - -### ClickHouse 版本 18.14.9, 2018-10-16 {#clickhouse-release-18-14-9-2018-10-16} - -#### 新特征: {#new-features-1} - -- `GROUP BY` 的 `WITH CUBE` 修饰符 (也可以使用替代语法 `GROUP BY CUBE(...)`). [#3172](https://github.com/ClickHouse/ClickHouse/pull/3172) -- 添加了 `formatDateTime` 函数. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/2770) -- 增加了`JDBC`表引擎和`jdbc`表功能 (需要安装clickhouse-jdbc-bridge) . [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3210) -- 添加了用于处理 ISO 周数的函数:`toISOWeek`、`toISOYear`、`toStartOfISOYear` 和 `toDayOfYear`. [#3146](https://github.com/ClickHouse/ClickHouse/pull/3146) -- 现在您可以为 `MySQL` 和 `ODBC` 表使用 `Nullable` 列. [#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) -- 嵌套数据结构可以作为 `JSONEachRow` 格式的嵌套对象读取. 添加了 `input_format_import_nested_json` 设置. [Veloman Yunkan](https://github.com/ClickHouse/ClickHouse/pull/3144) -- 插入数据时,许多`MATERIALIZED VIEW` 都可以进行并行处理. 请参阅`parallel_view_processing` 设置. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3208) -- 添加了 `SYSTEM FLUSH LOGS` 查询 (强制日志刷新到系统表,例如 `query_log`) [#3321](https://github.com/ClickHouse/ClickHouse/pull/3321) -- 现在你可以在声明 `Replicated` 表时使用预定义的 `database` 和 `table` 宏. [#3251](https://github.com/ClickHouse/ClickHouse/pull/3251) -- 添加了以工程符号 (表示十的幂) 读取 `Decimal` 类型值的能力. [#3153](https://github.com/ClickHouse/ClickHouse/pull/3153) - -#### 实验特征: {#experimental-features} - -- `LowCardinality` 数据类型的 GROUP BY 子句优化. [#3138](https://github.com/ClickHouse/ClickHouse/pull/3138) -- `LowCardinality` 数据类型表达式的优化计算. [#3200](https://github.com/ClickHouse/ClickHouse/pull/3200) - -#### 改进: {#improvements-2} - -- 使用 `ORDER BY` 和 `LIMIT` 显着减少了查询的内存消耗. 查看 `max_bytes_before_remerge_sort` 设置. [#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) -- 如果没有`JOIN` (`LEFT`,`INNER`,...), 则假定`INNER JOIN`. [#3147](https://github.com/ClickHouse/ClickHouse/pull/3147) -- 合格的星号在使用 `JOIN` 的查询中正常工作. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3202) -- `ODBC` 表引擎正确选择引用远程数据库的 SQL 方言中的标识符的方法. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3210) -- `compile_expressions` 设置 (表达式的 JIT 编译) 默认启用. -- 修复了同时 DROP DATABASE/TABLE IF EXISTS 和 CREATE DATABASE/TABLE IF NOT EXISTS 的行为. 以前, `CREATE DATABASE ... IF NOT EXISTS` 查询可能会返回错误消息"File ... already exists", 而`CREATE TABLE ... IF NOT EXISTS` 和 `DROP TABLE IF EXISTS` 查询可能会返回 `Table ...现在正在创建或附加`. [#3101](https://github.com/ClickHouse/ClickHouse/pull/3101) -- 当从 MySQL 或 ODBC 表查询时, 带有常量右半部分的 LIKE 和 IN 表达式被传递到远程服务器. [#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) -- 从 MySQL 和 ODBC 表查询时, 与 WHERE 子句中的常量表达式的比较将传递到远程服务器. 以前,只通过与常量的比较. [#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) -- 正确计算终端中 `Pretty` 格式的行宽,包括带有象形文字的字符串. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3257). -- 可以为 `ALTER UPDATE` 查询指定 `ON CLUSTER`. -- 提高了读取 `JSONEachRow` 格式数据的性能. [#3332](https://github.com/ClickHouse/ClickHouse/pull/3332) -- 为兼容性添加了`LENGTH` 和`CHARACTER_LENGTH` 函数的同义词. `CONCAT` 函数不再区分大小写. [#3306](https://github.com/ClickHouse/ClickHouse/pull/3306) -- 添加了`DateTime` 类型的`TIMESTAMP` 同义词. [#3390](https://github.com/ClickHouse/ClickHouse/pull/3390) -- 服务器日志中始终为 query_id 保留空间, 即使日志行与查询无关. 这使得使用第三方工具解析服务器文本日志变得更容易. -- 当查询超过整数 GB 的下一个级别时, 会记录查询的内存消耗. [#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) -- 为使用 Native 协议的客户端库错误发送的列数少于服务器预期的 INSERT 查询的情况添加了兼容模式. 这种情况在使用 clickhouse-cpp 库时是可能的. 之前这种场景导致服务器崩溃. [#3171](https://github.com/ClickHouse/ClickHouse/pull/3171) -- 在`clickhouse-copier` 中用户定义的WHERE 表达式中, 您现在可以使用`partition_key` 别名(用于按源表分区进行额外过滤). 如果分区方案在复制过程中发生变化, 但只是轻微变化, 这将很有用. [#3166](https://github.com/ClickHouse/ClickHouse/pull/3166) -- `Kafka` 引擎的工作流已移至后台线程池, 以便在高负载时自动降低数据读取速度. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3215). -- 支持以 `Cap'n'Proto format` 读取结构的 `Tuple` 和 `Nested` 值,例如 `struct`. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3216) -- `firstSignificantSubdomain` 函数的顶级域列表现在包括域 `biz`. [decaseal](https://github.com/ClickHouse/ClickHouse/pull/3219) -- 在外部字典的配置中, `null_value` 被解释为默认数据类型的值. [#3330](https://github.com/ClickHouse/ClickHouse/pull/3330) -- 支持 `Decimal` 的 `intDiv` 和 `intDivOrZero` 函数. [b48402e8](https://github.com/ClickHouse/ClickHouse/commit/b48402e8712e2b9b151e0eef8193811d433a1264) -- 支持`Date`、`DateTime`、`UUID`和 `Decimal`类型作为`sumMap`聚合函数的键. [#3281](https://github.com/ClickHouse/ClickHouse/pull/3281) -- 支持外部字典中的 `Decimal` 数据类型. [#3324](https://github.com/ClickHouse/ClickHouse/pull/3324) -- 支持 `SummingMergeTree` 表中的 `Decimal` 数据类型. [#3348](https://github.com/ClickHouse/ClickHouse/pull/3348) -- 在 `if` 中添加了针对 `UUID` 的专业化. [#3366](https://github.com/ClickHouse/ClickHouse/pull/3366) -- 从 `MergeTree 表` 读取时减少了 `open` 和 `close` 系统调用的数量. [#3283](https://github.com/ClickHouse/ClickHouse/pull/3283) -- `TRUNCATE TABLE` 查询可以在任何副本上执行 (查询被传递给领导副本). [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3375) - -#### Bug 修复: {#bug-fixes-10} - -- 修复了`range_hashed` 词典的`Dictionary` 表问题。 此错误发生在 18.12.17 版本中. [#1702](https://github.com/ClickHouse/ClickHouse/pull/1702) -- 修复了加载`range_hashed` 字典时的错误 (消息`Unsupported type Nullable (...)`). 此错误发生在 18.12.17 版本中. [#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) -- 修复了`pointInPolygon`函数中由于大量顶点彼此靠近的多边形计算不准确而导致的错误. [#3331](https://github.com/ClickHouse/ClickHouse/pull/3331) [#3341](https://github.com/ClickHouse/ClickHouse/pull/3341) -- 如果合并数据部分后, 结果部分的校验和与另一个副本中相同合并的结果不同, 则删除合并的结果并从另一个副本下载数据部分(这是正确的行为). 但是下载数据部分后, 由于该部分已经存在的错误, 无法将其添加到工作集中(因为数据部分在合并后有一些延迟被删除). 这导致循环尝试下载相同的数据. [#3194](https://github.com/ClickHouse/ClickHouse/pull/3194) -- 修复了查询总内存消耗的错误计算 (由于计算错误,`max_memory_usage_for_all_queries` 设置工作不正确,`MemoryTracking` 指标的值不正确). 此错误发生在 18.12.13 版本中. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3344) -- 修复了 `CREATE TABLE ... ON CLUSTER ... AS SELECT ...` 的功能.此错误发生在 18.12.13 版本. [#3247](https://github.com/ClickHouse/ClickHouse/pull/3247) -- 修复了如果 `JOIN` 仅在远程服务器上执行, 则在启动查询的服务器上为 `JOIN` 准备不必要的数据结构. [#3340](https://github.com/ClickHouse/ClickHouse/pull/3340) -- 修复了 `Kafka` 引擎中的错误:开始读取数据时出现异常后死锁, 完成后锁定. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3215). -- 对于 `Kafka` 表, 未传递可选的 `schema` 参数 (`Cap'n'Proto` 格式的模式). [Vojtech Splichal](https://github.com/ClickHouse/ClickHouse/pull/3150) -- 如果 ZooKeeper 服务器的集合具有接受连接但随后立即关闭它而不是响应握手的服务器, 则 ClickHouse 选择连接另一台服务器. 以前, 这会产生错误 `无法读取所有数据. 读取的字节数:0.预期的字节数:4`. 并且服务器无法启动. [8218cf3a](https://github.com/ClickHouse/ClickHouse/commit/8218cf3a5f39a43401953769d6d12a0bb8d29da9) -- 如果 ZooKeeper 服务器的集合包含 DNS 查询返回错误的服务器, 这些服务器将被忽略. [17b8e209](https://github.com/ClickHouse/ClickHouse/commit/17b8e209221061325ad7ba0539f03c6e65f87f29) -- `VALUES` 格式插入数据时, 修复了 `Date` 和 `DateTime` 之间的类型转换 (如果 `input_format_values_interpret_expressions = 1`). 之前在Unix Epoch时间的天数数值和Unix时间戳之间进行了转换, 导致了意想不到的结果. [#3229](https://github.com/ClickHouse/ClickHouse/pull/3229) -- 更正了 `Decimal` 和整数之间的类型转换. [#3211](https://github.com/ClickHouse/ClickHouse/pull/3211) -- 修复了 `enable_optimize_predicate_expression` 设置中的错误. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3231) -- 修复了 CSV 格式的浮点数解析错误, 如果使用非默认的 CSV 分隔符, 例如 `;`. [#3155](https://github.com/ClickHouse/ClickHouse/pull/3155) -- 修复了 `arrayCumSumNonNegative` 函数 (如果累加器小于零,它不会累加负值). [Aleksey Studnev](https://github.com/ClickHouse/ClickHouse/pull/3163) -- 修复了使用 `PREWHERE` 时 `Merge` 表如何在 `Distributed` 表之上工作. [#3165](https://github.com/ClickHouse/ClickHouse/pull/3165) -- ALTER UPDATE` 查询中的错误修复. -- 修复了 18.12 版本中出现的 `odbc` 表函数中的错误. [#3197](https://github.com/ClickHouse/ClickHouse/pull/3197) -- 修复了使用 `StateArray` 组合器的聚合函数的操作. [#3188](https://github.com/ClickHouse/ClickHouse/pull/3188) -- 修复了将 `Decimal` 值除以零时的崩溃. [69dd6609](https://github.com/ClickHouse/ClickHouse/commit/69dd6609193beb4e7acd3e6ad216eca0ccfb8179) -- 修复了使用 `Decimal` 和整数参数的操作类型的输出. [#3224](https://github.com/ClickHouse/ClickHouse/pull/3224) -- 修复了在 `Decimal128` 上的 `GROUP BY` 期间的段错误. [3359ba06](https://github.com/ClickHouse/ClickHouse/commit/3359ba06c39fcd05bfdb87d6c64154819621e13a) -- `log_query_threads` 设置 (关于查询执行的每个线程的日志信息) 现在只有在 `log_queries` 选项 (关于查询的日志信息) 设置为 1 时才生效. 由于`log_query_threads` 选项默认启用, 即使查询日志被禁用, 有关线程的信息也会被记录. [#3241](https://github.com/ClickHouse/ClickHouse/pull/3241) -- 修复了分位数聚合函数分布式操作的错误 (错误信息`Not found column quantile...`). [292a8855](https://github.com/ClickHouse/ClickHouse/commit/292a885533b8e3b41ce8993867069d14cbd5a664) -- 修复了同时在 18.12.17 版本服务器和旧服务器的集群上工作时的兼容性问题. 对于具有固定长度和非固定长度的 GROUP BY 键的分布式查询, 如果有大量数据要聚合, 则返回的数据并不总是完全聚合 (两个不同的行包含相同的聚合键). [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) -- 如果查询仅包含测试中声明的部分替换,则修复了对 `clickhouse-performance-test` 中替换的处理. [#3263](https://github.com/ClickHouse/ClickHouse/pull/3263) -- 修复了将 `FINAL` 与 `PREWHERE` 结合使用时的错误. [#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) -- 修复了在 `ALTER` 期间添加的列上使用 `PREWHERE` 时的错误. [#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) -- 添加了对`DEFAULT` 和`MATERIALIZED` 表达式是否缺少`arrayJoin` 的检查. 以前, `arrayJoin` 在插入数据时会导致错误. [#3337](https://github.com/ClickHouse/ClickHouse/pull/3337) -- 在`PREWHERE` 子句中添加了对缺少`arrayJoin` 的检查. 以前, 这会导致在执行查询时出现诸如 `Size ...不匹配` 或 `未知压缩方法` 之类的消息. [#3357](https://github.com/ClickHouse/ClickHouse/pull/3357) -- 修复了在用相应的 IN 表达式替换来自相等评估的 AND 链的优化后在极少数情况下可能发生的段错误. [liuyimin-bytedance](https://github.com/ClickHouse/ClickHouse/pull/3339) -- 对`clickhouse-benchmark` 的小修正: 以前, 客户端信息没有发送到服务器; 现在在关闭和限制迭代次数时更准确地计算执行的查询数. [#3351](https://github.com/ClickHouse/ClickHouse/pull/3351) [#3352](https://github.com/ClickHouse/ClickHouse/pull/3352) - -#### 向后不兼容的更新: {#backward-incompatible-changes-1} - -- 删除了 `allow_experimental_decimal_type` 选项. `Decimal` 数据类型可用于默认使用. [#3329](https://github.com/ClickHouse/ClickHouse/pull/3329) - -## ClickHouse 版本 18.12 {#clickhouse-release-18-12} - -### ClickHouse 版本 18.12.17, 2018-09-16 {#clickhouse-release-18-12-17-2018-09-16} - -#### 新特征: {#new-features-2} - -- 为 `clickhouse` 源实现了 `invalidate_query` (指定查询以检查是否需要更新外部字典的能力). [#3126](https://github.com/ClickHouse/ClickHouse/pull/3126) -- 添加了使用`UInt*`、`Int*` 和`DateTime` 数据类型 (连同`Date` 类型) 作为定义范围边界的`range_hashed` 外部字典键的功能. 现在可以使用 `NULL` 来指定一个开放范围. [Vasily Nemkov](https://github.com/ClickHouse/ClickHouse/pull/3123) -- `Decimal` 类型现在支持 `var*` 和 `stddev*` 聚合函数. [#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) -- `Decimal` 类型现在支持数学函数(`exp`、`sin` 等). [#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) -- `system.part_log` 表现在有 `partition_id` 列. [#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) - -#### Bug 修复: {#bug-fixes-11} - -- `Merge` 现在可以在 `Distributed` 表上正常工作. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3159) -- 修复了导致无法在 `Ubuntu Precise` 和旧版本上运行 ClickHouse 的不兼容性 (对 `glibc` 版本的不必要依赖). 18.12.13 版本出现不兼容. [#3130](https://github.com/ClickHouse/ClickHouse/pull/3130) -- 修复了 `enable_optimize_predicate_expression` 设置中的错误. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) -- 修复了在 18.12.13 之前的版本上使用副本集群并同时在具有较新版本的服务器上创建表的新副本时出现的向后兼容性小问题 (显示在消息 `无法克隆副本, 因为......更新为新的 ClickHouse 版本`, 这是合乎逻辑的, 但不应该发生). [#3122](https://github.com/ClickHouse/ClickHouse/pull/3122) - -#### 向后不兼容的更新: {#backward-incompatible-changes-2} - -- `enable_optimize_predicate_expression` 选项默认启用 (相当乐观). 如果出现与搜索列名相关的查询分析错误, 请将 `enable_optimize_predicate_expression` 设置为0. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) - -### ClickHouse 版本 18.12.14, 2018-09-13 {#clickhouse-release-18-12-14-2018-09-13} - -#### 新特征: {#new-features-3} - -- 添加了对 `ALTER UPDATE` 查询的支持. [#3035](https://github.com/ClickHouse/ClickHouse/pull/3035) -- 添加了 `allow_ddl` 选项, 用于限制用户对 DDL 查询的访问. [#3104](https://github.com/ClickHouse/ClickHouse/pull/3104) -- 为`MergeTree`引擎添加了`min_merge_bytes_to_use_direct_io`选项,它允许您设置合并总大小的阈值(当超过阈值时,数据部分文件将使用O_DIRECT处理). [#3117](https://github.com/ClickHouse/ClickHouse/pull/3117) -- `system.merges` 系统表现在包含 `partition_id` 列. [#3099](https://github.com/ClickHouse/ClickHouse/pull/3099) - -#### 改进 {#improvements-3} - -- 如果数据部分在变异期间保持不变,则副本不会下载它. [#3103](https://github.com/ClickHouse/ClickHouse/pull/3103) -- 使用 `clickhouse-client` 时,设置名称可自动完成. [#3106](https://github.com/ClickHouse/ClickHouse/pull/3106) - -#### Bug 修复: {#bug-fixes-12} - -- 添加了对插入时作为`Nested` 类型字段元素的数组大小的检查. [#3118](https://github.com/ClickHouse/ClickHouse/pull/3118) -- 修复了使用 `ODBC` 源和 `hashed` 存储更新外部词典的错误. 此错误发生在 18.12.13 版本中. -- 修复了从具有 `IN` 条件的查询创建临时表时崩溃的问题. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3098) -- 修复了可以包含 `NULL` 元素的数组的聚合函数中的错误. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3097) - -### ClickHouse 版本 18.12.13, 2018-09-10 {#clickhouse-release-18-12-13-2018-09-10} - -#### 新特征: {#new-features-4} - -- 添加了`DECIMAL(digits, scale)`数据类型(`Decimal32(scale)`、`Decimal64(scale)`、`Decimal128(scale)`). 要启用它, 请使用设置 `allow_experimental_decimal_type`. [#2846](https://github.com/ClickHouse/ClickHouse/pull/2846) [#2970](https://github.com/ClickHouse/ClickHouse/pull/2970) [#3008](https://github.com/ClickHouse/ClickHouse/pull/3008) [#3047](https://github.com/ClickHouse/ClickHouse/pull/3047) -- 用于`GROUP BY` 的新`WITH ROLLUP` 修饰符 (替代语法:`GROUP BY ROLLUP(...)`). [#2948](https://github.com/ClickHouse/ClickHouse/pull/2948) -- 在使用 JOIN 的查询中,星号扩展为所有表中的列列表, 符合 SQL 标准. 您可以通过在用户配置级别将 `asterisk_left_columns_only` 设置为 1 来恢复旧行为. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2787) -- 添加了对带有表函数的 JOIN 的支持. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2907) -- 通过在 clickhouse-client 中按 Tab 自动完成. [Sergey Shcherbin](https://github.com/ClickHouse/ClickHouse/pull/2447) -- clickhouse-client 中的 Ctrl+C 清除输入的查询. [#2877](https://github.com/ClickHouse/ClickHouse/pull/2877) -- 添加了`join_default_strictness` 设置 (值:`"`、`'any'`、`'all'`). 这允许您不为`JOIN` 指定`ANY` 或`ALL`. [#2982](https://github.com/ClickHouse/ClickHouse/pull/2982) -- 与查询处理相关的服务器日志的每一行都显示了查询 ID. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- 现在您可以在 clickhouse-client 中获取查询执行日志 (使用 `send_logs_level` 设置). 通过分布式查询处理, 日志从所有服务器级联. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- `system.query_log` 和 `system.processes`(`SHOW PROCESSLIST`) 表现在包含有关运行查询时所有更改设置的信息(`Settings`数据的嵌套结构). 添加了`log_query_settings` 设置. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- `system.query_log` 和 `system.processes` 表现在显示有关参与查询执行的线程数的信息 (请参阅 `thread_numbers` 列). [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- 添加了 `ProfileEvents` 计数器, 用于测量网络读写和磁盘读写所花费的时间、网络错误次数以及网络带宽受限时所花费的等待时间. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- 添加了包含来自 rusage 的系统指标的 `ProfileEvents` 计数器 (您可以使用它们来获取有关用户空间和内核中 CPU 使用率、页面错误和上下文切换的信息), 以及 taskstats 指标 (使用这些来获取有关 I /O 等待时间, CPU 等待时间, 以及读取和记录的数据量, 有无页缓存). [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- `ProfileEvents` 计数器适用于全局和每个查询, 以及每个查询执行线程, 它允许您按查询详细分析资源消耗. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- 添加了 `system.query_thread_log` 表, 其中包含有关每个查询执行线程的信息. 添加了`log_query_threads` 设置. [#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) -- `system.metrics` 和 `system.events` 表现在有内置文档. [#3016](https://github.com/ClickHouse/ClickHouse/pull/3016) -- 添加了`arrayEnumerateDense`函数. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2975) -- 添加了 `arrayCumSumNonNegative` 和 `arrayDifference` 函数. [Aleksey Studnev](https://github.com/ClickHouse/ClickHouse/pull/2942) -- 添加了`retention`聚合函数. [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2887) -- 现在您可以使用加号运算符添加 (合并) 聚合函数的状态, 并将聚合函数的状态乘以非负常数. [#3062](https://github.com/ClickHouse/ClickHouse/pull/3062) [#3034](https://github.com/ClickHouse/ClickHouse/pull/3034) -- MergeTree 系列中的表现在具有虚拟列 `_partition_id`. [#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) - -#### 实验功能: {#experimental-features-1} - -- 添加了 `LowCardinality(T)` 数据类型. 此数据类型会自动创建值的本地字典, 并允许在不解压字典的情况下进行数据处理. [#2830](https://github.com/ClickHouse/ClickHouse/pull/2830) -- 添加了 JIT 编译函数的缓存和编译前使用次数的计数器. 要 JIT 编译表达式, 请启用 `compile_expressions` 设置. [#2990](https://github.com/ClickHouse/ClickHouse/pull/2990) [#3077](https://github.com/ClickHouse/ClickHouse/pull/3077) - -#### 改进: {#improvements-4} - -- 修复了有废弃副本时复制日志无限累积的问题. 为延迟较长的副本添加了有效的恢复模式. -- 当其中一个是字符串而其他是固定长度时,具有多个聚合字段的`GROUP BY` 的性能得到改善. -- 使用 `PREWHERE` 和在 `PREWHERE` 中隐式传输表达式时提高了性能. -- 改进了文本格式的解析性能 (`CSV`, `TSV`). [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2977) [#2980](https://github.com/ClickHouse/ClickHouse/pull/2980) -- 改进了读取二进制格式的字符串和数组的性能. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2955) -- 当单个服务器上有大量表时,提高性能并减少对 `system.tables` 和 `system.columns` 的查询的内存消耗. [#2953](https://github.com/ClickHouse/ClickHouse/pull/2953) -- 修复了大量查询导致错误时的性能问题 (`_dl_addr`函数在`perf top`中可见,但服务器没有使用太多CPU). [#2938](https://github.com/ClickHouse/ClickHouse/pull/2938) -- 条件被投射到视图中 (当启用`enable_optimize_predicate_expression` 时). [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2907) -- 对 `UUID` 数据类型功能的改进. [#3074](https://github.com/ClickHouse/ClickHouse/pull/3074) [#2985](https://github.com/ClickHouse/ClickHouse/pull/2985) -- The-Alchemist 词典支持 `UUID` 数据类型. [#2822](https://github.com/ClickHouse/ClickHouse/pull/2822) -- `visitParamExtractRaw` 函数可以正确处理嵌套结构. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2974) -- 启用 `input_format_skip_unknown_fields` 设置时, 会正确跳过 `JSONEachRow` 格式的对象字段. [BlahGeek](https://github.com/ClickHouse/ClickHouse/pull/2958) -- 对于带有条件的 `CASE` 表达式, 您现在可以省略 `ELSE`, 它等价于 `ELSE NULL`. [#2920](https://github.com/ClickHouse/ClickHouse/pull/2920) -- 现在可以在使用 ZooKeeper 时配置操作超时. [urykhy](https://github.com/ClickHouse/ClickHouse/pull/2971) -- 您可以将 `LIMIT n, m` 的偏移量指定为 `LIMIT n OFFSET m`. [#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) -- 您可以使用 `SELECT TOP n` 语法作为 `LIMIT` 的替代. [#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) -- 增加了写入系统表的队列大小, 因此不会经常发生`SystemLog parameter queue is full`错误. -- `windowFunnel` 聚合函数现在支持满足多个条件的事件. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2801) -- 重复列可用于 `JOIN` 的 `USING` 子句中. [#3006](https://github.com/ClickHouse/ClickHouse/pull/3006) -- Pretty` 格式现在对按宽度对齐的列有限制. 使用 `output_format_pretty_max_column_pad_width` 设置. 如果一个值更宽, 它仍然会完整显示, 但表格中的其他单元格不会太宽. [#3003](https://github.com/ClickHouse/ClickHouse/pull/3003) -- `odbc` 表函数现在允许您指定数据库/模式名称. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2885) -- 添加了使用在 `clickhouse-client` 配置文件中指定的用户名的功能. [Vladimir Kozbin](https://github.com/ClickHouse/ClickHouse/pull/2909) -- `ZooKeeperExceptions` 计数器被分成三个计数器:`ZooKeeperUserExceptions`、`ZooKeeperHardwareExceptions` 和 `ZooKeeperOtherExceptions`. -- `ALTER DELETE` 查询适用于物化视图. -- 为 `ReplicatedMergeTree` 表定期运行清理线程时添加了随机化, 以避免在有大量 `ReplicatedMergeTree` 表时出现周期性负载峰值. -- 支持 `ATTACH TABLE ... ON CLUSTER` 查询. [#3025](https://github.com/ClickHouse/ClickHouse/pull/3025) - -#### Bug 修复: {#bug-fixes-13} - -- 修复了 `Dictionary` 表的问题 (抛出 `Size of offsets does not match size of column` 或 `Unknown compression method` 异常). 这个bug出现在18.10.3版本. [#2913](https://github.com/ClickHouse/ClickHouse/issues/2913) -- 修复了合并 `CollapsingMergeTree` 表时, 如果其中一个数据部分为空 (如果所有数据都被删除,这些部分是在合并或`ALTER DELETE`期间形成的), 并且使用 `vertical` 算法进行合并. [#3049](https://github.com/ClickHouse/ClickHouse/pull/3049) -- 修复了在 `DROP` 或 `TRUNCATE` 期间为 `Memory` 表同时执行 `SELECT` 的竞争条件, 这可能导致服务器崩溃. 这个bug出现在1.1.54388版本. [#3038](https://github.com/ClickHouse/ClickHouse/pull/3038) -- 修复了在返回 `Session is expired` 错误时插入 `Replicated` 表时数据丢失的可能性 (数据丢失可以通过 `ReplicatedDataLoss` 指标检测). 此错误发生在 1.1.54378 版本中. [#2939](https://github.com/ClickHouse/ClickHouse/pull/2939) [#2949](https://github.com/ClickHouse/ClickHouse/pull/2949) [#2964](https://github.com/ClickHouse/ClickHouse/pull/2964) -- 修复了`JOIN ... ON` 期间的段错误. [#3000](https://github.com/ClickHouse/ClickHouse/pull/3000) -- 修复了当`WHERE`表达式完全由一个限定的列名组成时搜索列名的错误,例如`WHERE table.column`. [#2994](https://github.com/ClickHouse/ClickHouse/pull/2994) -- 修复了如果从远程服务器请求包含带有子查询的 IN 表达式的单个列,则在执行分布式查询时发生的 `Not found column` 错误. [#3087](https://github.com/ClickHouse/ClickHouse/pull/3087) -- 修正了 `UNION流中的块结构不匹配: 列数不同` 错误, 如果一个分片是本地的, 而另一个不是, 分布式查询发生, 并优化移动到 `PREWHERE` 被触发. [#2226](https://github.com/ClickHouse/ClickHouse/pull/2226) [#3037](https://github.com/ClickHouse/ClickHouse/pull/3037) [#3055](https://github.com/ClickHouse/ClickHouse/pull/3055) [#3065](https://github.com/ClickHouse/ClickHouse/pull/3065) [#3073](https://github.com/ClickHouse/ClickHouse/pull/3073) [#3090](https://github.com/ClickHouse/ClickHouse/pull/3090) [#3093](https://github.com/ClickHouse/ClickHouse/pull/3093) -- 修复了某些非凸多边形情况下的 `pointInPolygon` 函数. [#2910](https://github.com/ClickHouse/ClickHouse/pull/2910) -- 修复了将 `nan` 与整数进行比较时的错误结果. [#3024](https://github.com/ClickHouse/ClickHouse/pull/3024) -- 修复了在极少数情况下可能导致段错误的 `zlib-ng` 库中的错误. [#2854](https://github.com/ClickHouse/ClickHouse/pull/2854) -- 修复了在插入到具有 `AggregateFunction` 列的表时, 如果聚合函数的状态不简单 (单独分配内存) , 并且如果单个插入请求导致多个小块时的内存泄漏. [#3084](https://github.com/ClickHouse/ClickHouse/pull/3084) -- 修复了同时创建和删除同一个 `Buffer` 或 `MergeTree` 表时的竞争条件. -- 修复了比较由某些非平凡类型 (例如元组) 组成的元组时出现段错误的可能性. [#2989](https://github.com/ClickHouse/ClickHouse/pull/2989) -- 修复了运行某些 `ON CLUSTER` 查询时出现段错误的可能性. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2960) -- 修复了 `arrayDistinct` 函数中 `Nullable` 数组元素的错误. [#2845](https://github.com/ClickHouse/ClickHouse/pull/2845) [#2937](https://github.com/ClickHouse/ClickHouse/pull/2937) -- `enable_optimize_predicate_expression` 选项现在可以正确支持带有 `SELECT *` 的情况. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2929) -- 修复了重新初始化 ZooKeeper 会话时的段错误. [#2917](https://github.com/ClickHouse/ClickHouse/pull/2917) -- 修复了使用 ZooKeeper 时潜在的阻塞问题. -- 修复了在 `SummingMergeTree` 中添加嵌套数据结构的错误代码. -- 为聚合函数的状态分配内存时, 正确地考虑了对齐, 这使得在实现聚合函数的状态时使用需要对齐的操作成为可能. [chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/2808) - -#### 安全修复: {#security-fix} - -- 安全使用 ODBC 数据源. 与 ODBC 驱动程序的交互使用单独的 `clickhouse-odbc-bridge` 过程. 第三方 ODBC 驱动程序中的错误不再导致服务器稳定性或漏洞问题. [#2828](https://github.com/ClickHouse/ClickHouse/pull/2828) [#2879](https://github.com/ClickHouse/ClickHouse/pull/2879) [#2886](https://github.com/ClickHouse/ClickHouse/pull/2886) [#2893](https://github.com/ClickHouse/ClickHouse/pull/2893) [#2921](https://github.com/ClickHouse/ClickHouse/pull/2921) -- 修复了 `catBoostPool` 表函数中文件路径的错误验证. [#2894](https://github.com/ClickHouse/ClickHouse/pull/2894) -- 系统表的内容 (`tables`、`databases`、`parts`、`columns`、`parts_columns`、`merges`、`mutations`、`replicas` 和 `replication_queue`) 根据用户配置进行过滤访问数据库(`allow_databases`). [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2856) - -#### 向后不兼容的更新: {#backward-incompatible-changes-3} - -- 在使用 JOIN 的查询中, 星号扩展为所有表中的列列表, 符合 SQL 标准. 您可以通过在用户配置级别将 `asterisk_left_columns_only` 设置为 1 来恢复旧行为. - -#### Build 更新: {#build-changes-2} - -- 大多数集成测试现在可以通过提交运行. -- 代码样式检查也可以通过提交运行. -- 在 CentOS7/Fedora 上构建时正确选择了 `memcpy` 实现. [Etienne Champetier](https://github.com/ClickHouse/ClickHouse/pull/2912) -- When using clang to build, some warnings from `-Weverything` have been added, in addition to the regular `-Wall-Wextra -Werror`. [#2957](https://github.com/ClickHouse/ClickHouse/pull/2957) -- 调试构建使用 `jemalloc` 调试选项. -- 与 ZooKeeper 交互的库的接口被声明为抽象的. [#2950](https://github.com/ClickHouse/ClickHouse/pull/2950) - -## ClickHouse 版本 18.10 {#clickhouse-release-18-10} - -### ClickHouse 版本 18.10.3, 2018-08-13 {#clickhouse-release-18-10-3-2018-08-13} - -#### 新特征: {#new-features-5} - -- HTTPS 可用于复制. [#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) -- 除了现有的 `murmurHash2_32` 之外, 还添加了函数 `murmurHash2_64`、`murmurHash3_32`、`murmurHash3_64` 和 `murmurHash3_128`. [#2791](https://github.com/ClickHouse/ClickHouse/pull/2791) -- 支持 ClickHouse ODBC 驱动程序中的 Nullable 类型 (`ODBCDriver2` 输出格式). [#2834](https://github.com/ClickHouse/ClickHouse/pull/2834) -- 支持键列中的 `UUID`. - -#### 改进: {#improvements-5} - -- 从配置文件中删除集群时,无需重新启动服务器即可删除集群. [#2777](https://github.com/ClickHouse/ClickHouse/pull/2777) -- 从配置文件中删除外部字典时,无需重新启动服务器即可删除它们. [#2779](https://github.com/ClickHouse/ClickHouse/pull/2779) -- 为 `Kafka` 表引擎添加了 `SETTINGS` 支持. [Alexander Marshalov](https://github.com/ClickHouse/ClickHouse/pull/2781) -- `UUID` 数据类型的改进 (尚未完成). [#2618](https://github.com/ClickHouse/ClickHouse/pull/2618) -- 支持在 `SummingMergeTree`、`CollapsingMergeTree` 和 `VersionedCollapsingMergeTree` 引擎中合并后的空部分. [#2815](https://github.com/ClickHouse/ClickHouse/pull/2815) -- 已完成突变的旧记录被删除 (`ALTER DELETE`). [#2784](https://github.com/ClickHouse/ClickHouse/pull/2784) -- 添加了 `system.merge_tree_settings` 表. [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/2841) -- `system.tables` 表现在有依赖列: `dependencies_database` 和 `dependencies_table`. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2851) -- 添加了 `max_partition_size_to_drop` 配置选项. [#2782](https://github.com/ClickHouse/ClickHouse/pull/2782) -- 添加了 `output_format_json_escape_forward_slashes` 选项. [Alexander Bocharov](https://github.com/ClickHouse/ClickHouse/pull/2812) -- 添加了 `max_fetch_partition_retries_count` 设置. [#2831](https://github.com/ClickHouse/ClickHouse/pull/2831) -- 添加了 `prefer_localhost_replica` 设置, 用于禁用本地副本的首选项并在没有进程间交互的情况下转到本地副本. [#2832](https://github.com/ClickHouse/ClickHouse/pull/2832) -- 在对空的 `Float32` 或 `Float64` 集合进行聚合的情况下, `quantileExact` 聚合函数返回 `nan` . [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2855) - -#### Bug 修复: {#bug-fixes-14} - -- 删除了 ODBC 连接字符串参数的不必要转义, 这使得无法建立连接. 此错误发生在 18.6.0 版本中. -- 修复了处理复制队列中 `REPLACE PARTITION` 命令的逻辑. 如果同一个分区有两个 `REPLACE` 命令, 不正确的逻辑可能会导致其中一个留在复制队列中而不会被执行. [#2814](https://github.com/ClickHouse/ClickHouse/pull/2814) -- 修复了所有数据部分为空时的合并错误 (如果所有数据都被删除, 则由合并或 `ALTER DELETE` 形成的部分). 这个bug出现在18.1.0版本. [#2930](https://github.com/ClickHouse/ClickHouse/pull/2930) -- 修复了并发 `Set` 或 `Join` 的错误. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2823) -- 修正了 `UNION流中的块结构不匹配: 列数不同` 错误, 如果一个 `SELECT` 查询包含重复的列名, 在子查询 `UNION ALL` 查询中发生的错误. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2094) -- 修复了连接 MySQL 服务器时发生异常时的内存泄漏. -- 修复了查询错误时不正确的 clickhouse-client 响应代码. -- 修复了包含 DISTINCT 的物化视图的错误行为. [#2795](https://github.com/ClickHouse/ClickHouse/issues/2795) - -#### 向后不兼容的更新: {#backward-incompatible-changes-4} - -- 删除了对分布式表的 CHECK TABLE 查询的支持. - -#### Build 更新: {#build-changes-3} - -- 分配器已被替换: 现在使用 `jemalloc` 代替 `tcmalloc`. 在某些情况下, 这将速度提高到 20%. 但是, 有些查询速度降低了 20%. 部分场景内存消耗降低约10%, 稳定性提升. 在竞争激烈的负载下, 用户空间和系统中的 CPU 使用率仅略有增加. [#2773](https://github.com/ClickHouse/ClickHouse/pull/2773) -- 使用子模块中的 libressl. [#1983](https://github.com/ClickHouse/ClickHouse/pull/1983) [#2807](https://github.com/ClickHouse/ClickHouse/pull/2807) -- 从子模块使用 unixodbc. [#2789](https://github.com/ClickHouse/ClickHouse/pull/2789) -- 使用子模块中的 mariadb-connector-c. [#2785](https://github.com/ClickHouse/ClickHouse/pull/2785) -- 将依赖于测试数据可用性的功能测试文件添加到存储库中 (暂时没有测试数据本身). - -## ClickHouse 版本 18.6 {#clickhouse-release-18-6} - -### ClickHouse 版本 18.6.0, 2018-08-02 {#clickhouse-release-18-6-0-2018-08-02} - -#### 新特征: {#new-features-6} - -- 添加了对 JOIN ON 语法的 ON 表达式的支持: - `JOIN ON Expr([table.]column ...) = Expr([table.]column, ...) [AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]` - 表达式必须是由 AND 运算符连接的等式链. 等式的每一边都可以是一个表的列上的任意表达式. 支持对右表使用完全限定的列名 (`table.name`、`database.table.name`、`table_alias.name`、`subquery_alias.name`). [#2742](https://github.com/ClickHouse/ClickHouse/pull/2742) -- 可以启用 HTTPS 进行复制. [#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) - -#### 改进: {#improvements-6} - -- 服务器将其版本的补丁组件传递给客户端. 关于补丁版本组件的数据在 `system.processes` 和 `query_log` 中. [#2646](https://github.com/ClickHouse/ClickHouse/pull/2646) - -## ClickHouse 版本 18.5 {#clickhouse-release-18-5} - -### ClickHouse 版本 18.5.1, 2018-07-31 {#clickhouse-release-18-5-1-2018-07-31} - -#### 新特征: {#new-features-7} - -- 添加了哈希函数 `murmurHash2_32`. [#2756](https://github.com/ClickHouse/ClickHouse/pull/2756). - -#### 改进: {#improvements-7} - -- 现在您可以使用 `from_env` [#2741](https://github.com/ClickHouse/ClickHouse/pull/2741) 属性从环境变量设置配置文件中的值. -- 添加了不区分大小写的 `coalesce`、`ifNull` 和 `nullIf 函数` [#2752](https://github.com/ClickHouse/ClickHouse/pull/2752). - -#### Bug 修复: {#bug-fixes-15} - -- 修复了启动副本时可能出现的错误 [#2759](https://github.com/ClickHouse/ClickHouse/pull/2759). - -## ClickHouse 版本 18.4 {#clickhouse-release-18-4} - -### ClickHouse 版本 18.4.0, 2018-07-28 {#clickhouse-release-18-4-0-2018-07-28} - -#### 新特征: {#new-features-8} - -- 添加系统表: `formats`、`data_type_families`、`aggregate_function_combinators`、`table_functions`、`table_engines`、`collations` [#2721](https://github.com/ClickHouse/ClickHouse/pull/2721). -- 添加了使用表函数而不是表作为 `remote` 或`cluster table function` 的参数的功能 [#2708](https://github.com/ClickHouse/ClickHouse/pull/2708). -- 支持复制协议中的 `HTTP Basic` 身份验证 [#2727](https://github.com/ClickHouse/ClickHouse/pull/2727). -- `has` 函数现在允许在 `Enum` 值数组中搜索数值 [Maxim Khrisanfov](https://github.com/ClickHouse/ClickHouse/pull/2699). -- 支持在从 `Kafka` 读取时添加任意消息分隔符 [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2701). - -#### 改进: {#improvements-8} - -- `ALTER TABLE t DELETE WHERE` 查询不会重写不受 WHERE 条件影响的数据部分 [#2694](https://github.com/ClickHouse/ClickHouse/pull/2694). -- `ReplicatedMergeTree` 表的 `use_minimalistic_checksums_in_zookeeper` 选项默认启用. 此设置是在 1.1.54378, 2018-04-16 版本中添加的. 不能再安装早于 1.1.54378 的版本. -- 支持运行指定 `ON CLUSTER` 的 `KILL` 和 `OPTIMIZE` 查询 [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2689). - -#### Bug 修复: {#bug-fixes-16} - -- 修复了错误 `Column ... is not under a aggregate function and not in GROUP BY` 用于使用 IN 表达式进行聚合. 这个bug出现在18.1.0版本 ([bbdd780b](https://github.com/ClickHouse/ClickHouse/commit/bbdd780be0be06a0f336775941cdd536878dd2c2)) -- 修复了 `windowFunnel aggregate function` 中的一个错误 [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2735). -- 修复了 `anyHeavy` 聚合函数中的错误 ([a2101df2](https://github.com/ClickHouse/ClickHouse/commit/a2101df25a6a0fba99aa71f8793d762af2b801ee)) -- 修复了使用 `countArray()` 聚合函数时服务器崩溃的问题. - -#### 向后不兼容的更新: {#backward-incompatible-changes-5} - -- `Kafka` 引擎的参数已从 `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_schema, kafka_num_consumers])` 更改为 `Kafka(kafka_broker_list, kafka_topic_list, kafka_topic_list, kafka_fka_name_schema_fka_format, kafka_fka_name_schema_group_format[, kafka_schema, kafka_num_consumers])` 如果您的表使用`kafka_schema` 或`kafka_num_consumers` 参数, 则必须手动编辑元数据文件`path/metadata/database/table.sql` 并添加具有`''` 值的`kafka_row_delimiter` 参数. - -## ClickHouse 版本 18.1 {#clickhouse-release-18-1} - -### ClickHouse 版本 18.1.0, 2018-07-23 {#clickhouse-release-18-1-0-2018-07-23} - -#### 新特征: {#new-features-9} - -- 支持对非复制 MergeTree 表的 `ALTER TABLE t DELETE WHERE` 查询 ([#2634](https://github.com/ClickHouse/ClickHouse/pull/2634)). -- 支持 `uniq*` 聚合函数系列的任意类型 ([#2010](https://github.com/ClickHouse/ClickHouse/issues/2010)). -- 支持比较运算符中的任意类型 ([#2026](https://github.com/ClickHouse/ClickHouse/issues/2026)). -- `users.xml` 文件允许设置格式为 `10.0.0.1/255.255.255.0` 的子网掩码. 这对于在中间为 0 的 IPv6 网络使用掩码是必要的 ([#2637](https://github.com/ClickHouse/ClickHouse/pull/2637)). -- 添加了 `arrayDistinct` 函数 ([#2670](https://github.com/ClickHouse/ClickHouse/pull/2670)). -- SummingMergeTree 引擎现在可以处理 AggregateFunction 类型的列 ([Constantin S. Pan](https://github.com/ClickHouse/ClickHouse/pull/2566)). - -#### 改进: {#improvements-9} - -- 更改了发布版本的编号方案. 现在第一部分包含发布年份 (莫斯科时区, 公元2000年), 第二部分包含主要更改的编号(大多数版本增加), 第三部分是补丁版本. 除非在变更日志中另有说明.否则版本仍然向后兼容. -- 更快地将浮点数转换为字符串. ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2664)). -- 如果在插入期间由于解析错误而跳过了某些行 (这在启用 `input_allow_errors_num` 和 `input_allow_errors_ratio` 设置的情况下是可能的), 跳过的行数现在会写入服务器日志([Leonardo Cecchi](https://github.com/ClickHouse/ClickHouse/pull/2669)). - -#### Bug 修复: {#bug-fixes-17} - -- 修复了临时表的 TRUNCATE 命令. ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2624)). -- 修复了 ZooKeeper 客户端库中读取响应时出现网络错误时发生的罕见死锁 ([c315200](https://github.com/ClickHouse/ClickHouse/commit/c315200e64b87e44bdf740707fc857d1fdf7e947)). -- 修复了 CAST 到 Nullable 类型期间的错误([#1322](https://github.com/ClickHouse/ClickHouse/issues/1322)). -- 修复了区间边界重合时 `maxIntersection()` 函数的错误结果 ([Michael Furmur](https://github.com/ClickHouse/ClickHouse/pull/2657)). -- 修复了函数参数中 OR 表达式链的错误转换 ([chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/2663)). -- 修复了在另一个子查询中包含 `IN(子查询)` 表达式的查询的性能下降 ([#2571](https://github.com/ClickHouse/ClickHouse/issues/2571)). -- 修复了使用非大写字母的 `CAST` 函数的分布式查询中不同版本服务器之间的不兼容问题 ([fe8c4d6](https://github.com/ClickHouse/ClickHouse/commit/fe8c4d64e434cacd4ceef34faa9005129f2190a5)). -- 添加了对外部 DBMS 的查询标识符的缺失引用 ([#2635](https://github.com/ClickHouse/ClickHouse/issues/2635)). - -#### 向后不兼容的更新: {#backward-incompatible-changes-6} - -- 将包含数字零的字符串转换为 DateTime 不起作用. 示例: `SELECT toDateTime('0')`. 这也是 `DateTime DEFAULT '0'` 在表中不起作用的原因, 以及在字典中 `0` 的原因. 解决方案: 用`0000-00-00 00:00:00`替换`0`. - -## ClickHouse 版本 1.1 {#clickhouse-release-1-1} - -### ClickHouse 版本 1.1.54394, 2018-07-12 {#clickhouse-release-1-1-54394-2018-07-12} - -#### 新特征: {#new-features-10} - -- 添加了 `histogram` 聚合函数 ([Mikhail Surin](https://github.com/ClickHouse/ClickHouse/pull/2521)). -- 现在可以使用 `OPTIMIZE TABLE ... FINAL` 而无需为 `ReplicatedMergeTree` 指定分区 ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2600)). - -#### Bug 修复: {#bug-fixes-18} - -- 修复了发送和下载复制数据时套接字读写超时 (一秒) 非常小的问题, 这使得如果网络或磁盘上有负载, 则无法下载较大的部分 (导致循环尝试 下载部分). 此错误发生在 1.1.54388 版本中. -- 修复了在表中插入重复数据块时在 ZooKeeper 中使用 chroot 时的问题. -- `has` 函数现在可以正确处理具有 Nullable 元素的数组 ([#2115](https://github.com/ClickHouse/ClickHouse/issues/2115)). -- `system.tables` 表在分布式查询中使用时可以正常工作. `metadata_modification_time` 和 `engine_full` 列现在是非虚拟的. 修复了仅从表中查询这些列时发生的错误. -- 修复了插入空数据块后空 `TinyLog` 表的工作方式 ([#2563](https://github.com/ClickHouse/ClickHouse/issues/2563)). -- 如果 ZooKeeper 中节点的值为 NULL, 则 `system.zookeeper` 表有效. - -### ClickHouse 版本 1.1.54390, 2018-07-06 {#clickhouse-release-1-1-54390-2018-07-06} - -#### 新特征: {#new-features-11} - -- 查询可以以 `multipart/form-data` 格式发送(在 `query` 字段中), 如果外部数据也被发送用于查询处理 ([Olga Hvostikova](https://github.com/ClickHouse/ClickHouse/pull/2490)). -- 添加了在以 CSV 格式读取数据时启用或禁用处理单引号或双引号的功能. 您可以在 `format_csv_allow_single_quotes` 和 `format_csv_allow_double_quotes` 设置中进行配置 ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2574)). -- 现在可以使用 `OPTIMIZE TABLE ... FINAL` 而不指定 `MergeTree` 的非复制变体的分区 ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2599)). - -#### 改进: {#improvements-10} - -- 当可以使用表索引时,通过使用 IN 运算符提高性能、减少内存消耗和正确的内存消耗跟踪 ([#2584](https://github.com/ClickHouse/ClickHouse/pull/2584)). -- 添加数据部分时删除了校验和的冗余检查. 这在有大量副本时很重要, 因为在这些情况下, 检查总数等于 N^2. -- 为 `arrayEnumerateUniq` 函数添加了对 `Array(Tuple(...))` 参数的支持 ([#2573](https://github.com/ClickHouse/ClickHouse/pull/2573)). -- 为 `runningDifference` 函数添加了 `Nullable` 支持 ([#2594](https://github.com/ClickHouse/ClickHouse/pull/2594)). -- 提高了表达式数量非常多时的查询分析性能 ([#2572](https://github.com/ClickHouse/ClickHouse/pull/2572)). -- 更快地选择要合并到 `ReplicatedMergeTree` 表中的数据部分. ZooKeeper 会话的更快恢复 ([#2597](https://github.com/ClickHouse/ClickHouse/pull/2597)). -- `MergeTree` 表的 `format_version.txt` 文件如果丢失则重新创建, 如果在复制没有文件的目录结构后启动 ClickHouse, 这是有意义的 ([Ciprian Hacman](https://github.com/ClickHouse/ClickHouse/pull/2593)). - -#### Bug 修复: {#bug-fixes-19} - -- 修复了使用 ZooKeeper 时无法在重新启动服务器之前恢复会话和表的只读状态的错误. -- 修复了使用 ZooKeeper 时可能导致会话中断时旧节点不会被删除的错误. -- 修复了 Float 参数的 `quantileTDigest` 函数中的错误 (这个错误是在 1.1.54388 版本中引入的) ([Mikhail Surin](https://github.com/ClickHouse/ClickHouse/pull/2553)). -- 如果主键列位于相同大小的有符号和无符号整数之间的类型转换函数内, 则修复了 MergeTree 表的索引中的错误 ([#2603](https://github.com/ClickHouse/ClickHouse/pull/2603)). -- 如果使用了宏但它们不在配置文件中, 则修复了段错误 ([#2570](https://github.com/ClickHouse/ClickHouse/pull/2570)). -- 修复重新连接客户端时切换到默认数据库的问题 ([#2583](https://github.com/ClickHouse/ClickHouse/pull/2583)). -- 修复了禁用 `use_index_for_in_with_subqueries` 设置时发生的错误. - -#### 安全修复: {#security-fix-1} - -- 连接到 MySQL 时不再可能发送文件 (`LOAD DATA LOCAL INFILE`). - -### ClickHouse 版本 1.1.54388, 2018-06-28 {#clickhouse-release-1-1-54388-2018-06-28} - -#### 新特征: {#new-features-12} - -- 支持对复制表的 `ALTER TABLE t DELETE WHERE` 查询. 添加了 `system.mutations` 表以跟踪此类查询的进度. -- 支持对 \*MergeTree 表的 `ALTER TABLE t [REPLACE|ATTACH] PARTITION` 查询. -- 支持`TRUNCATE TABLE`查询 ([Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2260)). -- 几个新的复制表的 `SYSTEM` 查询 (`RESTART REPLICAS`, `SYNC REPLICA`, `[STOP|START] [MERGES|FETCHES|SENDS REPLICATED|REPLICATION QUEUES]`). -- 增加了使用 MySQL 引擎写表的能力和对应的表函数 ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2294)). -- 添加了 `url()` 表函数和 `URL` 表引擎 ([Alexander Sapin](https://github.com/ClickHouse/ClickHouse/pull/2501)). -- 添加了 `windowFunnel` 聚合函数 ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2352)). -- 字符串的新 `startsWith` 和 `endsWith` 函数 ([Vadim Plakhtinsky](https://github.com/ClickHouse/ClickHouse/pull/2429)). -- `numbers()` 表函数现在允许您指定偏移量 ([Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2535)). -- `clickhouse-client` 的密码可以交互输入. -- 服务器日志现在可以发送到系统日志 ([Alexander Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/2459)). -- 支持使用共享库源登录词典 ([Alexander Sapin](https://github.com/ClickHouse/ClickHouse/pull/2472)). -- 支持自定义 CSV 分隔符 ([Ivan Zhukov](https://github.com/ClickHouse/ClickHouse/pull/2263)). -- 添加了 `date_time_input_format` 设置. 如果将此设置切换为 `best_effort`, 则会以多种格式读取 DateTime 值. -- 添加了用于数据混淆的 `clickhouse-obfuscator` 实用程序. 使用示例: 发布性能测试中使用的数据. - -#### 实验功能: {#experimental-features-2} - -- 添加了仅在需要的地方计算 `and` 参数的功能 ([Anastasia Tsarkova](https://github.com/ClickHouse/ClickHouse/pull/2272)). -- 一些表达式现在可以 JIT 编译为本机代码 ([pyos](https://github.com/ClickHouse/ClickHouse/pull/2277)). - -#### Bug 修复: {#bug-fixes-20} - -- 使用 `DISTINCT` 和 `ORDER BY` 查询不再出现重复. -- 使用`ARRAY JOIN` 和 `arrayFilter` 的查询不再返回不正确的结果. -- 修复了从嵌套结构中读取数组列时的错误 ([#2066](https://github.com/ClickHouse/ClickHouse/issues/2066)). -- 修复了使用 HAVING 子句 (如 `HAVING tuple IN (...)`) 分析查询时的错误. -- 修复了使用递归别名分析查询时的错误. -- 修复了从 ReplacingMergeTree 读取时的错误,在 PREWHERE 中使用过滤所有行的条件 ([#2525](https://github.com/ClickHouse/ClickHouse/issues/2525)). -- 在 HTTP 界面中使用会话时未应用用户配置文件设置. -- 修复了如何从 clickhouse-local 中的命令行参数应用设置. -- ZooKeeper 客户端库现在使用从服务器收到的会话超时. -- 修复了 ZooKeeper 客户端库中客户端等待服务器响应时间超过超时时间的错误. -- 修复了分区键列条件查询的部分修剪 ([#2342](https://github.com/ClickHouse/ClickHouse/issues/2342)). -- 现在可以在 `CLEAR COLUMN IN PARTITION` 之后进行合并 ([#2315](https://github.com/ClickHouse/ClickHouse/issues/2315)). -- 修复了 ODBC 表函数中的类型映射 ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2268)). -- 已修复带和不带时区的 `DateTime` 的类型比较 ([Alexander Bocharov](https://github.com/ClickHouse/ClickHouse/pull/2400)). -- 修复了 `CAST` 运算符的语法解析和格式设置. -- 固定插入到分布式表引擎的物化视图中 ([Babacar Diassé](https://github.com/ClickHouse/ClickHouse/pull/2411)). -- 修复了将数据从 `Kafka` 引擎写入物化视图时的竞争条件 ([Yangkuan Liu](https://github.com/ClickHouse/ClickHouse/pull/2448)). -- 修复了 remote() 表函数中的 SSRF. -- 修复了多行模式下 `clickhouse-client` 的退出行为 ([#2510](https://github.com/ClickHouse/ClickHouse/issues/2510)). - -#### 改进: {#improvements-11} - -- 复制表中的后台任务现在在线程池中执行,而不是在单独的线程中执行 ([Silviu Caragea](https://github.com/ClickHouse/ClickHouse/pull/1722)). -- 提高LZ4压缩性能. -- 对具有大量 JOIN 和子查询的查询进行更快的分析. -- 当网络错误过多时,DNS 缓存现在会自动更新. -- 如果由于其中一个物化视图有太多的部件而无法插入表,则不会再插入表. -- 更正了事件计数器 `Query`、`SelectQuery` 和 `InsertQuery` 中的差异. -- 如果元组类型匹配,则允许使用诸如 `tuple IN (SELECT tuple)` 之类的表达式. -- 即使您尚未配置 ZooKeeper,具有复制表的服务器也可以启动. -- 在计算可用 CPU 内核数时,现在考虑了 cgroup 的限制 ([Atri Sharma](https://github.com/ClickHouse/ClickHouse/pull/2325)). -- 在 systemd 配置文件中为配置目录添加了 chown ([Mikhail Shiryaev](https://github.com/ClickHouse/ClickHouse/pull/2421)). - -#### Build 更新: {#build-changes-4} - -- gcc8 编译器可用于构建. -- 添加了从子模块构建 llvm 的功能. -- ibrdkafka 库版本已更新至 v0.11.4. -- 添加了使用系统 libcpuid 库的功能. 库版本已更新至 0.4.0. -- 使用矢量类库修复了构建 ([Babacar Diassé](https://github.com/ClickHouse/ClickHouse/pull/2274)). -- Cmake 现在默认为 ninja 生成文件 (就像使用 `-G Ninja` 时一样). -- 添加了使用 libtinfo 库代替 libtermcap 的功能 ([Georgy Kondratiev](https://github.com/ClickHouse/ClickHouse/pull/2519)). -- 修复了 Fedora Rawhide 中的头文件冲突 ([#2520](https://github.com/ClickHouse/ClickHouse/issues/2520)). - -#### 向后不兼容的更新: {#backward-incompatible-changes-7} - -- 删除了 `Vertical` 和 `Pretty*` 格式的转义并删除了 `VerticalRaw` 格式. -- 如果在分布式查询中同时使用版本 1.1.54388(或更高版本)和旧版本的服务器,并且查询具有不带 `AS` 关键字的 `cast(x, 'Type')` 表达式并且没有大写的 `cast` 一词, 将抛出异常, 并显示类似 `Not found column cast(0, 'UInt8') in block` 的消息. 解决方案: 更新整个集群上的服务器. - -### ClickHouse 版本 1.1.54385, 2018-06-01 {#clickhouse-release-1-1-54385-2018-06-01} - -#### Bug 修复: {#bug-fixes-21} - -- 修复了某些情况下导致 ZooKeeper 操作阻塞的错误. - -### ClickHouse 版本 1.1.54383, 2018-05-22 {#clickhouse-release-1-1-54383-2018-05-22} - -#### Bug 修复: {#bug-fixes-22} - -- 修复了一个表有很多副本时复制队列变慢的问题. - -### ClickHouse Rel版本ease 1.1.54381, 2018-05-14 {#clickhouse-release-1-1-54381-2018-05-14} - -#### Bug 修复: {#bug-fixes-23} - -- 修复了当 ClickHouse 与 ZooKeeper 服务器失去连接时 ZooKeeper 中的节点泄漏. - -### ClickHouse 版本 1.1.54380, 2018-04-21 {#clickhouse-release-1-1-54380-2018-04-21} - -#### 新特征: {#new-features-13} - -- 添加了表函数 `file(path, format, structure)`. 从 `/dev/urandom` 读取字节的示例: ``` ln -s /dev/urandom /var/lib/clickhouse/user_files/random``clickhouse-client -q "SELECT * FROM file('random', ' RowBinary', 'd UInt8') LIMIT 10" ```. - -#### 改进: {#improvements-12} - -- 子查询可以包含在 `()` 括号中以提高查询的可读性. 例如: `(SELECT 1) UNION ALL (SELECT 1)`. -- 来自 `system.processes` 表的简单 `SELECT` 查询不包括在 `max_concurrent_queries` 限制中. - -#### Bug 修复: {#bug-fixes-24} - -- 修复了从 `物质化视图` 中选择时 `IN` 运算符的错误行为. -- 修复了在 `partition_key_column IN (...)` 等表达式中按分区索引不正确的过滤. -- 修复了如果在表上执行了 `REANAME`,则无法在非领导副本上执行 `OPTIMIZE` 查询. -- 修复了在非领导副本上执行 `OPTIMIZE` 或 `ALTER` 查询时的授权错误. -- 修复了 `KILL QUERY` 的冻结问题. -- 修复了 ZooKeeper 客户端库中的一个错误,如果在 ZooKeeper 配置中使用非空的 `chroot` 前缀,该错误会导致监视丢失、分布式 DDL 队列冻结以及复制队列变慢. - -#### 向后不兼容的更新: {#backward-incompatible-changes-8} - -- 删除了对像 `(a, b) IN (SELECT (a, b))` 这样的表达式的支持 (你可以使用等价的表达式 `(a, b) IN (SELECT a, b)`). 在以前的版本中, 这些表达式会导致未确定的 `WHERE` 过滤或导致错误. - -### ClickHouse 版本 1.1.54378, 2018-04-16 {#clickhouse-release-1-1-54378-2018-04-16} - -#### 新特征: {#new-features-14} - -- 无需重新启动服务器即可更改日志级别. -- 添加了 `SHOW CREATE DATABASE` 查询. -- `query_id` 可以传递给 `clickhouse-client` (elBroom). -- 新设置:`max_network_bandwidth_for_all_users`. -- 为 `MATERIALIZED VIEW` 添加了对 `ALTER TABLE ... PARTITION ...` 的支持. -- 在系统表中添加了有关未压缩形式的数据部分大小的信息. -- 分布式表的服务器到服务器加密支持( `` 中的副本配置中的 `1`). -- `ReplicatedMergeTree` 系列的表级别配置以最小化 Zookeeper 中存储的数据量: `use_minimalistic_checksums_in_zookeeper = 1` -- `clickhouse-client` 提示的配置. 默认情况下, 服务器名称现在会输出到提示中. 服务器的显示名称可以更改. 它还在 `X-ClickHouse-Display-Name` HTTP 标头中发送 (Kirill Shvakov). -- 可以为 `Kafka` 引擎指定多个逗号分隔的 `topics` (Tobias Adamson) -- 当查询被 `KILL QUERY` 或 `replace_running_query` 停止时,客户端会收到 `Query was cancelled` 异常而不是不完整的结果. - -#### 改进: {#improvements-13} - -- `ALTER TABLE ... DROP/DETACH PARTITION` 查询在复制队列的前面运行. -- `SELECT ... FINAL` 和 `OPTIMIZE ... FINAL` 可以在表只有一个数据部分时使用. -- 如果手动删除了 `query_log` 表,则会即时重新创建它 (Kirill Shvakov). -- `lengthUTF8` 函数运行速度更快 (zhang2014). -- 当分片数量非常多时,提高了在 `分布式` 表 (`insert_distributed_sync = 1`) 中同步插入的性能. -- 服务器接受来自客户端的 `send_timeout` 和 `receive_timeout` 设置,并在连接到客户端时应用它们 (它们以相反的顺序应用:服务器套接字的 `send_timeout` 设置为从客户端接收到的 `receive_timeout` 值, 反之亦然). -- 用于异步插入 `Distributed` 表的更强大的崩溃恢复. -- `countEqual` 函数的返回类型从 `UInt32` 更改为 `UInt64` (谢磊). - -#### Bug 修复: {#bug-fixes-25} - -- 修复了表达式左侧为 `Nullable` 时 `IN` 的错误. -- 当某些元组组件在表索引中时,现在使用带有 `IN` 的元组时会返回正确的结果. -- `max_execution_time` 限制现在可以在分布式查询中正常工作. -- 修复了计算 `system.columns` 表中复合列大小时的错误. -- 修复了创建临时表 `CREATE TEMPORARY TABLE IF NOT EXISTS` 时的错误. -- 修复了 `StorageKafka` 中的错误 (##2075) -- 固定服务器因某些聚合函数的无效参数而崩溃. -- 修复了阻止`DETACH DATABASE` 查询停止 `ReplicatedMergeTree` 表的后台任务的错误. -- 插入聚合物化视图时,不太可能发生 `Too many parts` 状态 (##2084). -- 如果替换必须跟在同一级别的另一个替换之后,则更正了配置中替换的递归处理. -- 在创建使用带有 `UNION ALL` 的查询的 `VIEW` 时更正了元数据文件中的语法. -- `SummingMergeTree` 现在可以正确地使用复合键对嵌套数据结构求和. -- 修复了在为 `ReplicatedMergeTree` 表选择领导者时出现竞争条件的可能性. - -#### Build 更新: {#build-changes-5} - -- 构建支持 `ninja` 而不是 `make`,并且默认使用 `ninja` 来构建版本. -- 重命名包:`clickhouse-common-static`中的`clickhouse-server-base`; `clickhouse-server` 中的 `clickhouse-server-common`; `clickhouse-common-static-dbg` 中的 `clickhouse-common-dbg`. 要安装, 请使用 `clickhouse-server clickhouse-client` . 具有旧名称的包仍将加载到存储库中以实现向后兼容性. - -#### 向后不兼容的更新: {#backward-incompatible-changes-9} - -- 如果在左侧指定了数组, 则删除了 IN 表达式的特殊解释. 以前, 表达式 `arr IN (set)` 被解释为"at least one `arr` 元素属于 `set`". 要在新版本中获得相同的行为, 请编写 `arrayExists(x -> x IN (set), arr)`. -- 禁用套接字选项`SO_REUSEPORT` 的错误使用,该选项默认在 Poco 库中被错误启用. 请注意, 在 Linux 上,不再有任何理由同时指定地址 `::` 和 `0.0.0.0` 用于侦听 – 仅使用 `::`, 它允许侦听通过 IPv4 和 IPv6 的连接(默认情况下) 内核配置设置). 您还可以通过在配置中指定 `1` 来恢复以前版本的行为. - -### ClickHouse 版本 1.1.54370, 2018-03-16 {#clickhouse-release-1-1-54370-2018-03-16} - -#### 新特征: {#new-features-15} - -- 添加了 `system.macros` 表和更改配置文件时宏的自动更新. -- 添加了 `SYSTEM RELOAD CONFIG` 查询. -- 新增`maxIntersections(left_col, right_col)`聚合函数, 返回最大同时相交区间数`[left; 对]`. `maxIntersectionsPosition(left, right)` 函数返回 "maximum" 间隔的开始. ([Michael Furmur](https://github.com/ClickHouse/ClickHouse/pull/2012)). - -#### 改进: {#improvements-14} - -- 在 `Replicated` 表中插入数据时, 对 `ZooKeeper` 的请求较少(并且大多数用户级错误已从 `ZooKeeper` 日志中消失). -- 添加了为数据集创建别名的功能. 示例: `WITH (1, 2, 3) AS set SELECT number IN set FROM system.numbers LIMIT 10`. - -#### Bug 修复: {#bug-fixes-26} - -- 修复了从 `Distributed` 表的合并表读取时的 `Illegal PREWHERE` 错误. -- 添加了允许您在仅支持 IPv4 的 Docker 容器中启动 clickhouse-server 的修复程序. -- 修复了从系统 `system.parts_columns` 表读取时的竞争条件. -- 在同步插入到 `Distributed` 表期间删除了双缓冲,这可能会导致连接超时. -- 修复了在开始 `SELECT` 查询之前导致等待不可用副本过长的错误. -- 修复了 `system.parts` 表中不正确的日期. -- 修复了如果在 `ZooKeeper` 集群的配置中 `chroot` 不为空, 则无法在 `Replicated` 表中插入数据的错误. -- 修复了空 `ORDER BY` 表的垂直合并算法. -- 恢复了在对远程表的查询中使用字典的能力, 即使这些字典不存在于请求者服务器上. 此功能在 1.1.54362 版中丢失. -- 当 `IN` 的右侧应该使用远程 `default.table` 时, 恢复了诸如 `SELECT * FROM remote('server2', default.table) WHERE col IN (SELECT col2 FROM default.table)` 之类的查询行为而不是本地的. 此行为在 1.1.54358 版本中被破坏. -- 删除了 `Not found column ... in block` 的无关错误级日志记录. - -### ClickHouse 版本 1.1.54362, 2018-03-11 {#clickhouse-release-1-1-54362-2018-03-11} - -#### 新特征: {#new-features-16} - -- 带 `GROUP BY` 的空集 (例如 `SELECT count(*) FROM table WHERE 0`) 的聚合现在返回一个结果, 其中一行带有聚合函数的空值, 符合 SQL 标准. 要恢复旧行为 (返回空结果), 请将 `empty_result_for_aggregation_by_empty_set` 设置为 1. -- 为 `UNION ALL` 添加了类型转换. `UNION ALL` 中的 `SELECT` 位置允许不同的别名, 符合SQL标准. -- `LIMIT BY` 子句支持任意表达式. 以前, 只能使用由`SELECT`产生的列. -- 当将 `IN` 应用于主键列中的表达式元组时, 将使用 `MergeTree` 表的索引. 示例: `WHERE (UserID, EventDate) IN ((123, '2000-01-01'), ...)` (Anastasiya Tsarkova). -- 添加了用于在集群之间复制和重新分片数据的 `clickhouse-copier` 工具(测试版). -- 加了一致性哈希函数: `yandexConsistentHash`、`jumpConsistentHash`、`sumburConsistentHash`. 它们可以用作分片键, 以减少后续重新分片期间的网络流量. -- 添加函数: `arrayAny`, `arrayAll`, `hasAny`, `hasAll`, `arrayIntersect`, `arrayResize`. -- 添加了 `arrayCumSum` 函数 (Javi Santana). -- 添加了 `parseDateTimeBestEffort`、`parseDateTimeBestEffortOrZero` 和 `parseDateTimeBestEffortOrNull` 函数以从包含各种可能格式的文本的字符串中读取 DateTime. -- 更新期间可以从外部字典中部分重新加载数据 (仅加载指定字段的值大于先前下载的记录) (Arsen Hakobyan). -- 添加了 `cluster` 表功能. 示例: `cluster(cluster_name, db, table)`。 `remote` 表函数可以接受集群名称作为第一个参数, 如果它被指定为标识符. -- `remote` 和 `cluster` 表函数可用于 `INSERT` 查询. -- 将 `create_table_query` 和 `engine_full` 虚拟列添加到 `system.tables`表. `metadata_modification_time` 列是虚拟的. -- 将 `data_path` 和 `metadata_path` 列添加到 `system.tables` 和 `system.databases` 表, 并将 `path` 列添加到 `system.parts` 和 `system.parts_columns` 表. -- 在 `system.part_log` 表中添加了有关合并的附加信息. -- `system.query_log` 表可以使用任意分区键 (Kirill Shvakov). -- `SHOW TABLES` 查询现在也显示临时表. 添加临时表和 `is_temporary` 列到 `system.tables` (zhang2014). -- 添加了`DROP TEMPORARY TABLE`和`EXISTS TEMPORARY TABLE`查询 (zhang2014). -- 支持临时表的 `SHOW CREATE TABLE` (zhang2014). -- 为内部进程使用的设置添加了`system_profile`配置参数. -- 支持加载 `object_id` 作为 `MongoDB` 字典中的属性 (Pavel Litvinenko). -- 使用 `MongoDB` 源加载外部字典的数据时, 将 `null` 作为默认值读取 (Pavel Litvinenko). -- 从没有单引号的 Unix 时间戳中读取 `Values` 格式的 `DateTime` 值. -- 在某些副本缺少请求的表的情况下, `remote` 表函数支持故障转移. -- 当您运行 `clickhouse-server` 时,可以在命令行中覆盖配置设置. 示例: `clickhouse-server -- --logger.level=information`. -- 从 `FixedString` 参数实现了 `empty` 函数: 如果字符串完全由空字节组成, 该函数返回 1 (zhang2014). -- 添加了`listen_try`配置参数, 用于在不退出的情况下至少侦听一个侦听地址, 如果某些地址无法侦听 (对于禁用对 IPv4 或 IPv6 支持的系统很有用). -- 添加了 `VersionedCollapsingMergeTree` 表引擎. -- 支持`library` 字典源的行和任意数字类型. -- `MergeTree` 表可以在没有主键的情况下使用 (你需要指定 `ORDER BY tuple()`). -- 如果参数不是 `NULL` , 则 `Nullable` 类型可以 `CAST` 到非 `Nullable` 类型. -- 可以为 `VIEW` 执行 `RENAME TABLE` . -- 添加了 `throwIf` 函数. -- 添加了 `odbc_default_field_size` 选项, 它允许您扩展从 ODBC 源加载的值的最大大小 (默认为 1024). -- `system.processes` 表和 `SHOW PROCESSLIST` 现在有 `is_cancelled` 和 `peak_memory_usage` 列. - -#### 改进: {#improvements-15} - -- 结果的限制和配额不再应用于 `INSERT SELECT` 查询或 `SELECT` 子查询的中间数据. -- 在服务器启动时检查 `Replicated` 表的状态时, 更少的 `force_restore_data` 错误触发器. -- 添加了 `allow_distributed_ddl` 选项. -- `MergeTree` 表键的表达式中不允许使用不确定性函数. -- 从 `config.d` 目录替换的文件按字母顺序加载. -- 在将空数组作为元素之一的常量多维数组的情况下, 改进了 `arrayElement` 函数的性能. 示例: `[[1], []][x]`. -- 当使用具有非常大的替换 (例如, 非常大的 IP 网络列表) 的配置文件时, 服务器现在启动速度更快. -- 运行查询时, 表值函数运行一次. 以前, `remote` 和 `mysql` 表值函数执行相同的查询两次以从远程服务器检索表结构. -- 使用了 `MkDocs` 文档生成器. -- 当您尝试删除其他列的 `DEFAULT` / `MATERIALIZED` 表达式所依赖的表列时, 抛出异常 (zhang2014). -- 添加了将文本格式中的空行解析为 `Float` 数据类型的数字 0 的功能. 此功能以前可用, 但在版本 1.1.54342 中丢失. -- `Enum` 值可用于 `min`、`max`、`sum` 和其他一些函数. 在这些情况下, 它使用相应的数值. 此功能以前可用, 但在 1.1.54337 版本中丢失. -- 添加了 `max_expanded_ast_elements` 以限制递归扩展别名后 AST 的大小. - -#### Bug 修复: {#bug-fixes-27} - -- 修复了错误地从子查询中删除不必要的列, 或未从包含 `UNION ALL` 的子查询中删除的情况. -- 修复了 `ReplaceMergeTree` 表的合并错误. -- 修复了 `Distributed` 表中的同步插入 (`insert_distributed_sync = 1`). -- 修复了子查询中重复列的 `FULL` 和 `RIGHT JOIN` 的某些用途的段错误. -- 修复了`replace_running_query`和`KILL QUERY`的某些用途的段错误. -- 修复了 `system.dictionaries` 表中 `source` 和 `last_exception` 列的顺序. -- 修复了 `DROP DATABASE` 查询未删除带有元数据的文件时的错误. -- 修复了 `Dictionary` 数据库的 `DROP DATABASE` 查询. -- 修复了基数大于 1 亿项的 `uniqHLL12` 和 `uniqCombined` 函数的低精度问题 (Alex Bocharov). -- 修复了在需要同时计算 INSERT 查询中的默认显式表达式时隐式默认值的计算 (zhang2014). -- 修复了对 `MergeTree` 表的查询无法完成的罕见情况 (chenxing-xc). -- 修复了在所有分片都是本地的情况下对 `Distributed` 表运行 `CHECK` 查询时发生的崩溃 (chenxing.xc). -- 修复了使用正则表达式的函数的轻微性能回归. -- 修复了从复杂表达式创建多维数组时的性能回归. -- 修复了可能导致额外的 `FORMAT` 部分出现在带有元数据的 `.sql` 文件中的错误. -- 修复了一个错误,该错误导致在尝试删除查看明确指定的表的 `MATERIALIZED VIEW` 时应用 `max_table_size_to_drop` 限制. -- 修复了与旧客户端不兼容的问题 (旧客户端有时会使用 `DateTime('timezone')` 类型发送数据, 他们不理解). -- 修复了当这些列的条件移至 `PREWHERE` 时读取使用 `ALTER` 添加但对于旧分区为空的结构的 `Nested` 列元素时的错误. -- 修复了在查询到 `Merge` 表时通过虚拟 `_table` 列过滤表时的错误. -- 修复了在 `Distributed` 表中使用 `ALIAS` 列时的错误. -- 修复了一个错误, 该错误使动态编译无法用于带有来自 `quantile` 系列的聚合函数的查询. -- 修复了查询执行管道中的争用条件, 这种情况在使用具有大量表的 `Merge` 表以及使用 `GLOBAL` 子查询时在非常罕见的情况下发生. -- 修复了在使用来自多个参数的聚合函数时将不同大小的数组传递给 `arrayReduce` 函数时发生的崩溃. -- 禁止在 `MATERIALIZED VIEW` 中使用带有 `UNION ALL` 的查询. -- 修复了服务器启动时 `part_log` 系统表初始化过程中的错误 (默认情况下, `part_log`是禁用的). - -#### 向后不兼容的更新: {#backward-incompatible-changes-10} - -- 删除了 `distributed_ddl_allow_replicated_alter` 选项. 默认情况下启用此行为. -- 删除了 `strict_insert_defaults` 设置. 如果您正在使用此功能, 请写信至 `clickhouse-feedback@yandex-team.com`. -- 移除了 `UnsortedMergeTree` 引擎. - -### ClickHouse 版本 1.1.54343, 2018-02-05 {#clickhouse-release-1-1-54343-2018-02-05} - -- 添加了在分布式 DDL 查询和分布式表构造函数中定义集群名称的宏支持: `CREATE TABLE distr ON CLUSTER '{cluster}' (...) ENGINE = Distributed('{cluster}', 'db', 'table')`. -- 现在像 `SELECT ... FROM table WHERE expr IN (subquery)` 这样的查询是使用 `table` 索引处理的. -- 在插入到复制表时改进了重复处理, 因此它们不再减慢复制队列的执行速度. - -### ClickHouse 版本 1.1.54342, 2018-01-22 {#clickhouse-release-1-1-54342-2018-01-22} - -此版本包含先前版本 1.1.54337 的错误修复: - -- 修复了 1.1.54337 中的回归: 如果默认用户具有只读访问权限, 则服务器拒绝启动并显示消息 `Cannot create database in readonly mode`. -- 修复了 1.1.54337 中的一个回归: 在具有 systemd 的系统上, 无论配置如何, 日志始终写入 syslog; 监视器脚本仍然使用 init.d. -- 修复了 1.1.54337 中的回归: Docker 镜像中的默认配置错误. -- 修复了 Graphite MergeTree 的非确定性行为 (您可以在日志消息中看到它 `合并后的数据与另一个副本上的数据不是字节相同` ). -- 修复了对复制表进行 OPTIMIZE 查询后可能导致合并不一致的错误 (您可能会在日志消息中看到它 `Part ... intersects the previous part` ). -- 当目标表中存在 MATERIALIZED 列时, 缓冲区表现在可以正常工作 (by zhang2014). -- 修复了 NULL 实现中的一个错误. - -### ClickHouse 版本 1.1.54337, 2018-01-18 {#clickhouse-release-1-1-54337-2018-01-18} - -#### 新特征: {#new-features-17} - -- 添加了对表中多维数组和元组 (`Tuple` 数据类型) 的存储的支持. -- 支持用于 `DESCRIBE` 和 `INSERT` 查询的表函数. 在 `DESCRIBE` 中添加了对子查询的支持. 示例:`DESC TABLE remote('host', default.hits)`; `DESC 表(选择 1)`; `插入表功能远程('host',default.hits)`. 除了 `INSERT INTO` , 还支持 `INSERT INTO TABLE`. -- 改进了对时区的支持. `DateTime` 数据类型可以使用用于解析和格式化文本格式的时区进行注释. 示例: `DateTime('Asia/Istanbul')`. 当在函数中为 DateTime 参数指定时区时, 返回类型将跟踪时区, 并且值将按预期显示. -- 添加了函数`toTimeZone`、`timeDiff`、`toQuarter`、`toRelativeQuarterNum`. `toRelativeHour`/`Minute`/`Second` 函数可以将 `Date` 类型的值作为参数. `now` 函数名区分大小写. -- 添加了 `toStartOfFifteenMinutes` 函数 (Kirill Shvakov). -- 添加了用于格式化查询的 `clickhouse format` 工具. -- 添加了 `format_schema_path` 配置参数 (Marek Vavruşa). 它用于以 `Cap'n Proto` 格式指定模式. 架构文件只能位于指定目录中. -- 添加了对配置替换 (`incl` 和 `conf.d`) 的支持, 用于配置外部词典和模型 (Pavel Yakunin). -- 添加了一个包含 `system.settings` 表文档的列 (Kirill Shvakov). -- 添加了 `system.parts_columns` 表, 其中包含有关 `MergeTree` 表的每个数据部分中的列大小的信息. -- 添加了 `system.models` 表, 其中包含有关已加载的 `CatBoost` 机器学习模型的信息. -- 增加了 `mysql` 和 `odbc` 表功能以及相应的 `MySQL` 和 `ODBC` 表引擎, 用于访问远程数据库. 此功能处于测试阶段. -- 添加了为 `groupArray` 聚合函数传递 `AggregateFunction` 类型参数的可能性 (这样你就可以创建一些聚合函数的状态数组). -- 删除了对聚合函数组合器的各种组合的限制. 例如, 您可以使用 `avgForEachIf` 和 `avgIfForEach` 聚合函数, 它们具有不同的行为. -- `-ForEach` 聚合函数组合器针对多参数聚合函数的情况进行了扩展. -- 添加了对 `Nullable` 参数的聚合函数的支持, 即使在函数返回非 `Nullable` 结果的情况下 (由 Silviu Caragea 添加). 示例: `groupArray`、`groupUniqArray`、`topK`. -- 为 `clickhouse-client` 添加了 `max_client_network_bandwidth` (Kirill Shvakov). -- 允许具有 `readonly = 2` 设置的用户使用 TEMPORARY 表 (CREATE, DROP, INSERT…) (Kirill Shvakov). -- 添加了对 `Kafka` 引擎使用多个消费者的支持. `Kafka` 的扩展配置选项 (Marek Vavruša). -- 添加了 `intExp3` 和 `intExp4` 函数. -- 添加了 `sumKahan` 聚合函数. -- 添加到 \* Number\* OrNull 函数, 其中 \* Number\* 是数字类型. -- 添加了对 `INSERT SELECT` 查询的 `WITH` 子句的支持 (author: zhang2014). -- 添加设置: `http_connection_timeout`、`http_send_timeout`、`http_receive_timeout`. 特别是, 这些设置用于下载数据部分以进行复制. 如果网络过载, 更改这些设置可以实现更快的故障转移. -- 为类型为 `Null` 的表添加了对 `ALTER` 的支持 (Anastasiya Tsarkova). -- `reinterpretAsString` 函数针对所有连续存储在内存中的数据类型进行了扩展. -- 为 `clickhouse-local` 工具添加了 `--silent` 选项. 它禁止在 stderr 中打印查询执行信息. -- 添加了对从文本中读取 `Date` 类型值的支持, 该格式使用一位数而不是两位数指定月份和/或月份中的某天 (Amos Bird). - -#### 性能优化: {#performance-optimizations} - -- 改进了来自字符串参数的聚合函数 `min`、`max`、`any`、`anyLast`、`anyHeavy`、`argMin`、`argMax` 的性能. -- 改进了函数 `isInfinite`、`isFinite`、`isNaN`、`roundToExp2` 的性能. -- 改进了以文本格式解析和格式化 `Date` 和 `DateTime` 类型值的性能. -- 提高解析浮点数的性能和精度. -- 在左右部分具有不包含在 `USING` 中的名称相同的列的情况下, 降低了 `JOIN` 的内存使用量. -- 通过降低计算稳定性, 改进了聚合函数 `varSamp`、`varPop`、`stddevSamp`、`stddevPop`、`covarSamp`、`covarPop`、`corr` 的性能. 旧函数可在名称 `varSampStable`、`varPopStable`、`stddevSampStable`、`stddevPopStable`、`covarSampStable`、`covarPopStable`、`corrStable` 下使用. - -#### Bug 修复: {#bug-fixes-28} - -- 在运行 `DROP` 或 `DETACH PARTITION` 查询后修复了重复数据删除. 在以前的版本中, 删除分区并再次插入相同的数据不起作用, 因为插入的块被认为是重复的. -- 修复了一个错误, 该错误可能导致对带有 `POPULATE` 的 `CREATE MATERIALIZED VIEW` 查询的 `WHERE` 子句的错误解释. -- 修复了在 `zookeeper_servers` 配置中使用 `root_path` 参数的错误. -- 修复了将 `Date` 参数传递给 `toStartOfDay` 的意外结果. -- 修复了 `addMonths` 和 `subtractMonths` 函数以及 `INTERVAL n MONTH` 在结果为上一年的情况下的算术. -- 为 `DISTINCT`、`JOIN` 和 `uniq` 聚合函数和外部词典 (Evgeniy Ivanov) 添加了对 `UUID` 数据类型的缺失支持. 对 `UUID` 的支持仍然不完整. -- 在行总和为零的情况下修复了 `SummingMergeTree` 行为. -- 对`Kafka` 引擎的各种修复 (Marek Vavruša). -- 修复了 `Join` 表引擎的错误行为 (Amos Bird). -- 修复了 FreeBSD 和 OS X 下不正确的分配器行为. -- `extractAll` 函数现在支持空匹配. -- 修复了阻止使用 `libressl` 而不是 `openssl` 的错误. -- 修复了来自临时表的 `CREATE TABLE AS SELECT` 查询. -- 修复了更新复制队列的非原子性. 这可能会导致副本不同步, 直到服务器重新启动. -- 修复了 `gcd`、`lcm` 和 `modulo` (`%` 运算符) 中可能出现的溢出 (Maks Skorokhod). -- `-preprocessed` 文件现在在更改 `umask` 后创建 (`umask` 可以在配置中更改). -- 修复了使用自定义分区键时部件背景检查 (`MergeTreePartChecker`) 的错误. -- 修复了文本格式元组 ( `Tuple` 数据类型的值) 的解析. -- 改进了有关传递给 `multiIf`、`array` 和一些其他函数的不兼容类型的错误消息. -- 重新设计了对 `Nullable` 类型的支持. 修复了可能导致服务器崩溃的错误. 修复了几乎所有与 `NULL` 支持相关的其他错误: INSERT SELECT 中的类型转换不正确、HAVING 和 PREWHERE 中对 Nullable 的支持不足、`join_use_nulls` 模式、可空类型作为 `OR` 运算符的参数等. -- 修复了与数据类型的内部语义相关的各种错误. 示例: 在 `SummingMergeTree` 中对 `Enum` 类型字段进行不必要的求和; `Pretty` 格式的 `Enum` 类型的对齐等. -- 更严格地检查允许的复合列组合. -- 修复了为 `FixedString` 数据类型指定一个非常大的参数时的溢出问题. -- 修复了一般情况下 `topK` 聚合函数中的错误. -- 使用 `-Array` 组合器在聚合函数的n元变体的参数中添加了对数组大小相等性的缺失检查. -- 修复了 `clickhouse-client` 的 `--pager` 中的错误 (author: ks1322). -- 修复了 `exp10` 函数的精度. -- 修复了`visitParamExtract` 函数的行为, 以便更好地遵守文档. -- 修复了指定错误数据类型时崩溃的问题. -- 修复了所有列都是常量的情况下 `DISTINCT` 的行为. -- 修复了在使用带有复杂常量表达式的 `tupleElement` 函数作为元组元素索引的情况下的查询格式. -- 修复了 `range_hashed` 字典的 `Dictionary` 表中的错误. -- 修复了导致 `FULL` 和 `RIGHT JOIN` 结果中行数过多的错误 (Amos Bird). -- 修复了在配置重新加载期间在 `config.d` 目录中创建和删除临时文件时服务器崩溃的问题. -- 修复了 `SYSTEM DROP DNS CACHE` 查询: 缓存已刷新但集群节点的地址未更新. -- 修复了对视图下的表执行 `DETACH TABLE` 后 `MATERIALIZED VIEW` 的行为 (Marek Vavruša). - -#### Build 改进: {#build-improvements-4} - -- `pbuilder` 工具用于构建. 构建过程几乎完全独立于构建宿主环境. -- 单个构建用于不同的操作系统版本. 软件包和二进制文件已与各种 Linux 系统兼容. -- 添加了 `clickhouse-test` 包. 它可用于运行功能测试. -- 源 tarball 现在可以发布到存储库. 它可用于在不使用 GitHub 的情况下重现构建. -- 添加了与 Travis CI 的有限集成. 由于 Travis 中构建时间的限制, 仅测试调试构建并运行有限的测试子集. -- 在默认版本中添加了对 `Cap'n'Proto` 的支持. -- 将文档来源的格式从 `Restricted Text` 更改为 `Markdown`. -- 添加了对 `systemd` (Vladimir Smirnov) 的支持. 由于与某些操作系统映像不兼容, 它默认被禁用, 可以手动启用. -- 对于动态代码生成, `clang` 和 `lld` 被嵌入到 `clickhouse` 二进制文件中. 它们也可以作为 `clickhouse clang` 和 `clickhouse lld` 调用. -- 从代码中删除了 GNU 扩展的使用. 启用了`-Wextra` 选项。 当使用 `clang` 构建时, 默认是 `libc++` 而不是 `libstdc++`. -- 提取了 `clickhouse_parsers` 和 `clickhouse_common_io` 库以加速各种工具的构建. - -#### 向后不兼容的更新: {#backward-incompatible-changes-11} - -- 包含 `Nullable` 列的 `Log` 类型表中标记的格式以向后不兼容的方式进行了更改. 如果你有这些表, 你应该在启动新的服务器版本之前将它们转换为 `TinyLog` 类型. 为此, 请在 `metadata` 目录中相应的 `.sql` 文件中将 `ENGINE = Log` 替换为 `ENGINE = TinyLog`. 如果你的表没有 `Nullable` 列或者你的表的类型不是 `Log`, 那么你不需要做任何事情. -- 删除了 `experimental_allow_extended_storage_definition_syntax` 设置. 现在默认启用此功能. -- `runningIncome` 函数被重命名为 `runningDifferenceStartingWithFirstvalue` 以避免混淆. -- 当在没有表的 FROM 之后直接指定 ARRAY JOIN 时, 删除了 `FROM ARRAY JOIN arr` 语法 (Amos Bird). -- 删除了仅用于演示目的的 `BlockTabSeparated` 格式. -- 更改了聚合函数 `varSamp`、`varPop`、`stddevSamp`、`stddevPop`、`covarSamp`、`covarPop`、`corr` 的状态格式. 如果您已将这些聚合函数的状态存储在表中 (使用 `AggregateFunction` 数据类型或具有相应状态的物化视图), 请写信至 clickhouse-feedback@yandex-team.com. -- 在以前的服务器版本中, 有一个未公开的功能: 如果聚合函数依赖于参数, 您仍然可以在 AggregateFunction 数据类型中不带参数地指定它. 示例: `AggregateFunction(quantiles, UInt64)` 而不是 `AggregateFunction(quantiles(0.5, 0.9), UInt64)` . 此功能已丢失. 虽然它没有记录, 但我们计划在未来的版本中再次支持它. -- 枚举数据类型不能用于最小/最大聚合函数. 此能力将在下一个版本中恢复. - -#### 升级时请注意: {#please-note-when-upgrading} - -- 在集群上进行滚动更新时, 当一些副本运行旧版本的 ClickHouse 和一些运行新版本时, 复制会暂时停止, 并且日志中出现消息 `unknown parameter 'shard'` . 更新集群的所有副本后, 复制将继续. -- 如果在集群服务器上运行不同版本的 ClickHouse, 使用以下函数的分布式查询可能会得到不正确的结果: `varSamp`、`varPop`、`stddevSamp`、`stddevPop`、`covarSamp`、`covarPop` , `corr`. 您应该更新所有集群节点. - -## [Changelog for 2017](../../whats-new/changelog/2017.md#clickhouse-release-1-1-54327-2017-12-21) {#changelog-for-2017} diff --git a/docs/zh/whats-new/changelog/2019.md b/docs/zh/whats-new/changelog/2019.md deleted file mode 100644 index 72c908c9e0e..00000000000 --- a/docs/zh/whats-new/changelog/2019.md +++ /dev/null @@ -1,1855 +0,0 @@ ---- -slug: /zh/whats-new/changelog/2019 ---- -## ClickHouse 版本 19.17 {#clickhouse-release-v19-17} - -### ClickHouse 版本 19.17.6.36, 2019-12-27 {#clickhouse-release-v19-17-6-36-2019-12-27} - -#### Bug 修复 {#bug-fix} - -- 修复了解压缩中潜在的缓冲区溢出. 恶意用户可以传递可能导致缓冲区读取的伪造压缩数据. 这个问题是由 Yandex 信息安全团队的 Eldar Zaitov 发现的. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了当服务器无法使用字符串数据类型的值 (需要 UTF-8 验证) 以 JSON 或 XML 格式发送或写入数据或使用 Brotli 算法或某些方式压缩结果数据时可能出现的服务器崩溃 ( `std::terminate` ) 其他罕见情况. [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了来自 clickhouse `VIEW` 源的词典, 现在阅读此类词典不会导致错误 `There is no query`. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了检查 users.xml 中指定的 host_regexp 是否允许客户端主机的问题. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241), [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) -- 分布式表的 `RENAME TABLE` 现在会在发送到分片之前重命名包含插入数据的文件夹. 这解决了连续重命名 `tableA->tableB`、`tableC->tableA` 的问题. [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) -- 由 DDL 查询创建的 `range_hashed` 外部字典现在允许任意数字类型的范围. [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) -- 修复了 `INSERT INTO table SELECT ... FROM mysql(...)` 表函数. [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) -- 复了插入不存在的文件时在 `INSERT INTO TABLE FUNCTION file()` 中的段错误. 现在在这种情况下将创建文件, 然后将处理插入. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复了聚合位图和标量位图相交时的位图和错误. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) -- 修复了在没有 `TABLE` 或 `DICTIONARY` 限定符的情况下使用 `EXISTS` 查询时的段错误,就像 `EXISTS`. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了函数 `rand` 和 `randConstant` 在参数可为空的情况下的返回类型. 现在函数总是返回 `UInt32` 而不是 `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了 `DROP DICTIONARY IF EXISTS db.dict`, 现在如果 `db` 不存在则不会抛出异常. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) -- 如果由于服务器崩溃而没有完全删除表, 服务器将尝试恢复并加载它. [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) -- 如果有两个以上的分片本地表,则修复了分布式表的简单计数查询. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) -- 修复了导致 DB::BlockStreamProfileInfo::calculateRowsBeforeLimit() 中数据竞争的错误. [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) -- 修复了在合并指定部分后立即执行的 `ALTER table MOVE part`, 这可能导致移动指定部分合并到的部分. 现在它正确地移动了指定的部分. [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 现在可以将字典的表达式指定为字符串. 这对于从非 ClickHouse 源中提取数据时计算属性很有用, 因为它允许对这些表达式使用非 ClickHouse 语法. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) -- 修复了 `clickhouse-copier` 中由于 ZXid 溢出而导致的非常罕见的竞争. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -- 修复了查询失败后 (例如由于"Too many simultaneous queries") 不会读取外部表信息的错误, 并且下一个请求会将此信息解释为下一个查询的开始, 从而导致 `Unknown packet from client`. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) -- 避免在"Unknown packet X from server"之后取消引用. [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) -- 恢复对所有 ICU 语言环境的支持, 添加为常量表达式应用排序规则的功能, 并将语言名称添加到 system.collations 表. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) -- 从 `StorageFile` 和 `StorageHDFS` 读取的流数量现在受到限制, 以避免超过内存限制. [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) -- 修复了没有键的 `*MergeTree` 表的 `CHECK TABLE` 查询. [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) -- 如果没有突变, 则从零件名称中删除了突变编号. 此删除改进了与旧版本的兼容性. [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) -- 修复部分附属部分由于 data_version 大于表突变版本而跳过突变的 bug. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) -- 允许在将零件移动到另一台设备后使用零件的冗余副本启动服务器. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复使用聚合函数列时可能出现的 `Sizes of columns does not match` 错误. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) -- 现在, 如果将 WITH TIES 与 LIMIT BY 一起使用, 则会抛出异常. 现在可以将 TOP 与 LIMIT BY 一起使用. [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- 修复字典重新加载, 如果它有 `invalidate_query`, 它会停止更新并且在之前的更新尝试中出现一些异常. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) - -### ClickHouse 版本 19.17.4.11, 2019-11-22 {#clickhouse-release-v19-17-4-11-2019-11-22} - -#### 向后不兼容的更新 {#backward-incompatible-change} - -- 使用列而不是 AST 来存储标量子查询结果以获得更好的性能. 设置 `enable_scalar_subquery_optimization` 是在 19.17 中添加的, 默认情况下启用. 在从以前的版本升级到 19.17.2 或 19.17.3 期间, 它会导致类似 [this](https://github.com/ClickHouse/ClickHouse/issues/7851) 的错误. 此设置在 19.17.4 中默认禁用, 以便从 19.16 和更旧版本升级而不会出错. [#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos Bird](https://github.com/amosbird)) - -#### 新特征 {#new-feature} - -- 添加使用 DDL 查询创建字典的功能. [#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin)) -- 使 `bloom_filter` 类型的索引支持 `LowCardinality` 和 `Nullable`. [#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 添加函数 `isValidJSON` 以检查传递的字符串是否是有效的 json. [#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir)) -- 实现 `arrayCompact` 函数. [#7328](https://github.com/ClickHouse/ClickHouse/pull/7328) ([Memo](https://github.com/Joeywzr)) -- 为十进制数创建函数 `hex` . 它的工作原理类似于 `hex(reinterpretAsString())`, 但不会删除最后的零字节. [#7355](https://github.com/ClickHouse/ClickHouse/pull/7355) ([Mikhail Korotov](https://github.com/millb)) -- 添加 `arrayFill` 和 `arrayReverseFill` 函数, 将元素替换为数组前/后的其他元素. [#7380](https://github.com/ClickHouse/ClickHouse/pull/7380) ([hcz](https://github.com/hczhcz)) -- 添加 `CRC32IEEE()` / `CRC64()` 支持. [#7480](https://github.com/ClickHouse/ClickHouse/pull/7480) ([Azat Khuzhin](https://github.com/azat)) -- 实现类似于 in 的 `char` 函数. [mysql](https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_char) [#7486](https://github.com/ClickHouse/ClickHouse/pull/7486) ([sundyli](https://github.com/sundy-li)) -- 添加 `bitmapTransform` 函数. 它将位图中的一组值转换为另一个值数组, 结果是一个新的位图. [#7598](https://github.com/ClickHouse/ClickHouse/pull/7598) ([Zhichang Yu](https://github.com/yuzhichang)) -- 实现了`javaHashUTF16LE()`函数. [#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([achimbab](https://github.com/achimbab)) -- 为分布式引擎添加 `_shard_num` 虚拟列. [#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat)) - -#### 实验功能 {#experimental-feature} - -- 支持 `MergeTree` 中的处理器 (新的查询执行管道). [#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### Bug 修复 {#bug-fix-1} - -- 修复 `Values` 中不正确的浮点解析. [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) -- 修复启用 trace_log 时可能发生的罕见死锁. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) -- 生成 Kafka 表时防止消息重复, 其中有任何 MV 从中选择. [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) -- 支持 `IN` 中的 `Array(LowCardinality(Nullable(String)))`. 解决 [#7364](https://github.com/ClickHouse/ClickHouse/issues/7364) [#7366](https://github.com/ClickHouse/ClickHouse/pull/7366) ([achimbab](https://github.com/achimbab)) -- 在 `SQL BIGINT` 中添加对 `SQL TINYINT` 的处理, 并在 ODBC Bridge 中修复对 `SQL_FLOAT` 数据源类型的处理. [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) -- 修复空十进制列上的聚合( `avg` 和分位数). [#7431](https://github.com/ClickHouse/ClickHouse/pull/7431) ([Andrey Konyaev](https://github.com/akonyaev90)) -- 将 `INSERT` 修复为 Distributed with `MATERIALIZED` 列. [#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) -- 如果分区的某些部分已经在目标磁盘或卷上, 则使 `MOVE PARTITION` 工作. [#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复了在多磁盘配置中 `ReplicatedMergeTree` 中的突变期间无法创建硬链接的错误. [#7558](https://github.com/ClickHouse/ClickHouse/pull/7558) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复了当整个部分保持不变并且在另一个磁盘上找到最佳空间时 MergeTree 上发生突变的错误. [#7602](https://github.com/ClickHouse/ClickHouse/pull/7602) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复了未从磁盘配置中读取 `keep_free_space_ratio` 的错误. [#7645](https://github.com/ClickHouse/ClickHouse/pull/7645) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复表中仅包含 `Tuple` 列或具有复杂路径的列的错误. 修复 [7541](https://github.com/ClickHouse/ClickHouse/issues/7541). [#7545](https://github.com/ClickHouse/ClickHouse/pull/7545) ([alesapin](https://github.com/alesapin)) -- 在 max_memory_usage 限制中不考虑 Buffer 引擎的内存. [#7552](https://github.com/ClickHouse/ClickHouse/pull/7552) ([Azat Khuzhin](https://github.com/azat)) -- 修复由 `tuple()` 排序的 `MergeTree` 表中的最终标记使用. 在极少数情况下, 它可能会在选择时导致 `Can't adjust last granule` 错误. [#7639](https://github.com/ClickHouse/ClickHouse/pull/7639) ([Anton Popov](https://github.com/CurtizJ)) -- 修复具有需要上下文的操作 (例如 json 的函数) 的谓词的突变中的错误, 这可能导致崩溃或奇怪的异常. [#7664](https://github.com/ClickHouse/ClickHouse/pull/7664) ([alesapin](https://github.com/alesapin)) -- 修复在 `data/` 和 `shadow/` 目录中转义的数据库和表名的不匹配. [#7575](https://github.com/ClickHouse/ClickHouse/pull/7575) ([Alexander Burmak](https://github.com/Alex-Burmak)) -- 支持 RIGHT\|FULL JOIN 中的重复键, 例如 `ON t.x = u.x AND t.x = u.y`. 在这种情况下修复崩溃. [#7586](https://github.com/ClickHouse/ClickHouse/pull/7586) ([Artem Zuikov](https://github.com/4ertus2)) -- 使用 RIGHT 或 FULL JOIN 连接表达式时, 修复 `Not found column in block` . [#7641](https://github.com/ClickHouse/ClickHouse/pull/7641) ([Artem Zuikov](https://github.com/4ertus2)) -- 再次尝试以 `PrettySpace` format 格式修复无限循环. [#7591](https://github.com/ClickHouse/ClickHouse/pull/7591) ([Olga Khvostikova](https://github.com/stavrolia)) -- 当所有参数都是相同大小的 `FixedString` 时, 修复 `concat` 函数中的错误. [#7635](https://github.com/ClickHouse/ClickHouse/pull/7635) ([alesapin](https://github.com/alesapin)) -- 修复了在定义 S3、URL 和 HDFS 存储时使用 1 个参数的异常. [#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复带有查询的视图的 InterpreterSelectQuery 的范围. [#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat)) - -#### 改进 {#improvement} - -- `Nullable` 列识别和 NULL 值由 ODBC 桥正确处理. [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) -- 以原子方式写入分布式发送的当前批次. [#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat)) -- 如果我们无法在查询中检测到列名的表, 则抛出异常. [#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2)) -- 将 `merge_max_block_size` 设置添加到 `MergeTreeSettings`. [#7412](https://github.com/ClickHouse/ClickHouse/pull/7412) ([Artem Zuikov](https://github.com/4ertus2)) -- 带有 `HAVING` 和不带 `GROUP BY` 的查询假定按常量分组. 所以, `SELECT 1 HAVING 1` 现在返回一个结果. [#7496](https://github.com/ClickHouse/ClickHouse/pull/7496) ([Amos Bird](https://github.com/amosbird)) -- 支持将 `(X,)` 解析为类似于python的元组. [#7501](https://github.com/ClickHouse/ClickHouse/pull/7501), [#7562](https://github.com/ClickHouse/ClickHouse/pull/7562) ([Amos Bird](https://github.com/amosbird)) -- 使 `range` 函数行为几乎像 Pythonic 一样. [#7518](https://github.com/ClickHouse/ClickHouse/pull/7518) ([sundyli](https://github.com/sundy-li)) -- 将 `constraints` 列添加到表 `system.settings`. [#7553](https://github.com/ClickHouse/ClickHouse/pull/7553) ([Vitaly Baranov](https://github.com/vitlibar)) -- 更好的 tcp 处理程序的 Null 格式,以便可以通过 clickhouse-client 使用 `select ignore() from table format Null` 进行性能测量. [#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos Bird](https://github.com/amosbird)) -- 像 `CREATE TABLE ... AS (SELECT (1, 2))` 这样的查询被正确解析. [#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([hcz](https://github.com/hczhcz)) - -#### 性能改进 {#performance-improvement} - -- 改进了对短字符串键的聚合性能. [#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alexander Kuzmenkov](https://github.com/akuzm), [Amos Bird](https://github.com/amosbird)) -- 在折叠常量谓词后运行另一遍语法/表达式分析以获得潜在的优化. [#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos Bird](https://github.com/amosbird)) -- 使用存储元信息来评估琐碎的 `SELECT count() FROM table;`. [#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos Bird](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov)) -- 矢量化处理 `arrayReduce` 类似于聚合器 `addBatch`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) -- `Kafka` 消费性能的小幅改进. [#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement} - -- 添加对 CPU 架构 AARCH64 的交叉编译支持. 重构打包脚本. [#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7)) -- 构建软件包时, 将 darwin-x86_64 和 linux-aarch64 工具链解压到已安装的 Docker 卷中. [#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7)) -- 更新二进制打包器的 Docker 镜像. [#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7)) -- 修复了 MacOS Catalina 上的编译错误. [#7585](https://github.com/ClickHouse/ClickHouse/pull/7585) ([Ernest Poletaev](https://github.com/ernestp)) -- 查询分析逻辑中的一些重构: 将复杂的类拆分成几个简单的类. [#7454](https://github.com/ClickHouse/ClickHouse/pull/7454) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复没有子模块的构建. [#7295](https://github.com/ClickHouse/ClickHouse/pull/7295) ([proller](https://github.com/proller)) -- CMake 文件中更好的 `add_globs`. [#7418](https://github.com/ClickHouse/ClickHouse/pull/7418) ([Amos Bird](https://github.com/amosbird)) -- 删除 `unwind` 目标中的硬编码路径. [#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok)) -- 允许使用没有 ssl 的 mysql 格式. [#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([proller](https://github.com/proller)) - -#### 其他 {#other} - -- 为 ClickHouse SQL 方言添加了 ANTLR4 语法. [#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## ClickHouse 版本 19.16 {#clickhouse-release-v19-16} - -#### ClickHouse 版本 19.16.14.65, 2020-03-25 {#clickhouse-release-v19-16-14-65-2020-03-25} - -- 修复了多个参数 (超过 10 个) 的三元逻辑 OP 的批量计算中的错误. [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) This bugfix was backported to version 19.16 by a special request from Altinity. - -#### ClickHouse 版本 19.16.14.65, 2020-03-05 {#clickhouse-release-v19-16-14-65-2020-03-05} - -- 修复分布式子查询与旧 CH 版本不兼容的问题. 修复 [#7851](https://github.com/ClickHouse/ClickHouse/issues/7851) [(tabplubix)](https://github.com/tavplubix) -- 执行 `CREATE` 查询时, 在存储引擎参数中折叠常量表达式. 用当前数据库替换空数据库名称. 修复 [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). 还修复了 `ClickHouseDictionarySource` 中本地地址的检查. [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) -- 现在 `*MergeTree` 表引擎族中的后台合并更准确地保留存储策略卷顺序. [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 在读取后缀之后但提交之前发生异常时, 在极少数情况下防止在 `Kafka` 中丢失数据. 修复 [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). 相关的: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) -- 修复在尝试使用/删除使用错误参数创建的 `Kafka` 表时导致服务器终止的错误. 修复 [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) -- 允许将 `MaterializedView` 与 `Kafka` 表上方的子查询一起使用. [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) - -#### 新特征 {#new-feature-1} - -- 添加 `deduplicate_blocks_in_dependent_materialized_views` 选项以控制幂等插入到具有物化视图的表中的行为. 应 Altinity 的特殊要求, 此新功能已添加到错误修复版本中. [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) - -### ClickHouse 版本 19.16.2.2, 2019-10-30 {#clickhouse-release-v19-16-2-2-2019-10-30} - -#### 向后不兼容的更新 {#backward-incompatible-change-1} - -- 为 count/countIf 添加缺失的 arity 验证. [#7095](https://github.com/ClickHouse/ClickHouse/issues/7095), [#7298](https://github.com/ClickHouse/ClickHouse/pull/7298) ([Vdimir](https://github.com/Vdimir)) -- 删除旧的 `asterisk_left_columns_only` 设置 (默认情况下禁用). [#7335](https://github.com/ClickHouse/ClickHouse/pull/7335) ([Artem Zuikov](https://github.com/4ertus2)) -- 模板数据格式的格式字符串现在在文件中指定. [#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) ([tavplubix](https://github.com/tavplubix)) - -#### 新特征 {#new-feature-2} - -- 引入 uniqCombined64() 计算大于 UINT_MAX 的基数. [#7213](https://github.com/ClickHouse/ClickHouse/pull/7213), [#7222](https://github.com/ClickHouse/ClickHouse/pull/7222) ([Azat Khuzhin](https://github.com/azat)) -- 支持数组列上的布隆过滤器索引. [#6984](https://github.com/ClickHouse/ClickHouse/pull/6984) ([achimbab](https://github.com/achimbab)) -- 添加一个函数 `getMacro(name)`, 它从服务器配置中返回带有相应 `` 值的字符串. [#7240](https://github.com/ClickHouse/ClickHouse/pull/7240) - ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 为基于 HTTP 源的字典设置两个配置选项: `credentials` 和 `http-headers`. [#7092](https://github.com/ClickHouse/ClickHouse/pull/7092) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 添加一个新的 ProfileEvent `Merge` 来计算启动的后台合并的数量. [#7093](https://github.com/ClickHouse/ClickHouse/pull/7093) ([Mikhail Korotov](https://github.com/millb)) -- 添加返回完全限定域名的 fullHostName 函数. [#7263](https://github.com/ClickHouse/ClickHouse/issues/7263) [#7291](https://github.com/ClickHouse/ClickHouse/pull/7291) ([sundyli](https://github.com/sundy-li)) -- 添加函数 `arraySplit` 和 `arrayReverseSplit`, 它们通过 `cut off` 条件拆分数组. 它们在时间序列处理中很有用. [#7294](https://github.com/ClickHouse/ClickHouse/pull/7294) ([hcz](https://github.com/hczhcz)) -- 添加返回 multiMatch 系列函数中所有匹配索引数组的新函数. [#7299](https://github.com/ClickHouse/ClickHouse/pull/7299) ([Danila Kutenin](https://github.com/danlark1)) -- 添加一个新的数据库引擎 `Lazy`, 该引擎针对存储大量小型 -Log 表进行了优化. [#7171](https://github.com/ClickHouse/ClickHouse/pull/7171) ([Nikita Vasilev](https://github.com/nikvas0)) -- 为位图列添加聚合函数 groupBitmapAnd、-Or、-Xor. [#7109](https://github.com/ClickHouse/ClickHouse/pull/7109) ([Zhichang Yu](https://github.com/yuzhichang)) -- 添加聚合函数组合子 -OrNull 和 -OrDefault,当没有聚合时返回空值或默认值. [#7331](https://github.com/ClickHouse/ClickHouse/pull/7331) ([hcz](https://github.com/hczhcz)) -- 引入支持自定义转义和分隔符规则的 CustomSeparated 数据格式. [#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) ([tavplubix](https://github.com/tavplubix)) -- 支持Redis作为外部字典的来源. [#4361](https://github.com/ClickHouse/ClickHouse/pull/4361) [#6962](https://github.com/ClickHouse/ClickHouse/pull/6962) ([comunodi](https://github.com/comunodi), [Anton Popov](https://github.com/CurtizJ)) - -#### Bug 修复 {#bug-fix-2} - -- 修复错误的查询结果, 如果它有 `WHERE IN (SELECT ...)` 部分并且使用了 `optimize_read_in_order`. [#7371](https://github.com/ClickHouse/ClickHouse/pull/7371) ([Anton - Popov](https://github.com/CurtizJ)) -- 禁用 MariaDB 身份验证插件, 该插件依赖于项目外的文件. [#7140](https://github.com/ClickHouse/ClickHouse/pull/7140) ([Yuriy Baranov](https://github.com/yurriy)) -- 修复异常 `Cannot convert column ... because it is constant but values of constants are different in source and result` , 这在函数 `now()`、`today()`、`yesterday()`、 `randConstant()` 时, 这种情况很少发生. [#7156](https://github.com/ClickHouse/ClickHouse/pull/7156) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了使用 HTTP 保持活动超时而不是 TCP 保持活动超时的问题. [#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily Nemkov](https://github.com/Enmk)) -- 修复了 groupBitmapOr 中的分段错误. (issue [#7109](https://github.com/ClickHouse/ClickHouse/issues/7109)). [#7289](https://github.com/ClickHouse/ClickHouse/pull/7289) ([Zhichang Yu](https://github.com/yuzhichang)) -- 对于物化视图,在写入所有数据后调用 Kafka 的提交. [#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) -- 修复了 `system.part_log` 表中错误的 `duration_ms` 值. 差了十倍. [#7172](https://github.com/ClickHouse/ClickHouse/pull/7172) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 解决 LIVE VIEW 表崩溃并重新启用所有 LIVE VIEW 测试的快速修复. [#7201](https://github.com/ClickHouse/ClickHouse/pull/7201) ([vzakaznikov](https://github.com/vzakaznikov)) -- 在 MergeTree 部分的最小/最大索引中正确序列化 NULL 值. [#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 当表创建为 `CREATE TABLE AS` 时, 不要将虚拟列放入 .sql 元数据. [#7183](https://github.com/ClickHouse/ClickHouse/pull/7183) ([Ivan](https://github.com/abyss7)) -- 修复 `ATTACH PART` 查询中的分段错误. [#7185](https://github.com/ClickHouse/ClickHouse/pull/7185) ([alesapin](https://github.com/alesapin)) -- 修复空 IN 子查询和空 INNER/RIGHT JOIN 优化给出的某些查询的错误结果. [#7284](https://github.com/ClickHouse/ClickHouse/pull/7284) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复 LIVE VIEW getHeader() 方法中的 AddressSanitizer 错误. [#7271](https://github.com/ClickHouse/ClickHouse/pull/7271) ([vzakaznikov](https://github.com/vzakaznikov)) - -#### 改进 {#improvement-1} - -- 在 queue_wait_max_ms 等待发生的情况下添加一条消息. [#7390](https://github.com/ClickHouse/ClickHouse/pull/7390) ([Azat Khuzhin](https://github.com/azat)) -- 在表级设置 `s3_min_upload_part_size` . [#7059](https://github.com/ClickHouse/ClickHouse/pull/7059) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 在 StorageFactory 中检查 TTL. [#7304](https://github.com/ClickHouse/ClickHouse/pull/7304) ([sundyli](https://github.com/sundy-li)) -- 在部分合并连接中挤压左侧块 (优化). [#7122](https://github.com/ClickHouse/ClickHouse/pull/7122) ([Artem Zuikov](https://github.com/4ertus2)) -- 不允许在复制表引擎的突变中使用非确定性函数, 因为这会导致副本之间的不一致. [#7247](https://github.com/ClickHouse/ClickHouse/pull/7247) ([Alexander Kazakov](https://github.com/Akazz)) -- 在将异常堆栈跟踪转换为字符串时禁用内存跟踪器. 可以防止服务器上的 `Memory limit exceeded` 类型的错误信息丢失, 导致客户端 `Attempt to read after eof` 异常. [#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 各种格式改进. 解决 [#6033](https://github.com/ClickHouse/ClickHouse/issues/6033), [#2633](https://github.com/ClickHouse/ClickHouse/issues/2633), [#6611](https://github.com/ClickHouse/ClickHouse/issues/6611), [#6742](https://github.com/ClickHouse/ClickHouse/issues/6742) [#7215](https://github.com/ClickHouse/ClickHouse/pull/7215) ([tavplubix](https://github.com/tavplubix)) -- ClickHouse 会忽略 IN 运算符右侧无法转换为左侧类型的值. 使其适用于复合类型——数组和元组. [#7283](https://github.com/ClickHouse/ClickHouse/pull/7283) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 支持 ASOF JOIN 缺失的不等式. 可以在 ON 语法中为 ASOF 列加入 less-or-equal 变体和严格的更大和更少变体. [#7282](https://github.com/ClickHouse/ClickHouse/pull/7282) ([Artem Zuikov](https://github.com/4ertus2)) -- 优化部分合并连接. [#7070](https://github.com/ClickHouse/ClickHouse/pull/7070) ([Artem Zuikov](https://github.com/4ertus2)) -- 不要在 uniqCombined 函数中使用超过 98K 的内存. [#7236](https://github.com/ClickHouse/ClickHouse/pull/7236), [#7270](https://github.com/ClickHouse/ClickHouse/pull/7270) ([Azat Khuzhin](https://github.com/azat)) -- 在 PartialMergeJoin 中刷新磁盘上右侧连接表的部分 (如果没有足够的内存). 需要时加载数据. [#7186](https://github.com/ClickHouse/ClickHouse/pull/7186) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 性能改进 {#performance-improvement-1} - -- 通过避免数据重复来加速使用 const 参数的 joinGet. [#7359](https://github.com/ClickHouse/ClickHouse/pull/7359) ([Amos Bird](https://github.com/amosbird)) -- 如果子查询为空, 则提前返回. [#7007](https://github.com/ClickHouse/ClickHouse/pull/7007) ([小路](https://github.com/nicelulu)) -- 优化 Values 中 SQL 表达式的解析. [#6781](https://github.com/ClickHouse/ClickHouse/pull/6781) ([tavplubix](https://github.com/tavplubix)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-1} - -- 禁用一些用于交叉编译到 Mac OS 的贡献. [#7101](https://github.com/ClickHouse/ClickHouse/pull/7101) ([Ivan](https://github.com/abyss7)) -- 为 clickhouse_common_io 添加缺少的 PocoXML 链接. [#7200](https://github.com/ClickHouse/ClickHouse/pull/7200) ([Azat Khuzhin](https://github.com/azat)) -- 在 clickhouse-test 中接受多个测试过滤器参数. [#7226](https://github.com/ClickHouse/ClickHouse/pull/7226) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 为 ARM 启用 musl 和 jemalloc. [#7300](https://github.com/ClickHouse/ClickHouse/pull/7300) ([Amos Bird](https://github.com/amosbird)) -- 将 `--client-option` 参数添加到 `clickhouse-test` 以将附加参数传递给客户端. [#7277](https://github.com/ClickHouse/ClickHouse/pull/7277) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 在 rpm 包升级时保留现有配置. [#7103](https://github.com/ClickHouse/ClickHouse/pull/7103) ([filimonov](https://github.com/filimonov)) -- 修复 PVS 检测到的错误. [#7153](https://github.com/ClickHouse/ClickHouse/pull/7153) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 Darwin 的构建. [#7149](https://github.com/ClickHouse/ClickHouse/pull/7149) ([Ivan](https://github.com/abyss7)) -- glibc 2.29 兼容性. [#7142](https://github.com/ClickHouse/ClickHouse/pull/7142) ([Amos Bird](https://github.com/amosbird)) -- 确保 dh_clean 不接触潜在的源文件. [#7205](https://github.com/ClickHouse/ClickHouse/pull/7205) ([Amos Bird](https://github.com/amosbird)) -- 从 altinity rpm 更新时尝试避免冲突 - 它在 clickhouse-server-common 中单独打包了配置文件. [#7073](https://github.com/ClickHouse/ClickHouse/pull/7073) ([filimonov](https://github.com/filimonov)) -- 优化一些头文件以加快重建速度. [#7212](https://github.com/ClickHouse/ClickHouse/pull/7212), [#7231](https://github.com/ClickHouse/ClickHouse/pull/7231) ([Alexander - Kuzmenkov](https://github.com/akuzm)) -- 添加日期和日期时间的性能测试. [#7332](https://github.com/ClickHouse/ClickHouse/pull/7332) ([Vasily Nemkov](https://github.com/Enmk)) -- 修复一些包含非确定性突变的测试. [#7132](https://github.com/ClickHouse/ClickHouse/pull/7132) ([Alexander Kazakov](https://github.com/Akazz)) -- 将带有 MemorySanitizer 的构建添加到 CI. [#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 避免在 MetricsTransmitter 中使用未初始化的值. [#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) -- 修复 MemorySanitizer 发现的 Fields 中的一些问题. [#7135](https://github.com/ClickHouse/ClickHouse/pull/7135), [#7179](https://github.com/ClickHouse/ClickHouse/pull/7179) ([Alexander Kuzmenkov](https://github.com/akuzm)), [#7376](https://github.com/ClickHouse/ClickHouse/pull/7376) ([Amos Bird](https://github.com/amosbird)) -- 修复 murmurhash32 中未定义的行为. [#7388](https://github.com/ClickHouse/ClickHouse/pull/7388) ([Amos Bird](https://github.com/amosbird)) -- 修复 StoragesInfoStream 中未定义的行为. [#7384](https://github.com/ClickHouse/ClickHouse/pull/7384) ([tavplubix](https://github.com/tavplubix)) -- 修复了外部数据库引擎 (MySQL、ODBC、JDBC) 的常量表达式折叠. 在以前的版本中, 它不适用于多个常量表达式, 也不适用于 Date、DateTime 和 UUID. 这修复了 [#7245](https://github.com/ClickHouse/ClickHouse/issues/7245) [#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在访问 no_users_thread 变量时修复实时视图中的 ThreadSanitizer 数据竞争错误. [#7353](https://github.com/ClickHouse/ClickHouse/pull/7353) ([vzakaznikov](https://github.com/vzakaznikov)) -- 摆脱 libcommon 中的 malloc 符号. [#7134](https://github.com/ClickHouse/ClickHouse/pull/7134), [#7065](https://github.com/ClickHouse/ClickHouse/pull/7065) ([Amos - Bird](https://github.com/amosbird)) -- 添加全局标志 ENABLE_LIBRARIES 以禁用所有库. [#7063](https://github.com/ClickHouse/ClickHouse/pull/7063) ([proller](https://github.com/proller)) - -#### 代码清理 {#code-cleanup} - -- 通用化配置存储库以准备 DDL for Dictionaries. [#7155](https://github.com/ClickHouse/ClickHouse/pull/7155) ([alesapin](https://github.com/alesapin)) -- 没有任何语义的字典 DDL 解析器. [#7209](https://github.com/ClickHouse/ClickHouse/pull/7209) ([alesapin](https://github.com/alesapin)) -- 将 ParserCreateQuery 拆分为不同的较小解析器. [#7253](https://github.com/ClickHouse/ClickHouse/pull/7253) ([alesapin](https://github.com/alesapin)) -- 外部字典附近的小重构和重命名. [#7111](https://github.com/ClickHouse/ClickHouse/pull/7111) ([alesapin](https://github.com/alesapin)) -- 重构一些代码, 为基于角色的访问控制做准备. [#7235](https://github.com/ClickHouse/ClickHouse/pull/7235) ([Vitaly Baranov](https://github.com/vitlibar)) -- DatabaseOrdinary 代码的一些改进. [#7086](https://github.com/ClickHouse/ClickHouse/pull/7086) ([Nikita Vasilev](https://github.com/nikvas0)) -- 不要在哈希表的 find() 和 emplace() 方法中使用迭代器. [#7026](https://github.com/ClickHouse/ClickHouse/pull/7026) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 修复 getMultipleValuesFromConfig 以防参数 root 不为空. [#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) ([Mikhail Korotov](https://github.com/millb)) -- 删除一些复制粘贴 (TemporaryFile 和 TemporaryFileStream). [#7166](https://github.com/ClickHouse/ClickHouse/pull/7166) ([Artem Zuikov](https://github.com/4ertus2)) -- 提高了一点代码可读性 (`MergeTreeData::getActiveContainingPart`). [#7361](https://github.com/ClickHouse/ClickHouse/pull/7361) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 如果 `ThreadPool::schedule(...)` 抛出异常, 则等待所有使用本地对象的计划作业. 将 `ThreadPool::schedule(...)` 重命名为 `ThreadPool::scheduleOrThrowOnError(...)` 并修复注释以明确它可能会抛出. [#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([tavplubix](https://github.com/tavplubix)) - -## ClickHouse 版本 19.15 {#clickhouse-release-19-15} - -### ClickHouse 版本 19.15.4.10, 2019-10-31 {#clickhouse-release-19-15-4-10-2019-10-31} - -#### Bug 修复 {#bug-fix-3} - -- 添加了 SQL BIGINT 中 SQL TINYINT 的处理, 并修复了 ODBC Bridge 中 SQL_FLOAT 数据源类型的处理. [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) -- 允许在 MOVE PARTITION 中的目标磁盘或卷上有一些部分. [#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 通过 ODBC 桥修复可空列中的 NULL 值. [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) -- 修复了 INSERT 到具有 MATERIALIZED 列的分布式非本地节点. [#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) -- 固定函数 getMultipleValuesFromConfig. [#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) ([Mikhail Korotov](https://github.com/millb)) -- 修复了使用 HTTP 保持活动超时而不是 TCP 保持活动超时的问题. [#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily Nemkov](https://github.com/Enmk)) -- 等待所有作业在异常情况下完成 (修复罕见的段错误). [#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([tavplubix](https://github.com/tavplubix)) -- 插入 Kafka 表时不要推送到 MV. [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) -- 禁用异常堆栈的内存跟踪器. [#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了转换外部数据库查询中的错误代码. [#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 避免在 MetricsTransmitter 中使用未初始化的值. [#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) -- 添加了带有用于测试的宏的示例配置. ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.15.3.6, 2019-10-09 {#clickhouse-release-19-15-3-6-2019-10-09} - -#### Bug 修复 {#bug-fix-4} - -- 修复了散列字典中的 bad_variant. ([alesapin](https://github.com/alesapin)) -- 修复了 ATTACH PART 查询中分段错误的错误. ([alesapin](https://github.com/alesapin)) -- 修复了`MergeTreeData` 中的时间计算. ([Vladimir Chebotarev](https://github.com/excitoon)) -- 在编写完成后显式提交给Kafka. [#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) -- 在 MergeTree 部分的最小/最大索引中正确序列化 NULL 值. [#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -### ClickHouse 版本 19.15.2.2, 2019-10-01 {#clickhouse-release-19-15-2-2-2019-10-01} - -#### 新特征 {#new-feature-3} - -- 分层存储: 支持使用 MergeTree 引擎为表使用多个存储卷. 可以在 SSD 上存储新数据并自动将旧数据移动到 HDD. ([example](https://clickhouse.github.io/clickhouse-presentations/meetup30/new_features/#12)). [#4918](https://github.com/ClickHouse/ClickHouse/pull/4918) ([Igr](https://github.com/ObjatieGroba)) [#6489](https://github.com/ClickHouse/ClickHouse/pull/6489) ([alesapin](https://github.com/alesapin)) -- 添加表函数 `input`, 用于在 `INSERT SELECT` 查询中读取传入数据. [#5450](https://github.com/ClickHouse/ClickHouse/pull/5450) ([palasonic1](https://github.com/palasonic1)) [#6832](https://github.com/ClickHouse/ClickHouse/pull/6832) ([Anton Popov](https://github.com/CurtizJ)) -- 添加一个 `sparse_hashed` 字典布局, 它在功能上等同于 `hashed` 布局, 但内存效率更高. 它以较慢的值检索为代价使用大约两倍的内存. [#6894](https://github.com/ClickHouse/ClickHouse/pull/6894) ([Azat Khuzhin](https://github.com/azat)) -- 实现定义用户列表以访问字典的能力. 仅使用当前连接的数据库. [#6907](https://github.com/ClickHouse/ClickHouse/pull/6907) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 向`SHOW`查询添加`LIMIT`选项. [#6944](https://github.com/ClickHouse/ClickHouse/pull/6944) ([Philipp Malkovsky](https://github.com/malkfilipp)) -- 添加 `bitmapSubsetLimit(bitmap, range_start, limit)` 函数, 返回集合中不小于 `range_start` 的最小 `limit` 值的子集. [#6957](https://github.com/ClickHouse/ClickHouse/pull/6957) ([Zhichang Yu](https://github.com/yuzhichang)) -- 添加 `bitmapMin` 和 `bitmapMax` 函数. [#6970](https://github.com/ClickHouse/ClickHouse/pull/6970) ([Zhichang Yu](https://github.com/yuzhichang)) -- 添加与 [issue-6648](https://github.com/ClickHouse/ClickHouse/issues/6648) [#6999](https://github.com/ClickHouse/ClickHouse/pull/6999) 相关的函数 `repeat`.([flynn](https://github.com/ucasFL)) - -#### 实验功能 {#experimental-feature-1} - -- 实现 (在内存中) 不改变当前管道的 Merge Join 变体. 结果按合并键部分排序. 设置 `partial_merge_join = 1` 以使用此功能. Merge Join 仍在开发中. [#6940](https://github.com/ClickHouse/ClickHouse/pull/6940) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加 `S3` 引擎和表格功能。. 它仍在开发中 (尚无身份验证支持). [#5596](https://github.com/ClickHouse/ClickHouse/pull/5596) ([Vladimir Chebotarev](https://github.com/excitoon)) - -#### 改进 {#improvement-2} - -- 从 Kafka 读取的每条消息都是以原子方式插入的. 这解决了 Kafka 引擎的几乎所有已知问题. [#6950](https://github.com/ClickHouse/ClickHouse/pull/6950) ([Ivan](https://github.com/abyss7)) -- 分布式查询故障转移的改进. 缩短恢复时间, 现在它也是可配置的, 可以在 `system.clusters` 中看到. [#6399](https://github.com/ClickHouse/ClickHouse/pull/6399) ([Vasily Nemkov](https://github.com/Enmk)) -- 直接在 `IN` 部分支持枚举的数值. #6766 [#6941](https://github.com/ClickHouse/ClickHouse/pull/6941) ([dimarub2000](https://github.com/dimarub2000)) -- 支持 (可选, 默认禁用) 重定向 URL 存储. [#6914](https://github.com/ClickHouse/ClickHouse/pull/6914) ([maqroll](https://github.com/maqroll)) -- 添加旧版本客户端连接到服务器时的信息消息. [#6893](https://github.com/ClickHouse/ClickHouse/pull/6893) ([Philipp Malkovsky](https://github.com/malkfilipp)) -- 删除分布式表中发送数据的最大退避睡眠时间限制. [#6895](https://github.com/ClickHouse/ClickHouse/pull/6895) ([Azat Khuzhin](https://github.com/azat)) -- 添加将带有累积值的配置文件事件 (计数器) 发送到石墨的功能. 它可以在服务器 `config.xml` 中的 `` 下启用. [#6969](https://github.com/ClickHouse/ClickHouse/pull/6969) ([Azat Khuzhin](https://github.com/azat)) -- 在通过 HTTP 以本机格式将数据插入类型为 `LowCardinality(T)` 的列中时, 自动将类型 `T` 添加到 `LowCardinality(T)` . [#6891](https://github.com/ClickHouse/ClickHouse/pull/6891) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 添加使用函数 `hex` 而不使用 `reinterpretAsString` 为 `Float32` 和 `Float64` 的能力. [#7024](https://github.com/ClickHouse/ClickHouse/pull/7024) ([Mikhail Korotov](https://github.com/millb)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-2} - -- 将 gdb-index 添加到带有调试信息的 clickhouse 二进制文件中. 它将加快 `gdb` 的启动时间. [#6947](https://github.com/ClickHouse/ClickHouse/pull/6947) ([alesapin](https://github.com/alesapin)) -- 使用使用 `pigz` 的补丁dpkg-deb加速deb打包. [#6960](https://github.com/ClickHouse/ClickHouse/pull/6960) ([alesapin](https://github.com/alesapin)) -- 设置 `enable_fuzzing = 1` 以启用所有项目代码的 libfuzzer 检测. [#7042](https://github.com/ClickHouse/ClickHouse/pull/7042) ([kyprizel](https://github.com/kyprizel)) -- 在 CI 中添加 split build 冒烟测试. [#7061](https://github.com/ClickHouse/ClickHouse/pull/7061) ([alesapin](https://github.com/alesapin)) -- 将带有 MemorySanitizer 的构建添加到 CI. [#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alexander Kuzmenkov](https://github.com/akuzm)) -- 用 `sparsehash-c11` 替换 `libsparsehash`. [#6965](https://github.com/ClickHouse/ClickHouse/pull/6965) ([Azat Khuzhin](https://github.com/azat)) - -#### Bug 修复 {#bug-fix-5} - -- 修复了大型表上复杂键的索引分析性能下降. 这修复了 #6924. [#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复从 Kafka 空主题中选择时导致段错误的逻辑错误. [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) -- 修复 `MySQLBlockInputStream.cpp` 中过早关闭 MySQL 连接. [#6882](https://github.com/ClickHouse/ClickHouse/pull/6882) ([Clément Rodriguez](https://github.com/clemrodriguez)) -- 返回对非常旧的 Linux 内核的支持 (修复 [#6841](https://github.com/ClickHouse/ClickHouse/issues/6841)) [#6853](https://github.com/ClickHouse/ClickHouse/pull/6853) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在输入流中出现空块的情况下,修复 `insert select` 查询中可能丢失的数据. #6834 #6862 [#6911](https://github.com/ClickHouse/ClickHouse/pull/6911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复参数中带有空数组的函数 `АrrayEnumerateUniqRanked`. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) -- 使用数组连接和全局子查询修复复杂查询. [#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) -- 使用多个 JOIN 修复 ORDER BY 和 GROUP BY 中的 `Unknown identifier` 错误. [#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了在执行带有 `LowCardinality` 参数的函数时发出的 `MSan` 警告. [#7062](https://github.com/ClickHouse/ClickHouse/pull/7062) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### 向后不兼容的更新 {#backward-incompatible-change-2} - -- 更改了位图\* 聚合函数状态的序列化格式以提高性能. 无法读取之前版本中位图\* 的序列化状态. [#6908](https://github.com/ClickHouse/ClickHouse/pull/6908) ([Zhichang Yu](https://github.com/yuzhichang)) - -## ClickHouse 版本 19.14 {#clickhouse-release-19-14} - -### ClickHouse 版本 19.14.7.15, 2019-10-02 {#clickhouse-release-19-14-7-15-2019-10-02} - -#### Bug 修复 {#bug-fix-6} - -- 此版本还包含 19.11.12.69 的所有错误修复. -- 修复了 19.14 和更早版本之间分布式查询的兼容性. 这修复了 [#7068](https://github.com/ClickHouse/ClickHouse/issues/7068). [#7069](https://github.com/ClickHouse/ClickHouse/pull/7069) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.14.6.12, 2019-09-19 {#clickhouse-release-19-14-6-12-2019-09-19} - -#### Bug 修复 {#bug-fix-7} - -- 修复函数 `АrrayEnumerateUniqRanked` 在参数中带有空数组. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) -- 使用带有别名的 `ARRAY JOIN` 和 `GLOBAL IN subquery` 修复了查询中的子查询名称. 如果指定了外部表名, 则使用子查询别名. [#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-3} - -- 修复 [flapping](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) 测试 `00715_fetch_merged_or_mutated_part_zookeeper` 00715_mut_fetch_merged 的脚本需要重新写入 shellpart_或重新写入它的 shell 脚本申请. [#6977](https://github.com/ClickHouse/ClickHouse/pull/6977) ([Alexander Kazakov](https://github.com/Akazz)) -- 修复了函数 `groupUniqArray` 中的 UBSan 和 MemSan 失败,并带有 emtpy 数组参数. 这是由于没有调用零单元格值的构造函数, 将空的 `PaddPODArray` 放入哈希表零单元格中引起的. [#6937](https://github.com/ClickHouse/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) - -### ClickHouse 版本 19.14.3.3, 2019-09-10 {#clickhouse-release-19-14-3-3-2019-09-10} - -#### 新特征 {#new-feature-4} - -- `ORDER BY` 的 `WITH FILL` 修饰符. (continuation of [#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) -- `LIMIT` 的 `WITH TIES` 修饰符. (continuation of [#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) -- 将未加引号的 `NULL` 文字解析为 NULL (如果设置 `format_csv_unquoted_null_literal_as_null=1`). 如果此字段的数据类型不可为空, 则使用默认值初始化空字段 (如果设置 `input_format_null_as_default=1`). [#5990](https://github.com/ClickHouse/ClickHouse/issues/5990) [#6055](https://github.com/ClickHouse/ClickHouse/pull/6055) ([tavplubix](https://github.com/tavplubix)) -- 支持在表函数 `file` 和 `hdfs` 的路径中使用通配符. 如果路径包含通配符, 则该表将是只读的. 用法示例: `select * from hdfs('hdfs://hdfs1:9000/some_dir/another_dir/*/file{0..9}{0..9}')` 和 `select * from file('some_dir /{some_file,another_file,yet_another}.tsv', 'TSV', 'value UInt32')`. [#6092](https://github.com/ClickHouse/ClickHouse/pull/6092) ([Olga Khvostikova](https://github.com/stavrolia)) -- 新的 `system.metric_log` 表, 它存储具有指定时间间隔的 `system.events` 和 `system.metrics` 的值. [#6363](https://github.com/ClickHouse/ClickHouse/issues/6363) [#6467](https://github.com/ClickHouse/ClickHouse/pull/6467) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [#6530](https://github.com/ClickHouse/ClickHouse/pull/6530) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 允许将 ClickHouse 文本日志写入 `system.text_log` 表. [#6037](https://github.com/ClickHouse/ClickHouse/issues/6037) [#6103](https://github.com/ClickHouse/ClickHouse/pull/6103) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [#6164](https://github.com/ClickHouse/ClickHouse/pull/6164) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在堆栈跟踪中显示私有符号 (这是通过解析 ELF 文件的符号表完成的). 如果存在调试信息, 则在堆栈跟踪中添加有关文件和行号的信息. 使用程序中存在的索引符号加速符号名称查找. 添加了用于自省的新 SQL 函数: `demangle` 和 `addressToLine`. 将函数 `symbolizeAddress` 重命名为 `addressToSymbol` 以保持一致性. 出于性能原因, 函数`addressToSymbol` 将返回损坏的名称, 您必须应用`demangle`. 添加了默认关闭的设置 `allow_introspection_functions`. [#6201](https://github.com/ClickHouse/ClickHouse/pull/6201) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 表函数 `values` (名称不区分大小写). 它允许从 [#5984](https://github.com/ClickHouse/ClickHouse/issues/5984) 中提出的 `VALUES` 列表中读取. 示例: `SELECT * FROM VALUES('a UInt64, s String', (1, 'one'), (2, 'two'), (3, 'three'))`. [#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([dimarub2000](https://github.com/dimarub2000)) -- 添加了更改存储设置的功能. 语法: `ALTER TABLE MODIFY SETTING = `. [#6366](https://github.com/ClickHouse/ClickHouse/pull/6366) [#6669](https://github.com/ClickHouse/ClickHouse/pull/6669) [#6685](https://github.com/ClickHouse/ClickHouse/pull/6685) ([alesapin](https://github.com/alesapin)) -- 支持拆卸分离部件. 语法: `ALTER TABLE DROP DETACHED PART ''`. [#6158](https://github.com/ClickHouse/ClickHouse/pull/6158) ([tavplubix](https://github.com/tavplubix)) -- 表约束. 允许向表定义添加约束, 这将在插入时检查. [#5273](https://github.com/ClickHouse/ClickHouse/pull/5273) ([Gleb Novikov](https://github.com/NanoBjorn)) [#6652](https://github.com/ClickHouse/ClickHouse/pull/6652) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 支持级联物化视图. [#6324](https://github.com/ClickHouse/ClickHouse/pull/6324) ([Amos Bird](https://github.com/amosbird)) -- 默认情况下打开查询分析器以每秒对每个查询执行线程进行一次采样. [#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 输入格式 `ORC`. [#6454](https://github.com/ClickHouse/ClickHouse/pull/6454) [#6703](https://github.com/ClickHouse/ClickHouse/pull/6703) ([akonyaev90](https://github.com/akonyaev90)) -- 添加了两个新函数: `sigmoid` 和 `tanh` (对机器学习应用程序很有用). [#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 函数 `hasToken(haystack, token)`, `hasTokenCaseInsensitive(haystack, token)` 来检查给定的令牌是否在 haystack 中. 令牌是两个非字母数字 ASCII 字符 (或 haystack 的边界) 之间的最大长度子串. 令牌必须是一个常量字符串. 由 tokenbf_v1 索引专业化支持. [#6596](https://github.com/ClickHouse/ClickHouse/pull/6596), [#6662](https://github.com/ClickHouse/ClickHouse/pull/6662) ([Vasily Nemkov](https://github.com/Enmk)) -- 新函数 `neighbor(value, offset[, default_value])`. 允许在数据块中的列中达到上一个/下一个值. [#5925](https://github.com/ClickHouse/ClickHouse/pull/5925) ([Alex Krash](https://github.com/alex-krash)) [6685365ab8c5b74f9650492c88a012596eb1b0c6](https://github.com/ClickHouse/ClickHouse/commit/6685365ab8c5b74f9650492c88a012596eb1b0c6) [341e2e4587a18065c2da1ca888c73389f48ce36c](https://github.com/ClickHouse/ClickHouse/commit/341e2e4587a18065c2da1ca888c73389f48ce36c) [Alexey Milovidov](https://github.com/alexey-milovidov) -- 创建了一个函数 `currentUser()`, 返回授权用户的登录信息. 添加别名 `user()` 以兼容 MySQL. [#6470](https://github.com/ClickHouse/ClickHouse/pull/6470) ([Alex Krash](https://github.com/alex-krash)) -- [#5885](https://github.com/ClickHouse/ClickHouse/issues/5885) 中提出的新聚合函数 `quantilesExactInclusive` 和`quantilesExactExclusive`. [#6477](https://github.com/ClickHouse/ClickHouse/pull/6477) ([dimarub2000](https://github.com/dimarub2000)) -- 函数 `bitmapRange(bitmap, range_begin, range_end)` 返回具有指定范围的新集合 (不包括`range_end`). [#6314](https://github.com/ClickHouse/ClickHouse/pull/6314) ([Zhichang Yu](https://github.com/yuzhichang)) -- 函数 `geohashesInBox(longitude_min, latitude_min, longitude_max, latitude_max, precision)` 它创建覆盖提供区域的 geohash-boxes 的精确长字符串数组. [#6127](https://github.com/ClickHouse/ClickHouse/pull/6127) ([Vasily Nemkov](https://github.com/Enmk)) -- 使用 `Kafka` 表实现对 INSERT 查询的支持. [#6012](https://github.com/ClickHouse/ClickHouse/pull/6012) ([Ivan](https://github.com/abyss7)) -- 向 Kafka 引擎添加了对 `_partition` 和 `_timestamp` 虚拟列的支持. [#6400](https://github.com/ClickHouse/ClickHouse/pull/6400) ([Ivan](https://github.com/abyss7)) -- 可以从`query_log`、服务器日志、使用基于正则表达式的规则的进程列表中删除敏感数据. [#5710](https://github.com/ClickHouse/ClickHouse/pull/5710) ([filimonov](https://github.com/filimonov)) - -#### 实验功能 {#experimental-feature-2} - -- 输入和输出数据格式 `Template` . 它允许为输入和输出指定自定义格式字符串. [#4354](https://github.com/ClickHouse/ClickHouse/issues/4354) [#6727](https://github.com/ClickHouse/ClickHouse/pull/6727) ([tavplubix](https://github.com/tavplubix)) -- 最初在 [#2898](https://github.com/ClickHouse/ClickHouse/pull/2898) 中提出的 `LIVE VIEW` 表的实现, 在 [#3925](https://github.com/ClickHouse) 中准备 /ClickHouse/issues/3925),然后在 [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) 中更新. 详细说明见[#5541](https://github.com/ClickHouse/ClickHouse/issues/5541). [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) ([vzakaznikov](https://github.com/vzakaznikov)) [#6425](https://github.com/ ClickHouse/ClickHouse/pull/6425) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#6656](https://github.com/ClickHouse/ClickHouse/pull/6656) ([vzakaznikov] (https://github.com/vzakaznikov)) 请注意,下一个版本可能会删除 `LIVE VIEW` 功能. - -#### Bug 修复 {#bug-fix-8} - -- 此版本还包含 19.13 和 19.11 的所有错误修复. -- 修复表有跳过索引和垂直合并时的分段错误. [#6723](https://github.com/ClickHouse/ClickHouse/pull/6723) ([alesapin](https://github.com/alesapin)) -- 使用非平凡的列默认值修复每列 TTL. 以前在强制 TTL 合并与 `OPTIMIZE ... FINAL` 查询的情况下, 过期值被替换为类型默认值而不是用户指定的列默认值. [#6796](https://github.com/ClickHouse/ClickHouse/pull/6796) ([Anton Popov](https://github.com/CurtizJ)) -- 修复正常服务器重启时Kafka消息重复问题. [#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) -- 修复了读取 Kafka 消息时的无限循环. 根本不要暂停/恢复消费者订阅 - 否则在某些情况下它可能会无限期暂停. [#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([Ivan](https://github.com/abyss7)) -- 修复 `bitmapContains` 函数中 `Key expression contains comparison between inconvertible types` 异常. [#6136](https://github.com/ClickHouse/ClickHouse/issues/6136) [#6146](https://github.com/ClickHouse/ClickHouse/issues/6146) [#6156](https://github.com/ClickHouse/ClickHouse/pull/6156) ([dimarub2000](https://github.com/dimarub2000)) -- 使用启用的 `optimize_skip_unused_shards` 和丢失的分片键修复段错误. [#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([Anton Popov](https://github.com/CurtizJ)) -- 修复了可能导致内存损坏的突变中的错误代码. 修复了由于从 `system.parts` 或 `system.parts_columns` 并发 `DROP TABLE` 和 `SELECT` 而可能发生的读取地址 `0x14c0` 的段错误. 修复了准备突变查询时的竞争条件. 修复复制表的 `OPTIMIZE` 和 ALTER 等并发修改操作导致的死锁. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在 MySQL 界面中删除了额外的详细日志记录. [#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 返回从配置文件中的 `true` 和 `false` 解析布尔设置的能力. [#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) -- 修复了 `Nullable(Decimal128)` 上 `quantile` 和 `median` 函数中的崩溃. [#6378](https://github.com/ClickHouse/ClickHouse/pull/6378) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了主键上包含转换为 Float 类型的带有 `WHERE` 条件的 `SELECT` 查询可能返回的不完整结果. 它是由 `toFloat` 函数中的单调性检查不正确引起的. [#6248](https://github.com/ClickHouse/ClickHouse/issues/6248) [#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) -- 检查 `max_expanded_ast_elements` 设置是否有突变. 在 `TRUNCATE TABLE` 之后清除突变. [#6205](https://github.com/ClickHouse/ClickHouse/pull/6205) ([Winter Zhang](https://github.com/zhang2014)) -- 与`join_use_nulls` 一起使用时,修复键列的JOIN 结果. 附加空值而不是列默认值. [#6249](https://github.com/ClickHouse/ClickHouse/pull/6249) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复具有垂直合并和更改的跳过索引. 修复 `Bad size of marks file` 异常. [#6594](https://github.com/ClickHouse/ClickHouse/issues/6594) [#6713](https://github.com/ClickHouse/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) -- 当合并/更改的部分之一为空(0 行)时, 修复 `ALTER MODIFY COLUMN` 和垂直合并中的罕见崩溃. [#6746](https://github.com/ClickHouse/ClickHouse/issues/6746) [#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) -- 修复了在 `AggregateFunctionFactory` 中转换 `LowCardinality` 类型的错误. 这修复了 [#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复 `topK` 和 `topK Weighted` 聚合函数中的错误行为和可能的段错误. [#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([Anton Popov](https://github.com/CurtizJ)) -- 修复了围绕 `getIdentifier` 函数的不安全代码. [#6401](https://github.com/ClickHouse/ClickHouse/issues/6401) [#6409](https://github.com/ClickHouse/ClickHouse/pull/6409) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 MySQL 有线协议中的错误(在连接到 ClickHouse 表单 MySQL 客户端时使用). 由 `PacketPayloadWriteBuffer` 中的堆缓冲区溢出引起. [#6212](https://github.com/ClickHouse/ClickHouse/pull/6212) ([Yuriy Baranov](https://github.com/yurriy)) -- 修复了 `bitmapSubsetInRange` 函数中的内存泄漏. [#6819](https://github.com/ClickHouse/ClickHouse/pull/6819) ([Zhichang Yu](https://github.com/yuzhichang)) -- 修复粒度更改后执行变异时的罕见错误. [#6816](https://github.com/ClickHouse/ClickHouse/pull/6816) ([alesapin](https://github.com/alesapin)) -- 默认允许所有字段的 protobuf 消息. [#6132](https://github.com/ClickHouse/ClickHouse/pull/6132) ([Vitaly Baranov](https://github.com/vitlibar)) -- 当我们在第二个参数上发送一个 `NULL` 参数时,解决了一个带有 `nullIf` 函数的错误. [#6446](https://github.com/ClickHouse/ClickHouse/pull/6446) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 修复带有字符串字段的复杂键缓存字典中内存分配/释放错误的罕见错误, 这会导致无限内存消耗(看起来像内存泄漏). 当字符串大小是从 8 开始的 2 的幂(8、16、32 等)时会重现错误. [#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) -- 修复了小序列上的 Gorilla 编码导致异常 `Cannot write after end of buffer`. [#6398](https://github.com/ClickHouse/ClickHouse/issues/6398) [#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Vasily Nemkov](https://github.com/Enmk)) -- 允许在启用了 `join_use_nulls` 的 JOIN 中使用不可为空的类型. [#6705](https://github.com/ClickHouse/ClickHouse/pull/6705) ([Artem Zuikov](https://github.com/4ertus2)) -- 在 `clickhouse-client` 中禁用查询中的 `Poco::AbstractConfiguration` 替换. [#6706](https://github.com/ClickHouse/ClickHouse/pull/6706) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 避免 `REPLACE PARTITION` 中的死锁. [#6677](https://github.com/ClickHouse/ClickHouse/pull/6677) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 对常量参数使用 `arrayReduce` 可能会导致段错误. [#6242](https://github.com/ClickHouse/ClickHouse/issues/6242) [#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复在 `DROP PARTITION` 之后恢复副本时可能出现的不一致部分. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -- 修复了 `JSONExtractRaw` 函数挂起的问题. [#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用自适应粒度修复不正确的跳过索引序列化和聚合的错误. [#6594](https://github.com/ClickHouse/ClickHouse/issues/6594). [#6748](https://github.com/ClickHouse/ClickHouse/pull/6748) ([alesapin](https://github.com/alesapin)) -- 使用两级聚合修复 `GROUP BY` 的 `WITH ROLLUP` 和 `WITH CUBE` 修饰符. [#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) -- 修复以自适应粒度写入二级索引标记的错误. [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) -- 修复服务器启动时的初始化顺序. 由于 `StorageMergeTree::background_task_handle` 是在 `startup()` 中初始化的, 所以 `MergeTreeBlockOutputStream::write()` 可能会在初始化之前尝试使用它. 只需检查它是否已初始化. [#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) -- 清除上一个因错误而完成的读操作的数据缓冲区. [#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) -- 修复为 Replicated\*MergeTree 表创建新副本时启用自适应粒度的错误. [#6394](https://github.com/ClickHouse/ClickHouse/issues/6394) [#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) -- 修复了在访问未初始化的 `ThreadStatus` 结构时出现异常时在 `libunwind` 中发生异常的情况下服务器启动期间可能发生的崩溃. [#6456](https://github.com/ClickHouse/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- 修复 `yandexConsistentHash` 函数中的崩溃. 通过模糊测试发现. [#6304](https://github.com/ClickHouse/ClickHouse/issues/6304) [#6305](https://github.com/ClickHouse/ClickHouse/pull/6305) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了服务器过载和全局线程池接近满时挂起查询的可能性. 这在具有大量分片(数百个)的集群上发生的可能性更高, 因为分布式查询为每个连接分配一个线程到每个分片. 例如, 如果 330 个分片的集群正在处理 30 个并发分布式查询, 则此问题可能会重现. 此问题影响从 19.2 开始的所有版本. [#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了`arrayEnumerateUniqRanked` 函数的逻辑. [#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修正解码符号表时的段错误. [#6603](https://github.com/ClickHouse/ClickHouse/pull/6603) ([Amos Bird](https://github.com/amosbird)) -- 修复了将 `LowCardinality(Nullable)` 转换为 not-Nullable 列中的不相关异常, 以防它不包含 Nulls (例如, 在像 `SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String)))] AS String 这样的查询中 )`. [#6094](https://github.com/ClickHouse/ClickHouse/issues/6094) [#6119](https://github.com/ClickHouse/ClickHouse/pull/6119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 删除了 `system.settings` 表中额外的描述引用. [#6696](https://github.com/ClickHouse/ClickHouse/issues/6696) [#6699](https://github.com/ClickHouse/ClickHouse/pull/6699) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 避免在复制表的 `TRUNCATE` 中可能出现死锁. [#6695](https://github.com/ClickHouse/ClickHouse/pull/6695) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修正了按键排序的顺序读取. [#6189](https://github.com/ClickHouse/ClickHouse/pull/6189) ([Anton Popov](https://github.com/CurtizJ)) -- 修复带有 `enable_mixed_granularity_parts=1` 的表的 `ALTER TABLE ... UPDATE` 查询. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -- 修复 [#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) 打开的错误 (自 19.4.0 起). 当我们不查询任何列 (`SELECT 1`) 时, 通过 MergeTree 表在对分布式表的查询中重现. [#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) -- 修复了有符号类型到无符号类型的整数除法中的溢出. 该行为与 C 或 C++ 语言 (整数提升规则) 中的行为完全相同, 这可能令人惊讶. 请注意, 将大有符号数除以大无符号数或反之亦然时仍然可能发生溢出 (但这种情况不太常见). 所有服务器版本都存在该问题. [#6214](https://github.com/ClickHouse/ClickHouse/issues/6214) [#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 当设置了 `max_execution_speed` 或 `max_execution_speed_bytes` 时, 限制用于节流的最大休眠时间. 修复了 `Estimated query execution time (inf seconds) is too long` 等错误错误. [#5547](https://github.com/ClickHouse/ClickHouse/issues/5547) [#6232](https://github.com/ClickHouse/ClickHouse/pull/6232) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在`MaterializedView`中使用`MATERIALIZED`列和别名的问题. [#448](https://github.com/ClickHouse/ClickHouse/issues/448) [#3484](https://github.com/ClickHouse/ClickHouse/issues/3484) [#3450](https://github.com/ClickHouse/ClickHouse/issues/3450) [#2878](https://github.com/ClickHouse/ClickHouse/issues/2878) [#2285](https://github.com/ClickHouse/ClickHouse/issues/2285) [#3796](https://github.com/ClickHouse/ClickHouse/pull/3796) ([Amos Bird](https://github.com/amosbird)) [#6316](https://github.com/ClickHouse/ClickHouse/pull/6316) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复未实现为处理器的输入流的 `FormatFactory` 行为. [#6495](https://github.com/ClickHouse/ClickHouse/pull/6495) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修正了错字. [#6631](https://github.com/ClickHouse/ClickHouse/pull/6631) ([Alex Ryndin](https://github.com/alexryndin)) -- 错误消息中的错字 ( is -\> are ). [#6839](https://github.com/ClickHouse/ClickHouse/pull/6839) ([Denis Zhuravlev](https://github.com/den-crane)) -- 如果类型包含逗号, 则从字符串解析列列表时已修复错误 (此问题与 `File`、`URL`、`HDFS` 存储相关) [#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([dimarub2000](https://github.com/dimarub2000)) - -#### 安全修复 {#security-fix} - -- 此版本还包含 19.13 和 19.11 的所有错误安全修复. -- 修复了由于 SQL 解析器中的堆栈溢出, 伪造查询导致服务器崩溃的可能性. 修复了合并和分布式表、物化视图和涉及子查询的行级安全条件中堆栈溢出的可能性. [#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 改进 {#improvement-3} - -- 正确实现 `AND/OR` 的三元逻辑. [#6048](https://github.com/ClickHouse/ClickHouse/pull/6048) ([Alexander Kazakov](https://github.com/Akazz)) -- 现在, 在从没有 TTL 信息或过时的 TTL 信息的旧部分进行 `OPTIMIZE ... FINAL` 查询后, 将删除具有过期 TTL 的值和行, 例如, 在 `ALTER ... MODIFY TTL` 查询之后. 添加了查询 `SYSTEM STOP/START TTL MERGES` 以禁止/允许使用 TTL 分配合并并过滤所有合并中的过期值. [#6274](https://github.com/ClickHouse/ClickHouse/pull/6274) ([Anton Popov](https://github.com/CurtizJ)) -- 可以使用 `CLICKHOUSE_HISTORY_FILE` env 为客户端更改 ClickHouse 历史文件的位置. [#6840](https://github.com/ClickHouse/ClickHouse/pull/6840) ([filimonov](https://github.com/filimonov)) -- 从 `InterpreterSelectQuery` 中删除 `dry_run` 标志. … [#6375](https://github.com/ClickHouse/ClickHouse/pull/6375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 支持带有 `ON` 部分的 `ASOF JOIN`. [#6211](https://github.com/ClickHouse/ClickHouse/pull/6211) ([Artem Zuikov](https://github.com/4ertus2)) -- 更好地支持用于突变和复制的跳过索引. 支持 `MATERIALIZE/CLEAR INDEX ... IN PARTITION` 查询. `UPDATE x = x` 重新计算所有使用列 `x` 的索引. [#5053](https://github.com/ClickHouse/ClickHouse/pull/5053) ([Nikita Vasilev](https://github.com/nikvas0)) -- 无论 `allow_experimental_live_view` 设置如何, 都允许使用 `ATTACH` 实时视图 (例如,在服务器启动时). [#6754](https://github.com/ClickHouse/ClickHouse/pull/6754) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 对于查询分析器收集的堆栈跟踪, 不包括查询分析器本身生成的堆栈帧. [#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 现在表函数`values`、`file`、`url`、`hdfs` 支持ALIAS 列. [#6255](https://github.com/ClickHouse/ClickHouse/pull/6255) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 如果 `config.d` 文件没有对应的根元素作为配置文件, 则抛出异常. [#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) -- 在 `no space left on device` 的异常消息中打印额外信息. [#6182](https://github.com/ClickHouse/ClickHouse/issues/6182), [#6252](https://github.com/ClickHouse/ClickHouse/issues/6252) [#6352](https://github.com/ClickHouse/ClickHouse/pull/6352) ([tavplubix](https://github.com/tavplubix)) -- 在确定读取查询要覆盖的 `Distributed` 表的分片时 (对于 `optimize_skip_unused_shards` =1), ClickHouse 现在检查来自 select 语句的 `prewhere` 和 `where` 子句的条件. [#6521](https://github.com/ClickHouse/ClickHouse/pull/6521) ([Alexander Kazakov](https://github.com/Akazz)) -- 为没有 AVX2 但有 SSE 4.2 和 PCLMUL 指令集的机器启用`SIMDJSON`. [#6285](https://github.com/ClickHouse/ClickHouse/issues/6285) [#6320](https://github.com/ClickHouse/ClickHouse/pull/6320) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- ClickHouse 可以在没有 `O_DIRECT` 支持的文件系统上工作(例如 ZFS 和 BtrFS),无需额外调整. [#4449](https://github.com/ClickHouse/ClickHouse/issues/4449) [#6730](https://github.com/ClickHouse/ClickHouse/pull/6730) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 支持最终子查询的下推谓词. [#6120](https://github.com/ClickHouse/ClickHouse/pull/6120) ([TCeason](https://github.com/TCeason)) [#6162](https://github.com/ClickHouse/ClickHouse/pull/6162) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 更好的`JOIN ON`键提取. [#6131](https://github.com/ClickHouse/ClickHouse/pull/6131) ([Artem Zuikov](https://github.com/4ertus2)) -- 更新了 `SIMDJSON`. [#6285](https://github.com/ClickHouse/ClickHouse/issues/6285). [#6306](https://github.com/ClickHouse/ClickHouse/pull/6306) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 优化 `SELECT count()` 查询最小列的选择. [#6344](https://github.com/ClickHouse/ClickHouse/pull/6344) ([Amos Bird](https://github.com/amosbird)) -- 在 `windowFunnel()` 中添加了 `strict` 参数. 当设置了 `strict` 时, `windowFunnel()` 只对唯一值应用条件. [#6548](https://github.com/ClickHouse/ClickHouse/pull/6548) ([achimbab](https://github.com/achimbab)) -- `mysqlxx::Pool` 更安全的接口. [#6150](https://github.com/ClickHouse/ClickHouse/pull/6150) ([avasiliev](https://github.com/avasiliev)) -- 使用 `--help` 选项执行时的选项行大小现在对应于终端大小. [#6590](https://github.com/ClickHouse/ClickHouse/pull/6590) ([dimarub2000](https://github.com/dimarub2000)) -- 禁用无键聚合的 `read in order` 优化. [#6599](https://github.com/ClickHouse/ClickHouse/pull/6599) ([Anton Popov](https://github.com/CurtizJ)) -- `INCORRECT_DATA` 和 `TYPE_MISMATCH` 错误代码的 HTTP 状态代码从默认的 `500 Internal Server Error` 更改为 `400 Bad Request`. [#6271](https://github.com/ClickHouse/ClickHouse/pull/6271) ([Alexander Rodin](https://github.com/a-rodin)) -- 将 Join 对象从 `ExpressionAction` 移动到 `AnalyzedJoin`. `ExpressionAnalyzer` 和 `ExpressionAction` 不再了解 `Join` 类. 它的逻辑被`AnalyzedJoin` iface隐藏了. [#6801](https://github.com/ClickHouse/ClickHouse/pull/6801) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了当分片之一是本地主机但查询是通过网络连接发送时分布式查询可能出现的死锁. [#6759](https://github.com/ClickHouse/ClickHouse/pull/6759) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 更改了多个表 `RENAME` 的语义以避免可能的死锁. [#6757](https://github.com/ClickHouse/ClickHouse/issues/6757). [#6756](https://github.com/ClickHouse/ClickHouse/pull/6756) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 重写 MySQL 兼容性服务器以防止在内存中加载完整的数据包负载. 每个连接的内存消耗减少到大约 `2 * DBMS_DEFAULT_BUFFER_SIZE` (读/写缓冲区). [#5811](https://github.com/ClickHouse/ClickHouse/pull/5811) ([Yuriy Baranov](https://github.com/yurriy)) -- 将 AST 别名解释逻辑移出不必了解查询语义的解析器. [#6108](https://github.com/ClickHouse/ClickHouse/pull/6108) ([Artem Zuikov](https://github.com/4ertus2)) -- NamesAndTypesList` 稍微更安全的解析. [#6408](https://github.com/ClickHouse/ClickHouse/issues/6408). [#6410](https://github.com/ClickHouse/ClickHouse/pull/6410) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `clickhouse-copier`: 允许在查询中使用带有 `partition_key` 别名的配置中的 `where_condition` 来检查分区是否存在 (之前它仅用于读取数据查询). [#6577](https://github.com/ClickHouse/ClickHouse/pull/6577) ([proller](https://github.com/proller)) -- 在 `throwIf` 中添加了可选的消息参数. ([#5772](https://github.com/ClickHouse/ClickHouse/issues/5772)) [#6329](https://github.com/ClickHouse/ClickHouse/pull/6329) ([Vdimir](https://github.com/Vdimir)) -- 发送插入数据时出现的服务器异常现在也在客户端处理. [#5891](https://github.com/ClickHouse/ClickHouse/issues/5891) [#6711](https://github.com/ClickHouse/ClickHouse/pull/6711) ([dimarub2000](https://github.com/dimarub2000)) -- 添加了一个指标 `DistributedFilesToInsert` , 显示文件系统中被分布式表选择发送到远程服务器的文件总数. 该数字是所有分片的总和. [#6600](https://github.com/ClickHouse/ClickHouse/pull/6600) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将大部分 JOIN 准备逻辑从 `ExpressionAction/ExpressionAnalyzer` 移动到 `AnalyzedJoin`. [#6785](https://github.com/ClickHouse/ClickHouse/pull/6785) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 TSan [warning](https://clickhouse-test-reports.s3.yandex.net/6399/c1c1d1daa98e199e620766f1bd06a5921050a00d/functional_stateful_tests_(thread).html) `lock-order-inversion`. [#6740](https://github.com/ClickHouse/ClickHouse/pull/6740) ([Vasily Nemkov](https://github.com/Enmk)) -- 关于缺乏 Linux 功能的更好的信息消息. 使用 `fatal` 级别记录致命错误, 这将使其更容易在 `system.text_log` 中找到. [#6441](https://github.com/ClickHouse/ClickHouse/pull/6441) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 当启用将临时数据转储到磁盘以限制 `GROUP BY`、`ORDER BY`期间的内存使用时, 它没有检查可用磁盘空间. 修复添加一个新的设置 `min_free_disk_space`, 当可用磁盘空间小于阈值时, 查询将停止并抛出 `ErrorCodes::NOT_ENOUGH_SPACE`. [#6678](https://github.com/ClickHouse/ClickHouse/pull/6678) ([Weiqing Xu](https://github.com/weiqxu)) [#6691](https://github.com/ClickHouse/ClickHouse/pull/6691) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 通过线程删除递归 rwlock. 这是没有意义的,因为线程在查询之间被重用. `SELECT` 查询可能会在一个线程中获取锁, 从另一个线程持有锁并从第一个线程退出. 同时, 第一个线程可以被 `DROP` 查询重用。 这将导致错误的 `Attempt to acquire exclusive lock recursively` 消息. [#6771](https://github.com/ClickHouse/ClickHouse/pull/6771) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 拆分 `Expression Analyzer.appendLine()`. 在 `ExpressionAnalyzer` 中为 `Merge Join` 准备一个位置. [#6524](https://github.com/ClickHouse/ClickHouse/pull/6524) ([Artem Zuikov](https://github.com/4ertus2)) -- 向 MySQL 兼容服务器添加了 `mysql_native_password` 身份验证插件. [#6194](https://github.com/ClickHouse/ClickHouse/pull/6194) ([Yuriy Baranov](https://github.com/yurriy)) -- 更少的 `clock_gettime` 调用; 修复了 `Allocator` 中调试/发布之间的 ABI 兼容性 (无关紧要的问题). [#6197](https://github.com/ClickHouse/ClickHouse/pull/6197) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将 `collectUsedColumns` 从 `ExpressionAnalyzer` 移至 `Syntax Analyzer`. `Syntax Analyzer` 现在生成 `required_source_columns`. [#6416](https://github.com/ClickHouse/ClickHouse/pull/6416) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加设置 `joined_subquery_requires_alias` 以要求 `FROM` 中存在多个表的子选择和表函数的别名 (比如使用JOIN的查询). [#6733](https://github.com/ClickHouse/ClickHouse/pull/6733) ([Artem Zuikov](https://github.com/4ertus2)) -- 从 `ExpressionAnalyzer` 中提取 `GetAggregatesVisitor` 类. [#6458](https://github.com/ClickHouse/ClickHouse/pull/6458) ([Artem Zuikov](https://github.com/4ertus2)) -- `system.query_log`: 将 `type` 列的数据类型更改为 `Enum`. [#6265](https://github.com/ClickHouse/ClickHouse/pull/6265) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- `sha256_password` 认证插件的静态链接. [#6512](https://github.com/ClickHouse/ClickHouse/pull/6512) ([Yuriy Baranov](https://github.com/yurriy)) -- 避免设置 `compile` 工作的额外依赖. 在以前的版本中, 用户可能会收到诸如 `cannot open crti.o`、`unable to find library -lc` 等错误. [#6309](https://github.com/ClickHouse/ClickHouse/pull/6309) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 对可能来自恶意副本的输入进行更多验证. [#6303](https://github.com/ClickHouse/ClickHouse/pull/6303) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 现在 `clickhouse-obfuscator` 文件在 `clickhouse-client` 包中可用. 在以前的版本中, 它可用作 `clickhouse obfuscator` (带空格). [#5816](https://github.com/ClickHouse/ClickHouse/issues/5816) [#6609](https://github.com/ClickHouse/ClickHouse/pull/6609) ([dimarub2000](https://github.com/dimarub2000)) -- 当我们有至少两个以不同顺序读取至少两个表的查询和另一个对其中一个表执行 DDL 操作的查询时, 修复了死锁. 修复了另一个非常罕见的僵局. [#6764](https://github.com/ClickHouse/ClickHouse/pull/6764) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 向 `system.processes` 和 `system.query_log` 添加了 `os_thread_ids` 列以获得更好的调试可能性. [#6763](https://github.com/ClickHouse/ClickHouse/pull/6763) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 当 `sha256_password` 用作默认身份验证插件时发生的PHP mysqlnd扩展错误的解决方法 (在[#6031](https://github.com/ClickHouse/ClickHouse/issues/6031)中描述). [#6113](https://github.com/ClickHouse/ClickHouse/pull/6113) ([Yuriy Baranov](https://github.com/yurriy)) -- 使用已更改的可空性列删除不需要的位置. [#6693](https://github.com/ClickHouse/ClickHouse/pull/6693) ([Artem Zuikov](https://github.com/4ertus2)) -- 将 `queue_max_wait_ms` 的默认值设置为零, 因为当前值(五秒) 没有意义. 在极少数情况下, 此设置有任何用处. 添加了设置 `replace_running_query_max_wait_ms`、`kafka_max_wait_ms` 和 `connection_pool_max_wait_ms` 以消除歧义. [#6692](https://github.com/ClickHouse/ClickHouse/pull/6692) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 从 `ExpressionAnalyzer` 中提取 `SelectQueryExpressionAnalyzer`. 保留最后一个用于非选择查询. [#6499](https://github.com/ClickHouse/ClickHouse/pull/6499) ([Artem Zuikov](https://github.com/4ertus2)) -- 删除了重复的输入和输出格式. [#6239](https://github.com/ClickHouse/ClickHouse/pull/6239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 允许用户在连接时覆盖 `poll_interval` 和 `idle_connection_timeout` 设置. [#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `MergeTree` 现在有一个附加选项 `ttl_only_drop_parts` (默认情况下禁用) 以避免部分修剪部分, 以便在部分中的所有行都过期时完全删除. [#6191](https://github.com/ClickHouse/ClickHouse/pull/6191) ([Sergi Vladykin](https://github.com/svladykin)) -- 类型检查集合索引函数. 如果函数类型错误, 则抛出异常. 这修复了 UBSan 的模糊测试. [#6511](https://github.com/ClickHouse/ClickHouse/pull/6511) ([Nikita Vasilev](https://github.com/nikvas0)) - -#### 性能改进 {#performance-improvement-2} - -- 使用 `ORDER BY expressions` 子句优化查询, 其中 `expressions` 具有与 `MergeTree` 表中的排序键重合的前缀. 此优化由 `optimize_read_in_order` 设置控制. [#6054](https://github.com/ClickHouse/ClickHouse/pull/6054) [#6629](https://github.com/ClickHouse/ClickHouse/pull/6629) ([Anton Popov](https://github.com/CurtizJ)) -- 允许在零件装载和拆卸期间使用多个螺纹. [#6372](https://github.com/ClickHouse/ClickHouse/issues/6372) [#6074](https://github.com/ClickHouse/ClickHouse/issues/6074) [#6438](https://github.com/ClickHouse/ClickHouse/pull/6438) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 实现了更新聚合函数状态的批处理变体. 它可能会带来性能优势. [#6435](https://github.com/ClickHouse/ClickHouse/pull/6435) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 对函数 `exp`、`log`、`sigmoid`、`tanh`使用 `FastOps` 库. FastOps 是 Michael Parakhin(Yandex CTO)的一个快速向量数学库. `exp` 和 `log` 函数的性能提高了 6 倍以上. `Float32` 参数中的函数 `exp` 和 `log` 将返回 `Float32`(在以前的版本中,它们总是返回 `Float64`). 现在 `exp(nan)` 可能返回 `inf`. `exp` 和 `log` 函数的结果可能不是最接近真实答案的机器可表示数字. [#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) 使用 Danila Kutenin 变体使 fastops 工作 [ #6317](https://github.com/ClickHouse/ClickHouse/pull/6317) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 禁用 `UInt8/16` 的连续键优化. [#6298](https://github.com/ClickHouse/ClickHouse/pull/6298) [#6701](https://github.com/ClickHouse/ClickHouse/pull/6701) ([akuzm](https://github.com/akuzm)) -- 通过摆脱 `ParsedJson::Iterator` 中的动态分配, 提高了 `simdjson` 库的性能. [#6479](https://github.com/ClickHouse/ClickHouse/pull/6479) ([Vitaly Baranov](https://github.com/vitlibar)) -- 使用 `mmap()` 分配内存时的故障前页面. [#6667](https://github.com/ClickHouse/ClickHouse/pull/6667) ([akuzm](https://github.com/akuzm)) -- 修复 `Decimal` 比较中的性能错误. [#6380](https://github.com/ClickHouse/ClickHouse/pull/6380) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-4} - -- 删除编译器 (运行时模板实例化), 因为我们已经赢得了它的性能. [#6646](https://github.com/ClickHouse/ClickHouse/pull/6646) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了性能测试, 以更孤立的方式显示 gcc-9 中的性能下降. [#6302](https://github.com/ClickHouse/ClickHouse/pull/6302) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了表函数`numbers_mt`, 它是`numbers`的多线程版本. 使用哈希函数更新性能测试. [#6554](https://github.com/ClickHouse/ClickHouse/pull/6554) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- `clickhouse-benchmark` 中的比较模式. [#6220](https://github.com/ClickHouse/ClickHouse/issues/6220) [#6343](https://github.com/ClickHouse/ClickHouse/pull/6343) ([dimarub2000](https://github.com/dimarub2000)) -- 尽最大努力打印堆栈跟踪. 还添加了 `SIGPROF` 作为调试信号以打印正在运行的线程的堆栈跟踪. [#6529](https://github.com/ClickHouse/ClickHouse/pull/6529) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 每个函数都在自己的文件中, 第 10 部分. [#6321](https://github.com/ClickHouse/ClickHouse/pull/6321) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 删除加倍的 const `TABLE_IS_READ_ONLY`. [#6566](https://github.com/ClickHouse/ClickHouse/pull/6566) ([filimonov](https://github.com/filimonov)) -- `StringHashMap` PR [#5417](https://github.com/ClickHouse/ClickHouse/issues/5417) 的格式更改. [#6700](https://github.com/ClickHouse/ClickHouse/pull/6700) ([akuzm](https://github.com/akuzm)) -- 在`ExpressionAnalyzer` 中创建连接的更好子查询. [#6824](https://github.com/ClickHouse/ClickHouse/pull/6824) ([Artem Zuikov](https://github.com/4ertus2)) -- 删除冗余条件 (由 PVS Studio 发现). [#6775](https://github.com/ClickHouse/ClickHouse/pull/6775) ([akuzm](https://github.com/akuzm)) -- 分离`ReverseIndex`的哈希表接口. [#6672](https://github.com/ClickHouse/ClickHouse/pull/6672) ([akuzm](https://github.com/akuzm)) -- 重构设置. [#6689](https://github.com/ClickHouse/ClickHouse/pull/6689) ([alesapin](https://github.com/alesapin)) -- 为 `set` 索引函数添加注释. [#6319](https://github.com/ClickHouse/ClickHouse/pull/6319) ([Nikita Vasilev](https://github.com/nikvas0)) -- 在 Linux 上的调试版本中提高 OOM 分数. [#6152](https://github.com/ClickHouse/ClickHouse/pull/6152) ([akuzm](https://github.com/akuzm)) -- HDFS HA 现在可用于调试版本. [#6650](https://github.com/ClickHouse/ClickHouse/pull/6650) ([Weiqing Xu](https://github.com/weiqxu)) -- 向 `transform_query_for_external_database` 添加了一个测试. [#6388](https://github.com/ClickHouse/ClickHouse/pull/6388) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 为 Kafka 表添加多个物化视图的测试. [#6509](https://github.com/ClickHouse/ClickHouse/pull/6509) ([Ivan](https://github.com/abyss7)) -- 制定更好的构建方案. [#6500](https://github.com/ClickHouse/ClickHouse/pull/6500) ([Ivan](https://github.com/abyss7)) -- 修复了 `test_external_dictionaries` 集成, 以防它在非 root 用户下执行. [#6507](https://github.com/ClickHouse/ClickHouse/pull/6507) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 当写入数据包的总大小超过`DBMS_DEFAULT_BUFFER_SIZE`时, 该错误会重现. [#6204](https://github.com/ClickHouse/ClickHouse/pull/6204) ([Yuriy Baranov](https://github.com/yurriy)) -- 添加了对 `RENAME` 表竞争条件的测试. [#6752](https://github.com/ClickHouse/ClickHouse/pull/6752) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 避免 `KILL QUERY` 中设置的数据竞争. [#6753](https://github.com/ClickHouse/ClickHouse/pull/6753) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加集成测试以通过缓存字典处理错误. [#6755](https://github.com/ClickHouse/ClickHouse/pull/6755) ([Vitaly Baranov](https://github.com/vitlibar)) -- 在 Mac OS 上禁用解析 ELF 对象文件, 因为它没有意义. [#6578](https://github.com/ClickHouse/ClickHouse/pull/6578) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 尝试使变更日志生成器更好. [#6327](https://github.com/ClickHouse/ClickHouse/pull/6327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将 `-Wshadow` 开关添加到 GCC. [#6325](https://github.com/ClickHouse/ClickHouse/pull/6325) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -- 删除了对 `mimalloc` 支持的过时代码. [#6715](https://github.com/ClickHouse/ClickHouse/pull/6715) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `zlib-ng` 确定 x86 功能并将此信息保存到全局变量. 这是在 defalteInit 调用中完成的, 它可以由不同的线程同时进行. 为避免多线程写入, 请在库启动时执行. [#6141](https://github.com/ClickHouse/ClickHouse/pull/6141) ([akuzm](https://github.com/akuzm)) -- 对在 [#5192](https://github.com/ClickHouse/ClickHouse/issues/5192) 中修复的连接中的错误进行回归测试. [#6147](https://github.com/ClickHouse/ClickHouse/pull/6147) ([Bakhtiyor Ruziev](https://github.com/theruziev)) -- 修复了 MSan 报告. [#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复抖动 TTL 测试. [#6782](https://github.com/ClickHouse/ClickHouse/pull/6782) ([Anton Popov](https://github.com/CurtizJ)) -- 修复了 `MergeTreeDataPart::is_frozen` 字段中的错误数据竞争. [#6583](https://github.com/ClickHouse/ClickHouse/pull/6583) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了模糊测试中的超时. 在以前的版本中, 它设法在查询 `SELECT * FROM numbers_mt(gccMurmurHash(''))` 中找到错误挂断. [#6582](https://github.com/ClickHouse/ClickHouse/pull/6582) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 向列的 `static_cast` 添加调试检查. [#6581](https://github.com/ClickHouse/ClickHouse/pull/6581) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在官方 RPM 包中支持 Oracle Linux. [#6356](https://github.com/ClickHouse/ClickHouse/issues/6356) [#6585](https://github.com/ClickHouse/ClickHouse/pull/6585) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将 json perftests 从 `once` 更改为 `loop` 类型. [#6536](https://github.com/ClickHouse/ClickHouse/pull/6536) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- `odbc-bridge.cpp` 定义了 `main()`, 因此它不应包含在 `clickhouse-lib` 中. [#6538](https://github.com/ClickHouse/ClickHouse/pull/6538) ([Orivej Desh](https://github.com/orivej)) -- 测试 `FULL|RIGHT JOIN` 中的崩溃, 右表的键为空. [#6362](https://github.com/ClickHouse/ClickHouse/pull/6362) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加了对别名扩展限制的测试以防万一. [#6442](https://github.com/ClickHouse/ClickHouse/pull/6442) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在适当的地方从 `boost::filesystem` 切换到 `std::filesystem`. [#6253](https://github.com/ClickHouse/ClickHouse/pull/6253) [#6385](https://github.com/ClickHouse/ClickHouse/pull/6385) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 向网站添加了 RPM 包. [#6251](https://github.com/ClickHouse/ClickHouse/pull/6251) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在 `IN` 部分添加修复 `Unknown identifier` 异常的测试. [#6708](https://github.com/ClickHouse/ClickHouse/pull/6708) ([Artem Zuikov](https://github.com/4ertus2)) -- 简化`shared_ptr_helper`, 因为人们难以理解它. [#6675](https://github.com/ClickHouse/ClickHouse/pull/6675) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了针对固定 Gorilla 和 DoubleDelta 编解码器的性能测试. [#6179](https://github.com/ClickHouse/ClickHouse/pull/6179) ([Vasily Nemkov](https://github.com/Enmk)) -- 将集成测试 `test_dictionaries` 拆分为 4 个单独的测试. [#6776](https://github.com/ClickHouse/ClickHouse/pull/6776) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复 `PipelineExecutor` 中的 PVS-Studio 警告. [#6777](https://github.com/ClickHouse/ClickHouse/pull/6777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 允许在 ASan 中使用 `library` 字典源. [#6482](https://github.com/ClickHouse/ClickHouse/pull/6482) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了从 PR 列表生成变更日志的选项. [#6350](https://github.com/ClickHouse/ClickHouse/pull/6350) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 读取时锁定`TinyLog`存储. [#6226](https://github.com/ClickHouse/ClickHouse/pull/6226) ([akuzm](https://github.com/akuzm)) -- 检查 CI 中损坏的符号链接. [#6634](https://github.com/ClickHouse/ClickHouse/pull/6634) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 增加 `stack overflow` 测试的超时时间,因为在调试构建中可能需要很长时间. [#6637](https://github.com/ClickHouse/ClickHouse/pull/6637) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了对双空格的检查. [#6643](https://github.com/ClickHouse/ClickHouse/pull/6643) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复使用消毒剂构建时的 `new/delete` 内存跟踪. 跟踪不清楚. 它只防止测试中的内存限制异常. [#6450](https://github.com/ClickHouse/ClickHouse/pull/6450) ([Artem Zuikov](https://github.com/4ertus2)) -- 在链接时启用对未定义符号的检查. [#6453](https://github.com/ClickHouse/ClickHouse/pull/6453) ([Ivan](https://github.com/abyss7)) -- 避免每天重建 `hyperscan` . [#6307](https://github.com/ClickHouse/ClickHouse/pull/6307) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `ProtobufWriter` 中的 UBSan 报告. [#6163](https://github.com/ClickHouse/ClickHouse/pull/6163) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 不允许将查询分析器与消毒剂一起使用, 因为它不兼容. [#6769](https://github.com/ClickHouse/ClickHouse/pull/6769) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加测试以在计时器失败后重新加载字典. [#6114](https://github.com/ClickHouse/ClickHouse/pull/6114) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复 `PipelineExecutor::prepareProcessor` 参数类型的不一致. [#6494](https://github.com/ClickHouse/ClickHouse/pull/6494) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 添加了对错误 URI 的测试. [#6493](https://github.com/ClickHouse/ClickHouse/pull/6493) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 向 `CAST` 函数添加了更多检查. 这应该在模糊测试中获得有关分段错误的更多信息. [#6346](https://github.com/ClickHouse/ClickHouse/pull/6346) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 为在本地构建镜像的 `docker/builder` 容器添加了 `gcc-9` 支持. [#6333](https://github.com/ClickHouse/ClickHouse/pull/6333) ([Gleb Novikov](https://github.com/NanoBjorn)) -- 使用 `Low Cardinality(String)` 测试主键. [#5044](https://github.com/ClickHouse/ClickHouse/issues/5044) [#6219](https://github.com/ClickHouse/ClickHouse/pull/6219) ([dimarub2000](https://github.com/dimarub2000)) -- 修复了受缓慢堆栈跟踪打印影响的测试. [#6315](https://github.com/ClickHouse/ClickHouse/pull/6315) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在 [#6029](https://github.com/ClickHouse/ClickHouse/pull/6029) 中修复的`groupUniqArray` 中添加崩溃测试用例. [#4402](https://github.com/ClickHouse/ClickHouse/issues/4402) [#6129](https://github.com/ClickHouse/ClickHouse/pull/6129) ([akuzm](https://github.com/akuzm)) -- 修复指数变异测试. [#6645](https://github.com/ClickHouse/ClickHouse/pull/6645) ([Nikita Vasilev](https://github.com/nikvas0)) -- 在性能测试中, 不要读取我们没有运行的查询的查询日志. [#6427](https://github.com/ClickHouse/ClickHouse/pull/6427) ([akuzm](https://github.com/akuzm)) -- 现在可以使用任何低基数类型创建物化视图, 而不管有关可疑低基数类型的设置. [#6428](https://github.com/ClickHouse/ClickHouse/pull/6428) ([Olga Khvostikova](https://github.com/stavrolia)) -- 更新了 `send_logs_level` 设置的测试. [#6207](https://github.com/ClickHouse/ClickHouse/pull/6207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复 gcc-8.2 下的构建. [#6196](https://github.com/ClickHouse/ClickHouse/pull/6196) ([Max Akhmedov](https://github.com/zlobober)) -- 使用内部 libc++ 修复构建. [#6724](https://github.com/ClickHouse/ClickHouse/pull/6724) ([Ivan](https://github.com/abyss7)) -- 使用 `rdkafka` 库修复共享构建. [#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) -- Mac OS 版本的修复 (不完整). [#6390](https://github.com/ClickHouse/ClickHouse/pull/6390) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#6429](https://github.com/ClickHouse/ClickHouse/pull/6429) ([alex-zaitsev](https://github.com/alex-zaitsev)) -- 修复 `splitted` 构建. [#6618](https://github.com/ClickHouse/ClickHouse/pull/6618) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 其他构建修复: [#6186](https://github.com/ClickHouse/ClickHouse/pull/6186) ([Amos Bird](https://github.com/amosbird)) [#6486](https://github.com/ClickHouse/ClickHouse/pull/6486) [#6348](https://github.com/ClickHouse/ClickHouse/pull/6348) ([vxider](https://github.com/Vxider)) [#6744](https://github.com/ClickHouse/ClickHouse/pull/6744) ([Ivan](https://github.com/abyss7)) [#6016](https://github.com/ClickHouse/ClickHouse/pull/6016) [#6421](https://github.com/ClickHouse/ClickHouse/pull/6421) [#6491](https://github.com/ClickHouse/ClickHouse/pull/6491) ([proller](https://github.com/proller)) - -#### 向后不兼容的更新 {#backward-incompatible-change-3} - -- 删除了很少使用的表函数 `catBoostPool` 和存储 `CatBoostPool`. 如果您使用过此表格功能, 请写邮件至 `clickhouse-feedback@yandex-team.com`. 请注意, CatBoost 集成仍然存在并将得到支持. [#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 默认情况下禁用 `ANY RIGHT JOIN` 和 `ANY FULL JOIN`. 设置 `any_join_distinct_right_table_keys` 设置以启用它们. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) - -## ClickHouse 版本 19.13 {#clickhouse-release-19-13} - -### ClickHouse 版本 19.13.6.51, 2019-10-02 {#clickhouse-release-19-13-6-51-2019-10-02} - -#### Bug 修复 {#bug-fix-9} - -- 此版本还包含 19.11.12.69 的所有错误修复. - -### ClickHouse 版本 19.13.5.44, 2019-09-20 {#clickhouse-release-19-13-5-44-2019-09-20} - -#### Bug 修复 {#bug-fix-10} - -- 此版本还包含 19.14.6.12 的所有错误修复. -- 修复了在 Zookeeper 不可访问时对复制表执行 `DROP` 查询时可能出现的表不一致状态. [#6045](https://github.com/ClickHouse/ClickHouse/issues/6045) [#6413](https://github.com/ClickHouse/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- 修复 StorageMerge 中的数据竞争. [#6717](https://github.com/ClickHouse/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复查询分析器中引入的错误, 该错误导致来自套接字的无休止的接收. [#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) ([alesapin](https://github.com/alesapin)) -- 修复在布尔值上执行 `JSONExtractRaw` 函数时 CPU 使用率过高的问题. [#6208](https://github.com/ClickHouse/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复了推送到物化视图时的回归. [#6415](https://github.com/ClickHouse/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) -- 表函数 `url` 存在允许攻击者在请求中注入任意 HTTP 头的漏洞. 此问题由 [Nikita Tikhomirov](https://github.com/NSTikhomirov) 发现. [#6466](https://github.com/ClickHouse/ClickHouse/pull/6466) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 Set index 中无用的 `AST` 检查. [#6510](https://github.com/ClickHouse/ClickHouse/issues/6510) [#6651](https://github.com/ClickHouse/ClickHouse/pull/6651) ([Nikita Vasilev](https://github.com/nikvas0)) -- 修复了对嵌入在查询中的 `AggregateFunction` 值的解析. [#6575](https://github.com/ClickHouse/ClickHouse/issues/6575) [#6773](https://github.com/ClickHouse/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) -- 修复了`trim` 函数族的错误行为. [#6647](https://github.com/ClickHouse/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.13.4.32, 2019-09-10 {#clickhouse-release-19-13-4-32-2019-09-10} - -#### Bug 修复 {#bug-fix-11} - -- 此版本还包含 19.11.9.52 和 19.11.10.54 的所有错误安全修复. -- 修复了 `system.parts` 表和 `ALTER` 查询中的数据争用. [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245) [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在从带有样本和 prewhere 的空分布式表中读取时发生的流中不匹配的标头. [#6167](https://github.com/ClickHouse/ClickHouse/issues/6167) ([Lixiang Qian](https://github.com/fancyqlx)) [#6823](https://github.com/ClickHouse/ClickHouse/pull/6823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了在带有元组的子查询中使用 `IN` 子句时崩溃的问题. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -- 修复 `GLOBAL JOIN ON` 部分中列名相同的情况. [#6181](https://github.com/ClickHouse/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复将类型转换为不支持它的 `Decimal` 时的崩溃。 改为抛出异常. [#6297](https://github.com/ClickHouse/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了 `extractAll()` 函数中的崩溃问题. [#6644](https://github.com/ClickHouse/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) -- `MySQL`、`ODBC`、`JDBC` 表函数的查询转换现在可以正常用于带有多个 `AND` 表达式的 `SELECT WHERE` 查询. [#6381](https://github.com/ClickHouse/ClickHouse/issues/6381) [#6676](https://github.com/ClickHouse/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) -- 为 MySQL 8 集成添加了先前的声明检查. [#6569](https://github.com/ClickHouse/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) - -#### 安全修复 {#security-fix-1} - -- 修复解压阶段编解码器的两个漏洞 (恶意用户可以编造压缩数据导致解压缓冲区溢出). [#6670](https://github.com/ClickHouse/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) - -### ClickHouse 版本 19.13.3.26, 2019-08-22 {#clickhouse-release-19-13-3-26-2019-08-22} - -#### Bug 修复 {#bug-fix-12} - -- 修复带有 `enable_mixed_granularity_parts=1` 的表的 `ALTER TABLE ... UPDATE` 查询. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -- 用带有元组的子查询的 IN 子句时修复 NPE. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -- 修复了一个问题, 如果一个陈旧的副本变得活跃, 它可能仍然有被 DROP PARTITION 删除的数据部分. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -- 解决了解析 CSV 的问题. [#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) -- 修复了 system.parts 表和 ALTER 查询中的数据竞争. 这修复了 [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了可能导致内存损坏的突变中的错误代码. 修复了由于从 `system.parts` 或 `system.parts_columns` 并发 `DROP TABLE` 和 `SELECT` 而可能发生的读取地址 `0x14c0` 的段错误. 修复了准备突变查询时的竞争条件. 修复复制表的 `OPTIMIZE` 和 ALTER 等并发修改操作导致的死锁. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在跳过索引的表上进行`ALTER DELETE`查询后可能丢失的数据. [#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) - -#### 安全修复 {#security-fix-2} - -- 如果攻击者拥有对 ZooKeeper 的写访问权限, 并且能够从运行 ClickHouse 的网络中运行可用的自定义服务器, 则它可以创建自定义构建的恶意服务器, 该服务器将充当 ClickHouse 副本并将其注册到 ZooKeeper 中. 当另一个副本从恶意副本中获取数据部分时, 它可以强制 clickhouse-server 写入文件系统上的任意路径. 由 Yandex 信息安全团队 Eldar Zaitov 发现. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.13.2.19, 2019-08-14 {#clickhouse-release-19-13-2-19-2019-08-14} - -#### 新特征 {#new-feature-5} - -- 查询级别的采样分析器. [Example](https://gist.github.com/alexey-milovidov/92758583dd41c24c360fdb8d6a4da194). [#4247](https://github.com/ClickHouse/ClickHouse/issues/4247) ([laplab](https://github.com/laplab)) [#6124](https://github.com/ClickHouse/ClickHouse/pull/6124) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) [#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) [#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) -- 允许使用 `COLUMNS('regexp')` 表达式指定一个列列表, 它的工作方式类似于更复杂的 `*` 星号变体. [#5951](https://github.com/ClickHouse/ClickHouse/pull/5951) ([mfridental](https://github.com/mfridental)), ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `CREATE TABLE AS table_function()` 现在可以了. [#6057](https://github.com/ClickHouse/ClickHouse/pull/6057) ([dimarub2000](https://github.com/dimarub2000)) -- 于随机梯度下降的 Adam 优化器默认用于 `stochasticLinearRegression()` 和 `stochasticLogisticRegression()` 聚合函数, 因为它在几乎没有任何调整的情况下显示出良好的质量. [#6000](https://github.com/ClickHouse/ClickHouse/pull/6000) ([Quid37](https://github.com/Quid37)) -- 添加了使用自定义周数的功能. [#5212](https://github.com/ClickHouse/ClickHouse/pull/5212) ([Andy Yang](https://github.com/andyyzh)) -- RENAME` 查询现在适用于所有存储. [#5953](https://github.com/ClickHouse/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) -- 现在, 无论服务器设置中指定的日志级别如何, 客户端都可以通过设置 `send_logs_level` 从服务器接收任何所需级别的日志. [#5964](https://github.com/ClickHouse/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) - -#### 向后不兼容的更新 {#backward-incompatible-change-4} - -- 默认情况下启用设置 `input_format_defaults_for_omited_fields`。 分布式表中的插入需要此设置在集群上相同 (您需要在滚动更新之前设置它). 它可以为 `JSONEachRow` 和 `CSV*` 格式的省略字段计算复杂的默认表达式. 它应该是预期的行为, 但可能导致可以忽略不计的性能差异. [#6043](https://github.com/ClickHouse/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) - -#### 实验功能 {#experimental-features} - -- 新的查询处理管道. 使用 `experimental_use_processors=1` 选项来启用它. 用于自己的麻烦. [#4914](https://github.com/ClickHouse/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### Bug 修复 {#bug-fix-13} - -- Kafka 集成已在此版本中修复. -- 修复了大 `DoubleDelta` 值的 `Int64` 的 `DoubleDelta` 编码, 改进了 `Int32` 随机数据的 `DoubleDelta` 编码. [#5998](https://github.com/ClickHouse/ClickHouse/pull/5998) ([Vasily Nemkov](https://github.com/Enmk)) -- 如果设置 `merge_tree_uniform_read_distribution` 设置为 0, 则修复了对 `max_rows_to_read` 的高估. [#6019](https://github.com/ClickHouse/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 改进 {#improvement-4} - -- 如果 `config.d` 文件没有对应的根元素作为配置文件, 则抛出异常. [#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) - -#### 性能改进 {#performance-improvement-3} - -- 优化 `count()`. 现在它使用最小的列 (如果可能). [#6028](https://github.com/ClickHouse/ClickHouse/pull/6028) ([Amos Bird](https://github.com/amosbird)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-5} - -- 报告性能测试中的内存使用情况. [#5899](https://github.com/ClickHouse/ClickHouse/pull/5899) ([akuzm](https://github.com/akuzm)) -- 使用外部 `libcxx` 修复构建. [#6010](https://github.com/ClickHouse/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) -- 使用 `rdkafka` 库修复共享构建. [#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) - -## ClickHouse 版本 19.11 {#clickhouse-release-19-11} - -### ClickHouse 版本 19.11.13.74, 2019-11-01 {#clickhouse-release-19-11-13-74-2019-11-01} - -#### Bug 修复 {#bug-fix-14} - -- 修复了 `ALTER MODIFY COLUMN` 和垂直合并中的罕见崩溃,当合并/更改的部分之一为空时 (0 行). [#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) -- 手动更新`SIMDJSON`. 这修复了使用虚假 json 诊断消息可能导致 stderr 文件泛滥的问题. [#7548](https://github.com/ClickHouse/ClickHouse/pull/7548) ([Alexander Kazakov](https://github.com/Akazz)) -- 修复了用于突变的 `mrk` 文件扩展名的错误. ([alesapin](https://github.com/alesapin)) - -### ClickHouse 版本 19.11.12.69, 2019-10-02 {#clickhouse-release-19-11-12-69-2019-10-02} - -#### Bug 修复 {#bug-fix-15} - -- 修复了大型表上复杂键的索引分析性能下降. 这修复了 [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924). [#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用分布式引擎在表中发送数据时避免罕见的 SIGSEGV (`Failed to send batch: file with index XXXXX is absent`). [#7032](https://github.com/ClickHouse/ClickHouse/pull/7032) ([Azat Khuzhin](https://github.com/azat)) -- 使用多个连接修复 `Unknown identifier` . 这修复了 [#5254](https://github.com/ClickHouse/ClickHouse/issues/5254). [#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) - -### ClickHouse 版本 19.11.11.57, 2019-09-13 {#clickhouse-release-19-11-11-57-2019-09-13} - -- 修复从 Kafka 空主题中选择时导致段错误的逻辑错误. [#6902](https://github.com/ClickHouse/ClickHouse/issues/6902) [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) -- 修复函数 `АrrayEnumerateUniqRanked` 在参数中带有空数组. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) - -### ClickHouse 版本 19.11.10.54, 2019-09-10 {#clickhouse-release-19-11-10-54-2019-09-10} - -#### Bug 修复 {#bug-fix-16} - -- 手动为 Kafka 消息存储偏移量,以便能够一次为所有分区提交所有偏移量。 修复了 `one consumer - many partitions` 场景中的潜在重复. [#6872](https://github.com/ClickHouse/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) - -### ClickHouse 版本 19.11.9.52, 2019-09-6 {#clickhouse-release-19-11-9-52-2019-09-6} - -- 改进缓存字典中的错误处理. [#6737](https://github.com/ClickHouse/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复了函数 `arrayEnumerateUniqRanked` 中的错误. [#6779](https://github.com/ClickHouse/ClickHouse/pull/6779) ([proller](https://github.com/proller)) -- 在从 JSON 中提取 `Tuple` 时修复 `JSONExtract` 函数. [#6718](https://github.com/ClickHouse/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复了在跳过索引的表上进行`ALTER DELETE`查询后可能丢失的数据. [#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) -- 修复了性能测试. [#6392](https://github.com/ClickHouse/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Parquet: 修复读取布尔列. [#6579](https://github.com/ClickHouse/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了常量参数的 `nullIf` 函数的错误行为. [#6518](https://github.com/ClickHouse/ClickHouse/pull/6518) ([Guillaume Tassery](https://github.com/YiuRULE)) [#6580](https://github.com/ClickHouse/ClickHouse/pull/6580) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复正常服务器重启时Kafka消息重复问题. [#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) -- 修复了长时间 `ALTER UPDATE` 或 `ALTER DELETE` 可能会阻止常规合并运行的问题. 如果没有足够的可用线程, 则防止执行突变. [#6502](https://github.com/ClickHouse/ClickHouse/issues/6502) [#6617](https://github.com/ClickHouse/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) -- 修复了在服务器配置文件中处理 `timezone` 的错误. [#6709](https://github.com/ClickHouse/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 kafka 测试. [#6805](https://github.com/ClickHouse/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) - -#### 安全修复 {#security-fix-3} - -- 果攻击者拥有对 ZooKeeper 的写访问权限, 并且能够从运行 ClickHouse 的网络中运行可用的自定义服务器, 则它可以创建自定义构建的恶意服务器, 该服务器将充当 ClickHouse 副本并将其注册到 ZooKeeper 中. 当另一个副本从恶意副本中获取数据部分时, 它可以强制 clickhouse-server 写入文件系统上的任意路径. 由 Yandex 信息安全团队 Eldar Zaitov 发现. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.11.8.46, 2019-08-22 {#clickhouse-release-19-11-8-46-2019-08-22} - -#### Bug 修复 {#bug-fix-17} - -- 修复带有 `enable_mixed_granularity_parts=1` 的表的 `ALTER TABLE ... UPDATE` 查询. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -- 用带有元组的子查询的 IN 子句时修复 NPE. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -- 修复了一个问题, 如果一个陈旧的副本变得活跃, 它可能仍然有被 DROP PARTITION 删除的数据部分. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -- 修复了解析 CSV 的问题 [#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) -- 修复了 system.parts 表和 ALTER 查询中的数据竞争. 这修复了 [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了可能导致内存损坏的突变中的错误代码. 修复了由于从 `system.parts` 或 `system.parts_columns` 并发 `DROP TABLE` 和 `SELECT` 而可能发生的读取地址 `0x14c0` 的段错误. 修复了准备突变查询时的竞争条件. 修复复制表的 `OPTIMIZE` 和 ALTER 等并发修改操作导致的死锁. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.11.7.40, 2019-08-14 {#clickhouse-release-19-11-7-40-2019-08-14} - -#### Bug 修复 {#bug-fix-18} - -- Kafka 集成已在此版本中修复. -- 修复了当使用 `arrayReduce` 作为常量参数时的段错误. [#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `toFloat()` 单调性. [#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) -- 修复了启用的 `optimize_skip_unused_shards` 和丢失的分片键修复段错误. [#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([CurtizJ](https://github.com/CurtizJ)) -- 修复了`arrayEnumerateUniqRanked` 函数的逻辑. [#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 从 MySQL 处理程序中删除了额外的详细日志记录. [#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `topK` 和 `topK Weighted` 聚合函数中的错误行为和可能的段错误. [#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([CurtizJ](https://github.com/CurtizJ)) -- 不要在 `system.columns` 表中暴露虚拟列. 这是向后兼容所必需的. [#6406](https://github.com/ClickHouse/ClickHouse/pull/6406) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复复杂键缓存字典中字符串字段的内存分配错误. [#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) -- 修复在为 `Replicated*MergeTree` 表创建新副本时启用自适应粒度的错误. [#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) -- 修复读取 Kafka 消息时的无限循环. [#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([abyss7](https://github.com/abyss7)) -- 修复了由于 SQL 解析器中的堆栈溢出以及 `Merge` 和 `Distributed` 表中的堆栈溢出的可能性, 伪造查询导致服务器崩溃的可能性 [#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了小序列上的 Gorilla 编码错误. [#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Enmk](https://github.com/Enmk)) - -#### 改进 {#improvement-5} - -- 允许用户在连接时覆盖 `poll_interval` 和 `idle_connection_timeout` 设置. [#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.11.5.28, 2019-08-05 {#clickhouse-release-19-11-5-28-2019-08-05} - -#### Bug 修复 {#bug-fix-19} - -- 修复了服务器过载时挂起查询的可能性. [#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 yandexConsistentHash 函数中的 FPE. 这修复了 [#6304](https://github.com/ClickHouse/ClickHouse/issues/6304). [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在 `AggregateFunctionFactory` 中转换 `LowCardinality` 类型的错误. 这修复了 [#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复从配置文件中的 `true` 和 `false` 字符串解析 `bool` 设置. [#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) -- 当 `WHERE` 的一部分移动到 `PREWHERE` 时, 修复了在对 `MergeTree` 表上的 `Distributed` 表的查询中流标头不兼容的罕见错误. [#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) -- 修复了有符号类型到无符号类型的整数除法中的溢出. 这修复了 [#6214](https://github.com/ClickHouse/ClickHouse/issues/6214). [#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 向后不兼容的更新 {#backward-incompatible-change-5} - -- `Kafka` 仍然损坏. - -### ClickHouse 版本 19.11.4.24, 2019-08-01 {#clickhouse-release-19-11-4-24-2019-08-01} - -#### Bug 修复 {#bug-fix-20} - -- 修复以自适应粒度写入二级索引标记的错误. [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) -- 修复了带有两级聚合的 `GROUP BY` 的 `WITH ROLLUP` 和 `WITH CUBE` 修饰符. [#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) -- 修复了 `JSONExtractRaw` 函数中的挂起问题. 已修复 [#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 ExternalLoader::reloadOutdated() 中的段错误. [#6082](https://github.com/ClickHouse/ClickHouse/pull/6082) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复了服务器可能会关闭侦听套接字但不会关闭并继续服务剩余查询的情况. 您最终可能会运行两个 clickhouse-server 进程. 有时, 服务器可能会为剩余的查询返回错误 `bad_function_call`. [#6231](https://github.com/ClickHouse/ClickHouse/pull/6231) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了通过 ODBC、MySQL、ClickHouse 和 HTTP 初始加载外部词典时更新字段上无用和不正确的条件. 这修复了 [#6069](https://github.com/ClickHouse/ClickHouse/issues/6069) [#6083](https://github.com/ClickHouse/ClickHouse/pull/6083) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了将 `LowCardinality(Nullable)` 转换为 not-Nullable 列中的不相关异常,以防它不包含 Nulls(例如, 像在 `SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String))) AS String` 这样的查询中). [#6094](https://github.com/ClickHouse/ClickHouse/issues/6094) [#6119](https://github.com/ClickHouse/ClickHouse/pull/6119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 在极少数情况下修复 `uniq` 聚合函数的非确定性结果. 该错误存在于所有 ClickHouse 版本中. [#6058](https://github.com/ClickHouse/ClickHouse/pull/6058) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 当我们在函数 `IPv6CIDRToRange` 上设置太高的 CIDR 时出现段错误. [#6068](https://github.com/ClickHouse/ClickHouse/pull/6068) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 修复了服务器从许多不同的上下文中抛出许多异常时的小内存泄漏. [#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复消费者订阅前暂停, 订阅后未恢复的情况. [#6075](https://github.com/ClickHouse/ClickHouse/pull/6075) ([Ivan](https://github.com/abyss7)) Note that Kafka is broken in this version. -- 清除之前已完成但出现错误的读取操作中的 Kafka 数据缓冲区 [#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/ bopohaa)) 请注意, 此版本中 Kafka 已损坏. -- 由于 `StorageMergeTree::background_task_handle` 是在`startup()` 中初始化的, 所以 `MergeTreeBlockOutputStream::write()` 可能会在初始化之前尝试使用它. 只需检查它是否已初始化. [#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-6} - -- 添加官方`rpm`包. [#5740](https://github.com/ClickHouse/ClickHouse/pull/5740) ([proller](https://github.com/proller)) ([alesapin](https://github.com/alesapin)) -- 添加使用`packager`脚本构建`.rpm`和`.tgz`包的能力. [#5769](https://github.com/ClickHouse/ClickHouse/pull/5769) ([alesapin](https://github.com/alesapin)) -- 修复“Arcadia”构建系统. [#6223](https://github.com/ClickHouse/ClickHouse/pull/6223) ([proller](https://github.com/proller)) - -#### 向后不兼容的更新 {#backward-incompatible-change-6} - -- `Kafka` 在这个版本中仍然被损坏. - -### ClickHouse 版本 19.11.3.11, 2019-07-18 {#clickhouse-release-19-11-3-11-2019-07-18} - -#### 新特征 {#new-feature-6} - -- 添加了对准备好的语句的支持. [#5331](https://github.com/ClickHouse/ClickHouse/pull/5331/) ([Alexander](https://github.com/sanych73)) [#5630](https://github.com/ClickHouse/ClickHouse/pull/5630) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `DoubleDelta` 和 `Gorilla` 列编解码器. [#5600](https://github.com/ClickHouse/ClickHouse/pull/5600) ([Vasily Nemkov](https://github.com/Enmk)) -- 添加了 `os_thread_priority` 设置, 允许控制操作系统使用的查询处理线程的 `nice` 值来调整动态调度优先级. 它需要 `CAP_SYS_NICE` 功能才能工作. 这实现 [#5858](https://github.com/ClickHouse/ClickHouse/issues/5858) [#5909](https://github.com/ClickHouse/ClickHouse/pull/5909) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 为 Kafka 引擎实现 `_topic`、`_offset`、`_key` 列 [#5382](https://github.com/ClickHouse/ClickHouse/pull/5382) ([Ivan](https://github.com/ abyss7)) 请注意, 这个版本中 Kafka 已损坏. -- 添加聚合函数组合器 `-Resample`. [#5590](https://github.com/ClickHouse/ClickHouse/pull/5590) ([hcz](https://github.com/hczhcz)) -- 聚合函数`groupArrayMovingSum(win_size)(x)` 和`groupArrayMovingAvg(win_size)(x)`, 计算有或没有窗口大小限制的移动总和/平均. [#5595](https://github.com/ClickHouse/ClickHouse/pull/5595) ([inv2004](https://github.com/inv2004)) -- 添加同义词 `arrayFlatten` \<-\> `flatten`. [#5764](https://github.com/ClickHouse/ClickHouse/pull/5764) ([hcz](https://github.com/hczhcz)) -- 将优步的H3功能 `geoToH3` 集成在一起. [#4724](https://github.com/ClickHouse/ClickHouse/pull/4724) ([Remen Ivan](https://github.com/BHYCHIK)) [#5805](https://github.com/ClickHouse/ClickHouse/pull/5805) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Bug 修复 {#bug-fix-21} - -- 使用异步更新实现 DNS 缓存. 单独的线程解析所有主机并使用周期更新 DNS 缓存 (设置 `dns_cache_update_period`). 当主机的 ip 频繁更改时. 它应该会有所帮助. [#5857](https://github.com/ClickHouse/ClickHouse/pull/5857) ([Anton Popov](https://github.com/CurtizJ)) -- 修复 `Delta` 编解码器中的段错误, 该错误会影响值小于 32 位大小的列. 该错误导致随机内存损坏. [#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) -- 修复 TTL 合并中的段错误与块中的非物理列. [#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) -- 修复了检查带有 `LowCardinality` 列的部分的罕见错误. 以前, `checkDataPart` 对于带有 `LowCardinality` 列的部分总是失败. [#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) -- 避免在服务器线程池已满时挂起连接. 当连接超时很长时, 对于从 `remote` 表函数的连接或到没有副本的分片的连接很重要. 这修复了 [#5878](https://github.com/ClickHouse/ClickHouse/issues/5878) [#5881](https://github.com/ClickHouse/ClickHouse/pull/5881) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 支持 `evalMLModel` 函数的常量参数. 这修复了 [#5817](https://github.com/ClickHouse/ClickHouse/issues/5817) [#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 ClickHouse 将默认时区确定为 `UCT` 而不是 `UTC` 时的问题. 这修复了 [#5804](https://github.com/ClickHouse/ClickHouse/issues/5804). [#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `visitParamExtractRaw` 中的缓冲区下溢. 这修复了 [#5901](https://github.com/ClickHouse/ClickHouse/issues/5901) [#5902](https://github.com/ClickHouse/ClickHouse/pull/5902) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 现在分布式的`DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER`查询将直接在leader副本上执行. [#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) -- 使用 `ColumnNullable` + 相关更改修复 `ColumnConst` 的 `coalesce`. [#5755](https://github.com/ClickHouse/ClickHouse/pull/5755) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 `ReadBufferFromKafkaConsumer` 以便它在 `commit()` 之后继续读取新消息,即使它之前被停止. [#5852](https://github.com/ClickHouse/ClickHouse/pull/5852) ([Ivan](https://github.com/abyss7)) -- 在右表中的 `Nullable` 键上连接时修复 `FULL` 和 `RIGHT` 连接结果. [#5859](https://github.com/ClickHouse/ClickHouse/pull/5859) ([Artem Zuikov](https://github.com/4ertus2)) -- 低优先级查询无限休眠的可能修复. [#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复竞争条件, 导致某些查询在 `SYSTEM FLUSH LOGS` 查询后可能不会出现在 query_log 中. [#5456](https://github.com/ClickHouse/ClickHouse/issues/5456) [#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) -- 修复了 ClusterCopier 中的 `heap-use-after-free` ASan 警告,由 watch 导致尝试使用已删除的复制器对象. [#5871](https://github.com/ClickHouse/ClickHouse/pull/5871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了由 `IColumn::deserializeAndInsertFromArena` 的某些实现返回的错误 `StringRef` 指针. 此错误仅影响单元测试. [#5973](https://github.com/ClickHouse/ClickHouse/pull/5973) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 防止屏蔽同名列的源和中间数组连接列. [#5941](https://github.com/ClickHouse/ClickHouse/pull/5941) ([Artem Zuikov](https://github.com/4ertus2)) -- 使用 MySQL 样式标识符引用修复对 MySQL 引擎的插入和选择查询. [#5704](https://github.com/ClickHouse/ClickHouse/pull/5704) ([Winter Zhang](https://github.com/zhang2014)) -- 现在 `CHECK TABLE` 查询可以与 MergeTree 引擎系列一起使用. 如果每个部分 (或在更简单的引擎情况下的文件) 有任何检查状态和消息, 它会返回检查状态和消息. 此外, 修复了获取损坏部分的错误. [#5865](https://github.com/ClickHouse/ClickHouse/pull/5865) ([alesapin](https://github.com/alesapin)) -- 修复 SPLIT_SHARED_LIBRARIES 运行时. [#5793](https://github.com/ClickHouse/ClickHouse/pull/5793) ([Danila Kutenin](https://github.com/danlark1)) -- 当 `/etc/localtime` 是一个像 `../usr/share/zoneinfo/Asia/Istanbul` 这样的相对符号链接时, 修复了时区初始化 [#5922](https://github.com/ClickHouse/ClickHouse/pull/5922) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- clickhouse-copier: 修复关机后免费使用. [#5752](https://github.com/ClickHouse/ClickHouse/pull/5752) ([proller](https://github.com/proller)) -- 更新了 `simdjson` . 修复部分无效的零字节JSON解析成功的问题. [#5938](https://github.com/ClickHouse/ClickHouse/pull/5938) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复关闭系统日志 [#5802](https://github.com/ClickHouse/ClickHouse/pull/5802) ([Anton Popov](https://github.com/CurtizJ)) -- 修复 invalidate_query 中的条件依赖于字典时挂起. [#6011](https://github.com/ClickHouse/ClickHouse/pull/6011) ([Vitaly Baranov](https://github.com/vitlibar)) - -#### 改进 {#improvement-6} - -- 在集群配置中允许无法解析的地址. 它们将被视为不可用并在每次连接尝试时尝试解决. 这对 Kubernetes 尤其有用. 这修复了 [#5714](https://github.com/ClickHouse/ClickHouse/issues/5714) [#5924](https://github.com/ClickHouse/ClickHouse/pull/5924) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 关闭空闲的 TCP 连接 (默认超时一小时). 这对于在每台服务器上有多个分布式表的大型集群尤其重要, 因为每台服务器都可能保持与其他所有服务器的连接池, 并且在查询并发峰值之后, 连接将停止. 这修复了 [#5879](https://github.com/ClickHouse/ClickHouse/issues/5879) [#5880](https://github.com/ClickHouse/ClickHouse/pull/5880) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `topK` 函数的质量更好. 如果新元素具有更大的权重, 则更改 SavingSpace 设置行为以删除最后一个元素. [#5833](https://github.com/ClickHouse/ClickHouse/issues/5833) [#5850](https://github.com/ClickHouse/ClickHouse/pull/5850) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 用于域的 URL 函数现在可以用于没有方案的不完整 URL. [#5725](https://github.com/ClickHouse/ClickHouse/pull/5725) ([alesapin](https://github.com/alesapin)) -- 将校验和添加到 `system.parts_columns` 表. [#5874](https://github.com/ClickHouse/ClickHouse/pull/5874) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -- 添加了 `Enum` 数据类型作为 `Enum` 或 `Enum 16` 的同义词. [#5886](https://github.com/ClickHouse/ClickHouse/pull/5886) ([dimarub2000](https://github.com/dimarub2000)) -- `T64` 编解码器的全位转置变体. 可以使用 `zstd` 实现更好的压缩. [#5742](https://github.com/ClickHouse/ClickHouse/pull/5742) ([Artem Zuikov](https://github.com/4ertus2)) -- `startsWith` 函数的条件现在可以使用主键. 这修复了 [#5310](https://github.com/ClickHouse/ClickHouse/issues/5310) and [#5882](https://github.com/ClickHouse/ClickHouse/issues/5882) [#5919](https://github.com/ClickHouse/ClickHouse/pull/5919) ([dimarub2000](https://github.com/dimarub2000)) -- 通过允许空数据库名称, 允许将 `clickhouse-copier` 与交叉复制集群拓扑一起使用. [#5745](https://github.com/ClickHouse/ClickHouse/pull/5745) ([nvartolomei](https://github.com/nvartolomei)) -- 在没有 `tzdata` 的系统 (例如裸 Docker 容器) 上使用 `UTC` 作为默认时区. 在此补丁之前,打印错误消息 `Could not determine local time zone` 并且服务器或客户端拒绝启动. [#5827](https://github.com/ClickHouse/ClickHouse/pull/5827) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 返回对函数 `quantileTiming` 中浮点参数的支持, 以实现向后兼容. [#5911](https://github.com/ClickHouse/ClickHouse/pull/5911) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在错误消息中显示哪个表缺少列. [#5768](https://github.com/ClickHouse/ClickHouse/pull/5768) ([Ivan](https://github.com/abyss7)) -- 禁止不同用户使用相同的 query_id 运行查询. [#5430](https://github.com/ClickHouse/ClickHouse/pull/5430) ([proller](https://github.com/proller)) -- 用于将指标发送到 Graphite 的更强大的代码。 即使在长时间的 `RENAME TABLE` 操作期间它也能工作. [#5875](https://github.com/ClickHouse/ClickHouse/pull/5875) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 当 ThreadPool 无法安排要执行的任务时, 将显示更多信息性错误消息. 这修复了 [#5305](https://github.com/ClickHouse/ClickHouse/issues/5305) [#5801](https://github.com/ClickHouse/ClickHouse/pull/5801) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 倒置ngramSearch更直观. [#5807](https://github.com/ClickHouse/ClickHouse/pull/5807) ([Danila Kutenin](https://github.com/danlark1)) -- 在 HDFS 引擎构建器中添加用户解析. [#5946](https://github.com/ClickHouse/ClickHouse/pull/5946) ([akonyaev90](https://github.com/akonyaev90)) -- 更新 `max_ast_elements parameter` 的默认值. [#5933](https://github.com/ClickHouse/ClickHouse/pull/5933) ([Artem Konovalov](https://github.com/izebit)) -- 添加了过时设置的概念. 可以使用过时的设置 `allow_experimental_low_cardinality_type` 没有效果. [0f15c01c6802f7ce1a1494c12c846be8c98944cd](https://github.com/ClickHouse/ClickHouse/commit/0f15c01c6802f7ce1a1494c12c846be8c98944cd) [Alexey Milovidov](https://github.com/alexey-milovidov) - -#### 性能改进 {#performance-improvement-4} - -- 增加从合并表中选择的流数量, 以实现更均匀的线程分布. 添加了设置 `max_streams_multiplier_for_merge_tables`. 这修复了 [#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [#5915](https://github.com/ClickHouse/ClickHouse/pull/5915) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-7} - -- 为客户端-服务器与不同版本的clickhouse交互添加向后兼容性测试. [#5868](https://github.com/ClickHouse/ClickHouse/pull/5868) ([alesapin](https://github.com/alesapin)) -- 在每个提交和拉取请求中测试覆盖率信息. [#5896](https://github.com/ClickHouse/ClickHouse/pull/5896) ([alesapin](https://github.com/alesapin)) -- 与 address sanitizer 合作以支持我们的自定义分配器 (`Arena` 和 `ArenaWithFreeLists`), 以便更好地调试 `use-after-free` 错误. [#5728](https://github.com/ClickHouse/ClickHouse/pull/5728) ([akuzm](https://github.com/akuzm)) -- 切换到 [LLVM libunwind implementation](https://github.com/llvm-mirror/libunwind) 以进行 C++ 异常处理和堆栈跟踪打印. [#4828](https://github.com/ClickHouse/ClickHouse/pull/4828) ([Nikita Lapkov](https://github.com/laplab)) -- 添加来自 -Weverything 的另外两个警告. [#5923](https://github.com/ClickHouse/ClickHouse/pull/5923) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 允许使用 Memory Sanitizer 构建 ClickHouse. [#3949](https://github.com/ClickHouse/ClickHouse/pull/3949) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了关于模糊测试中 `bitTest` 函数的 ubsan 报告. [#5943](https://github.com/ClickHouse/ClickHouse/pull/5943) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- Docker: 增加了初始化需要身份验证的 ClickHouse 实例的可能性. [#5727](https://github.com/ClickHouse/ClickHouse/pull/5727) ([Korviakov Andrey](https://github.com/shurshun)) -- 将 librdkafka 更新至 1.1.0 版. [#5872](https://github.com/ClickHouse/ClickHouse/pull/5872) ([Ivan](https://github.com/abyss7)) -- 为集成测试添加全局超时并在测试代码中禁用其中的一些. [#5741](https://github.com/ClickHouse/ClickHouse/pull/5741) ([alesapin](https://github.com/alesapin)) -- 修复一些 ThreadSanitizer 故障. [#5854](https://github.com/ClickHouse/ClickHouse/pull/5854) ([akuzm](https://github.com/akuzm)) -- `--no-undefined` 选项强制链接器在链接时检查所有外部名称是否存在. 在拆分构建模式下跟踪库之间的真实依赖关系非常有用. [#5855](https://github.com/ClickHouse/ClickHouse/pull/5855) ([Ivan](https://github.com/abyss7)) -- 添加了 [#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [#5914](https://github.com/ClickHouse/ClickHouse/pull/5914) ([alexey-milovidov](https://github.com/alexey-milovidov)) 的性能测试. -- 修复了与 gcc-7 的兼容性. [#5840](https://github.com/ClickHouse/ClickHouse/pull/5840) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了对 gcc-9 的支持. 这修复了 [#5717](https://github.com/ClickHouse/ClickHouse/issues/5717) [#5774](https://github.com/ClickHouse/ClickHouse/pull/5774) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 libunwind 可以错误链接时的错误. [#5948](https://github.com/ClickHouse/ClickHouse/pull/5948) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 PVS-Studio 发现的一些警告. [#5921](https://github.com/ClickHouse/ClickHouse/pull/5921) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了对 `clang-tidy` 静态分析器的初始支持. [#5806](https://github.com/ClickHouse/ClickHouse/pull/5806) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将 BSD/Linux 字节序宏 (`be64toh` 和 `htobe64`) 转换为 Mac OS X 等价物. [#5785](https://github.com/ClickHouse/ClickHouse/pull/5785) ([Fu Chen](https://github.com/fredchenbj)) -- 改进的集成测试指南. [#5796](https://github.com/ClickHouse/ClickHouse/pull/5796) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 修复在macosx + gcc9的构建. [#5822](https://github.com/ClickHouse/ClickHouse/pull/5822) ([filimonov](https://github.com/filimonov)) -- 修复一个难以发现的拼写错误: aggreGate -\>aggregate. [#5753](https://github.com/ClickHouse/ClickHouse/pull/5753) ([akuzm](https://github.com/akuzm)) -- 修复 freebsd 构建. [#5760](https://github.com/ClickHouse/ClickHouse/pull/5760) ([proller](https://github.com/proller)) -- 将实验性 YouTube 频道的链接添加到网站 [#5845](https://github.com/ClickHouse/ClickHouse/pull/5845) ([Ivan Blinkov](https://github.com/blinkov)) -- CMake: 添加覆盖标志选项: WITH_COVERAGE [#5776](https://github.com/ClickHouse/ClickHouse/pull/5776) ([proller](https://github.com/proller)) -- 修复一些内联 PODArray 的初始大小. [#5787](https://github.com/ClickHouse/ClickHouse/pull/5787) ([akuzm](https://github.com/akuzm)) -- clickhouse-server.postinst: 修复 centos 6 的操作系统检测. [#5788](https://github.com/ClickHouse/ClickHouse/pull/5788) ([proller](https://github.com/proller)) -- 添加了 Arch linux 包生成. [#5719](https://github.com/ClickHouse/ClickHouse/pull/5719) ([Vladimir Chebotarev](https://github.com/excitoon)) -- 通过 libs (dbms) 拆分 Common/config.h. [#5715](https://github.com/ClickHouse/ClickHouse/pull/5715) ([proller](https://github.com/proller)) -- 修复 `Arcadia` 构建平台 [#5795](https://github.com/ClickHouse/ClickHouse/pull/5795) ([proller](https://github.com/proller)) -- 修复非常规构建 (gcc9, 无子模块). [#5792](https://github.com/ClickHouse/ClickHouse/pull/5792) ([proller](https://github.com/proller)) -- 在 unalignedStore 中需要显式类型, 因为它被证明容易出错. [#5791](https://github.com/ClickHouse/ClickHouse/pull/5791) ([akuzm](https://github.com/akuzm)) -- 修复 MacOS 构建. [#5830](https://github.com/ClickHouse/ClickHouse/pull/5830) ([filimonov](https://github.com/filimonov)) -- 根据此处的要求, 关于具有更大数据集的新 JIT 功能的性能测试 [#5263](https://github.com/ClickHouse/ClickHouse/issues/5263) [#5887](https://github.com/ClickHouse/ClickHouse/pull/5887) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 在压力测试中运行状态测试 [12693e568722f11e19859742f56428455501fd2a](https://github.com/ClickHouse/ClickHouse/commit/12693e568722f11e19859742f56428455501fd2a) ([alesapin](https://github.com/alesapin)) - -#### 向后不兼容变更 {#backward-incompatible-change-7} - -- `Kafka` 在这个版本中被损坏. -- 默认情况下为新的 `MergeTree` 表启用 `adaptive_index_granularity` = 10MB. 如果您在 19.11+ 版本上创建了新的 MergeTree 表, 则无法降级到 19.6 之前的版本. [#5628](https://github.com/ClickHouse/ClickHouse/pull/5628) ([alesapin](https://github.com/alesapin)) -- 删除了 Yandex.Metrica 使用的过时的未记录的嵌入式词典. `OSIn`、`SEIn`、`OSToRoot`、`SEToRoot`、`OSHierarchy`、`SEHierarchy` 函数不再可用. 如果您正在使用这些功能, 请发送电子邮件至 clickhouse-feedback@yandex-team.com. 注意: 最后一刻我们决定暂时保留这些功能. [#5780](https://github.com/ClickHouse/ClickHouse/pull/5780) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## ClickHouse 版本 19.10 {#clickhouse-release-19-10} - -### ClickHouse 版本 19.10.1.5, 2019-07-12 {#clickhouse-release-19-10-1-5-2019-07-12} - -#### 新特征 {#new-feature-7} - -- 添加新的列编解码器:`T64`. 适用于 (U)IntX/EnumX/Data(Time)/DecimalX 列. 它应该适用于具有恒定值或小范围值的列. 编解码器本身允许放大或缩小数据类型而无需重新压缩. [#5557](https://github.com/ClickHouse/ClickHouse/pull/5557) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加数据库引擎`MySQL`, 允许查看远程MySQL服务器中的所有表. [#5599](https://github.com/ClickHouse/ClickHouse/pull/5599) ([Winter Zhang](https://github.com/zhang2014)) -- `bitmapContains` 实现. 如果第二个位图包含一个元素, 它比 `bitmapHasAny` 快 2 倍. [#5535](https://github.com/ClickHouse/ClickHouse/pull/5535) ([Zhichang Yu](https://github.com/yuzhichang)) -- 支持 `crc32` 函数 (行为与 MySQL 或 PHP 完全相同). 如果您需要哈希函数, 请不要使用它. [#5661](https://github.com/ClickHouse/ClickHouse/pull/5661) ([Remen Ivan](https://github.com/BHYCHIK)) -- 实现了 `SYSTEM START/STOP DISTRIBUTED SENDS` 查询以控制对 `Distributed` 表的异步插入. [#4935](https://github.com/ClickHouse/ClickHouse/pull/4935) ([Winter Zhang](https://github.com/zhang2014)) - -#### Bug 修复 {#bug-fix-22} - -- 执行突变时忽略查询执行限制和合并限制的最大部分大小. [#5659](https://github.com/ClickHouse/ClickHouse/pull/5659) ([Anton Popov](https://github.com/CurtizJ)) -- 修复可能导致正常块重复数据删除 (极其罕见) 和插入重复块 (更常见) 的错误. [#5549](https://github.com/ClickHouse/ClickHouse/pull/5549) ([alesapin](https://github.com/alesapin)) -- 修复函数 `arrayEnumerateUniqRanked` 的空数组参数. [#5559](https://github.com/ClickHouse/ClickHouse/pull/5559) ([proller](https://github.com/proller)) -- 不要在无意轮询任何消息的情况下订阅 Kafka 主题. [#5698](https://github.com/ClickHouse/ClickHouse/pull/5698) ([Ivan](https://github.com/abyss7)) -- 使设置 `join_use_nulls` 对不能在 Nullable 内的类型无效. [#5700](https://github.com/ClickHouse/ClickHouse/pull/5700) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复了 `Incorrect size of index granularity` 错误. [#5720](https://github.com/ClickHouse/ClickHouse/pull/5720) ([coraxster](https://github.com/coraxster)) -- 修复 Float 到 Decimal 转换溢出. [#5607](https://github.com/ClickHouse/ClickHouse/pull/5607) ([coraxster](https://github.com/coraxster)) -- 当 `WriteBufferFromHDFS` 的析构函数被调用时刷新缓冲区. 这修复了写入`HDFS`. [#5684](https://github.com/ClickHouse/ClickHouse/pull/5684) ([Xindong Peng](https://github.com/eejoin)) - -#### 改进 {#improvement-7} - -- 启用设置 `input_format_defaults_for_omited_fields` 时,将 `CSV` 中的空单元格视为默认值. [#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) -- 非阻塞加载外部字典. [#5567](https://github.com/ClickHouse/ClickHouse/pull/5567) ([Vitaly Baranov](https://github.com/vitlibar)) -- 可以根据设置为已建立的连接动态更改网络超时. [#4558](https://github.com/ClickHouse/ClickHouse/pull/4558) ([Konstantin Podshumok](https://github.com/podshumok)) -- 使用 `public_suffix_list` 作为函数 `firstSignificantSubdomain`、`cutToFirstSignificantSubdomain`. 它使用由`gperf` 生成的完美哈希表和从文件生成的列表: https://publicsuffix.org/list/public_suffix_list.dat. (例如, 现在我们认为域 `ac.uk` 不重要). [#5030](https://github.com/ClickHouse/ClickHouse/pull/5030) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 在系统表中采用了 `IPv6` 数据类型; `system.processes` 和 `system.query_log` 中的统一客户端信息列. [#5640](https://github.com/ClickHouse/ClickHouse/pull/5640) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用会话与 MySQL 兼容协议进行连接. #5476 [#5646](https://github.com/ClickHouse/ClickHouse/pull/5646) ([Yuriy Baranov](https://github.com/yurriy)) -- 支持更多 `ALTER` 查询 `ON CLUSTER`. [#5593](https://github.com/ClickHouse/ClickHouse/pull/5593) [#5613](https://github.com/ClickHouse/ClickHouse/pull/5613) ([sundyli](https://github.com/sundy-li)) -- 支持 `clickhouse-local` 配置文件中的 `` 部分. [#5540](https://github.com/ClickHouse/ClickHouse/pull/5540) ([proller](https://github.com/proller)) -- 允许在 `clickhouse-local` 中使用 `remote` 表函数运行查询. [#5627](https://github.com/ClickHouse/ClickHouse/pull/5627) ([proller](https://github.com/proller)) - -#### 性能改进 {#performance-improvement-5} - -- 添加在 MergeTree 列末尾写入最终标记的可能性. 它允许避免对超出表数据范围的键进行无用的读取. 仅在使用自适应索引粒度时才启用. [#5624](https://github.com/ClickHouse/ClickHouse/pull/5624) ([alesapin](https://github.com/alesapin)) -- 通过减少 `stat` 系统调用的数量, 改进了 MergeTree 表在非常慢的文件系统上的性能. [#5648](https://github.com/ClickHouse/ClickHouse/pull/5648) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了从版本 19.6 中引入的 MergeTree 表读取时的性能下降. 修复 #5631. [#5633](https://github.com/ClickHouse/ClickHouse/pull/5633) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-8} - -- 实现了 `TestKeeper` 作为用于测试的 ZooKeeper 接口的实现. [#5643](https://github.com/ClickHouse/ClickHouse/pull/5643) ([alexey-milovidov](https://github.com/alexey-milovidov)) ([levushkin aleksej](https://github.com/alexey-milovidov)) -- 现在开始, `.sql` 测试可以由服务器独立运行, 与随机数据库并行运行. 它允许更快地运行它们, 使用自定义服务器配置添加新测试, 并确保不同的测试不会相互影响. [#5554](https://github.com/ClickHouse/ClickHouse/pull/5554) ([Ivan](https://github.com/abyss7)) -- 从性能测试中删除 `` 和 ``. [#5672](https://github.com/ClickHouse/ClickHouse/pull/5672) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复了 `Pretty` 格式的 `select_format` 性能测试. [#5642](https://github.com/ClickHouse/ClickHouse/pull/5642) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## ClickHouse 版本 19.9 {#clickhouse-release-19-9} - -### ClickHouse 版本 19.9.3.31, 2019-07-05 {#clickhouse-release-19-9-3-31-2019-07-05} - -#### Bug 修复 {#bug-fix-23} - -- 修复 Delta 编解码器中的段错误, 该错误会影响值小于 32 位大小的列. 该错误导致随机内存损坏. [#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) -- 修复使用低基数列检查零件时的罕见错误. [#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) -- 修复 TTL 合并中的段错误与块中的非物理列. [#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) -- 修复低优先级查询的潜在无限睡眠. [#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 ClickHouse 如何将默认时区确定为 UCT 而不是 UTC. [#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复关于在领导者副本之前在跟随者副本上执行分布式 DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER 查询的错误. 现在它们将直接在领导者副本上执行. [#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) -- 修复竞争条件, 导致某些查询在 SYSTEM FLUSH LOGS 查询后可能不会立即出现在 query_log 中. [#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) -- 为 `evalMLModel` 函数添加了对常量参数的缺失支持. [#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.9.2.4, 2019-06-24 {#clickhouse-release-19-9-2-4-2019-06-24} - -#### 新特征 {#new-feature-8} - -- 在 `system.parts` 表中打印有关冻结部件的信息. [#5471](https://github.com/ClickHouse/ClickHouse/pull/5471) ([proller](https://github.com/proller)) -- 如果未在参数中设置, 则在 tty 上启动 clickhouse-client 时询问客户端密码. [#5092](https://github.com/ClickHouse/ClickHouse/pull/5092) ([proller](https://github.com/proller)) -- 为 Decimal 类型实现 `dictGet` 和 `dictGetOrDefault` 函数. [#5394](https://github.com/ClickHouse/ClickHouse/pull/5394) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 改进 {#improvement-8} - -- Debian init: 添加服务停止超时. [#5522](https://github.com/ClickHouse/ClickHouse/pull/5522) ([proller](https://github.com/proller)) -- 添加默认禁止设置为 LowCardinality 创建具有可疑类型的表. [#5448](https://github.com/ClickHouse/ClickHouse/pull/5448) ([Olga Khvostikova](https://github.com/stavrolia)) -- 回归函数在不用作函数 `evalMLMethod` 中的状态时返回模型权重. [#5411](https://github.com/ClickHouse/ClickHouse/pull/5411) ([Quid37](https://github.com/Quid37)) -- 重命名和改进回归方法. [#5492](https://github.com/ClickHouse/ClickHouse/pull/5492) ([Quid37](https://github.com/Quid37)) -- 字符串搜索器的界面更清晰. [#5586](https://github.com/ClickHouse/ClickHouse/pull/5586) ([Danila Kutenin](https://github.com/danlark1)) - -#### Bug 修复 {#bug-fix-24} - -- 修复 Kafka 中潜在的数据丢失问题. [#5445](https://github.com/ClickHouse/ClickHouse/pull/5445) ([Ivan](https://github.com/abyss7)) -- 使用零列调用时以 `PrettySpace` 格式修复潜在的无限循环. [#5560](https://github.com/ClickHouse/ClickHouse/pull/5560) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复了线性模型中的 UInt32 溢出错误. 允许 eval ML 模型用于非常量模型参数. [#5516](https://github.com/ClickHouse/ClickHouse/pull/5516) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- `ALTER TABLE ... DROP INDEX IF EXISTS ...` 如果提供的索引不存在,则不应引发异常. [#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) -- 在标量子查询中使用 `bitmapHasAny` 修复段错误. [#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) -- 修复了复制连接池不重试解析主机时的错误, 即使 DNS 缓存已删除. [#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) -- 修复了 ReplicatedMergeTree 上的 `ALTER ... MODIFY TTL` . [#5539](https://github.com/ClickHouse/ClickHouse/pull/5539) ([Anton Popov](https://github.com/CurtizJ)) -- 使用 MATERIALIZED 列将 INSERT 修复到分布式表中. [#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) -- 在截断 Join 存储时修复错误分配 [#5437](https://github.com/ClickHouse/ClickHouse/pull/5437) ([TCeason](https://github.com/TCeason)) -- 在包 tzdata 的最新版本中, 一些文件现在是符号链接. 当前检测默认时区的机制被破坏, 并为某些时区提供了错误的名称. 现在至少我们将时区名称强制为 TZ 的内容 (如果提供). [#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) -- 使用 MultiVolnitsky 搜索器修复一些非常罕见的情况, 当总针数总和至少为 16KB 时. 该算法遗漏或覆盖了之前的结果, 这可能导致 `multiSearchAny` 的结果不正确. [#5588](https://github.com/ClickHouse/ClickHouse/pull/5588) ([Danila Kutenin](https://github.com/danlark1)) -- 修复 ExternalData 请求的设置无法使用 ClickHouse 设置时的问题. 此外, 目前无法使用设置 `date_time_input_format` 和 `low_cardinality_allow_in_native_format`, 因为名称不明确 (在外部数据中它可以解释为表格式, 在查询中它可以是一个设置). [#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila Kutenin](https://github.com/danlark1)) -- 修复了仅从 FS 中删除部件而不将它们从 Zookeeper 中删除的错误. [#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) -- 从 MySQL 协议中删除调试日志记录. [#5478](https://github.com/ClickHouse/ClickHouse/pull/5478) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在 DDL 查询处理过程中跳过 ZNONODE. [#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) -- 修复混合 `UNION ALL` 结果列类型. 存在数据和结果列的列类型不一致的情况. [#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) -- 在 dictGetT 函数中对错误的整数抛出异常而不是崩溃. [#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 `system.dictionaries` 表中散列字典的错误 element_count 和 load_factor. [#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-9} - -- 修复了没有 `Brotli` HTTP 压缩支持的构建 (`ENABLE_BROTLI=OFF` cmake 变量). [#5521](https://github.com/ClickHouse/ClickHouse/pull/5521) ([Anton Yuzhaninov](https://github.com/citrin)) -- 包含 roaring.h 作为 roaring/roaring.h. [#5523](https://github.com/ClickHouse/ClickHouse/pull/5523) ([Orivej Desh](https://github.com/orivej)) -- 修复 hyperscan 中的 gcc9 警告 (#line directive is evil!). [#5546](https://github.com/ClickHouse/ClickHouse/pull/5546) ([Danila Kutenin](https://github.com/danlark1)) -- 修复使用 gcc-9 编译时的所有警告. 修复一些贡献问题. 修复 gcc9 ICE 并提交给 bugzilla. [#5498](https://github.com/ClickHouse/ClickHouse/pull/5498) ([Danila Kutenin](https://github.com/danlark1)) -- 修复了与 lld 的链接. [#5477](https://github.com/ClickHouse/ClickHouse/pull/5477) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 删除字典中未使用的专业化. [#5452](https://github.com/ClickHouse/ClickHouse/pull/5452) ([Artem Zuikov](https://github.com/4ertus2)) -- 针对不同类型文件的格式化和解析表的改进性能测试. [#5497](https://github.com/ClickHouse/ClickHouse/pull/5497) ([Olga Khvostikova](https://github.com/stavrolia)) -- 修复并行测试运行. [#5506](https://github.com/ClickHouse/ClickHouse/pull/5506) ([proller](https://github.com/proller)) -- Docker: 使用 clickhouse-test 中的配置. [#5531](https://github.com/ClickHouse/ClickHouse/pull/5531) ([proller](https://github.com/proller)) -- 修复 FreeBSD 编译. [#5447](https://github.com/ClickHouse/ClickHouse/pull/5447) ([proller](https://github.com/proller)) -- 升级 boost 到 1.70. [#5570](https://github.com/ClickHouse/ClickHouse/pull/5570) ([proller](https://github.com/proller)) -- 修复构建clickhouse作为子模块的问题. [#5574](https://github.com/ClickHouse/ClickHouse/pull/5574) ([proller](https://github.com/proller)) -- 改进 JSONExtract 性能测试. [#5444](https://github.com/ClickHouse/ClickHouse/pull/5444) ([Vitaly Baranov](https://github.com/vitlibar)) - -## ClickHouse 版本 19.8 {#clickhouse-release-19-8} - -### ClickHouse 版本 19.8.3.8, 2019-06-11 {#clickhouse-release-19-8-3-8-2019-06-11} - -#### 新特征 {#new-features} - -- 添加了使用 JSON 的函数. [#4686](https://github.com/ClickHouse/ClickHouse/pull/4686) ([hcz](https://github.com/hczhcz)) [#5124](https://github.com/ClickHouse/ClickHouse/pull/5124). ([Vitaly Baranov](https://github.com/vitlibar)) -- 添加一个函数 basename, 其行为类似于 basename 函数, 该函数存在于很多语言中 (python 中的 `os.path.basename`,PHP 中的 `basename` 等......). 使用类 UNIX 路径或 Windows 路径. [#5136](https://github.com/ClickHouse/ClickHouse/pull/5136) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 添加了 `LIMIT n, m BY` 或 `LIMIT m OFFSET n BY` 语法来为LIMIT BY 子句设置n 的偏移量. [#5138](https://github.com/ClickHouse/ClickHouse/pull/5138) ([Anton Popov](https://github.com/CurtizJ)) -- 添加了新的数据类型 `SimpleAggregateFunction`, 它允许在 `AggregatingMergeTree` 中具有轻聚合的列. 这只能与简单的函数一起使用, 如 `any`、`anyLast`、`sum`、`min`、`max`. [#4629](https://github.com/ClickHouse/ClickHouse/pull/4629) ([Boris Granveaud](https://github.com/bgranvea)) -- 在函数 `ngramDistance` 中添加了对非常量参数的支持. [#5198](https://github.com/ClickHouse/ClickHouse/pull/5198) ([Danila Kutenin](https://github.com/danlark1)) -- 添加函数 `skewPop`、`skewSamp`、`kurtPop` 和 `kurtSamp` 分别计算序列偏度、样本偏度、峰度和样本峰度. [#5200](https://github.com/ClickHouse/ClickHouse/pull/5200) ([hcz](https://github.com/hczhcz)) -- 支持 `MaterializeView` 存储的重命名操作. [#5209](https://github.com/ClickHouse/ClickHouse/pull/5209) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 添加了允许使用 MySQL 客户端连接到 ClickHouse 的服务器. [#4715](https://github.com/ClickHouse/ClickHouse/pull/4715) ([Yuriy Baranov](https://github.com/yurriy)) -- 添加 `toDecimal*OrZero` 和 `toDecimal*OrNull` 函数. [#5291](https://github.com/ClickHouse/ClickHouse/pull/5291) ([Artem Zuikov](https://github.com/4ertus2)) -- 支持函数中的十进制类型: `quantile`、`quantiles`、`median`、`quantileExactWeighted`、`quantilesExactWeighted`、`medianExactWeighted`. [#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加了 `toValidUTF8` 函数,用替换字符 � (U+FFFD) 替换所有无效的 UTF-8 字符. [#5322](https://github.com/ClickHouse/ClickHouse/pull/5322) ([Danila Kutenin](https://github.com/danlark1)) -- 添加了 `format` 功能. 使用参数中列出的字符串格式化常量模式 (简化的 Python 格式模式). [#5330](https://github.com/ClickHouse/ClickHouse/pull/5330) ([Danila Kutenin](https://github.com/danlark1)) -- 添加了 `system.detached_parts` 表, 其中包含有关 `MergeTree` 表的分离部分的信息. [#5353](https://github.com/ClickHouse/ClickHouse/pull/5353) ([akuzm](https://github.com/akuzm)) -- 添加了 `ngramSearch` 函数来计算needle和haystack之间的非对称差异. [#5418](https://github.com/ClickHouse/ClickHouse/pull/5418)[#5422](https://github.com/ClickHouse/ClickHouse/pull/5422) ([Danila Kutenin](https://github.com/danlark1)) -- 使用聚合函数接口实现基本机器学习方法 (随机线性回归和逻辑回归). 有不同的更新模型权重的策略 (简单梯度下降、动量方法、Nesterov 方法). 还支持自定义尺寸的小批量. [#4943](https://github.com/ClickHouse/ClickHouse/pull/4943) ([Quid37](https://github.com/Quid37)) -- `geohashEncode` 和 `geohashDecode` 函数的实现. [#5003](https://github.com/ClickHouse/ClickHouse/pull/5003) ([Vasily Nemkov](https://github.com/Enmk)) -- 添加聚合函数`timeSeriesGroupSum`, 可以聚合采样时间戳不对齐的不同时间序列. 它将在两个样本时间戳之间使用线性插值, 然后将时间序列相加. 添加聚合函数 `timeSeriesGroupRateSum` , 它计算时间序列的速率, 然后将速率相加. [#4542](https://github.com/ClickHouse/ClickHouse/pull/4542) ([Yangkuan Liu](https://github.com/LiuYangkuan)) -- 添加了函数 `IPv4CIDRtoIPv4Range` 和 `IPv6CIDRtoIPv6Range` 以使用 CIDR 计算子网中 IP 的下限和上限. [#5095](https://github.com/ClickHouse/ClickHouse/pull/5095) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 当我们使用启用设置 `send_progress_in_http_headers` 的 HTTP 发送查询时, 添加 X-ClickHouse-Summary 标头. 返回 X-ClickHouse-Progress 的常用信息, 附加信息, 如查询中插入了多少行和字节数. [#5116](https://github.com/ClickHouse/ClickHouse/pull/5116) ([Guillaume Tassery](https://github.com/YiuRULE)) - -#### 改进 {#improvements} - -- 为 MergeTree 系列表(默认值: 100 000) 添加了 `max_parts_in_total` 设置, 以防止不安全的分区键规范 #5166. [#5171](https://github.com/ClickHouse/ClickHouse/pull/5171) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `clickhouse-obfuscator`: 通过将初始种子与列名而不是列位置相结合来为各个列派生种子. 这旨在转换具有多个相关表的数据集, 以便转换后表保持可连接. [#5178](https://github.com/ClickHouse/ClickHouse/pull/5178) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了函数`JSONExtractRaw`、`JSONExtractKeyAndValues`. 将函数 `jsonExtract` 重命名为 `JSONExtract`. 当出现问题时, 这些函数返回相应的值, 而不是 `NULL` . 修改了函数`JSONExtract`, 现在它从它的最后一个参数中获取返回类型并且不注入可空值. 在 AVX2 指令不可用的情况下, 实现了对 RapidJSON 的回退. Simdjson 库更新到新版本. [#5235](https://github.com/ClickHouse/ClickHouse/pull/5235) ([Vitaly Baranov](https://github.com/vitlibar)) -- 现在 `if` 和 `multiIf` 函数不依赖于条件的 `Nullable` , 而是依赖于分支的sql兼容性. [#5238](https://github.com/ClickHouse/ClickHouse/pull/5238) ([Jian Wu](https://github.com/janplus)) -- `In` 谓词现在从 `Null` 输入生成 `Null` 结果, 就像 `Equal` 函数一样. [#5152](https://github.com/ClickHouse/ClickHouse/pull/5152) ([Jian Wu](https://github.com/janplus)) -- 查来自 Kafka 的每 (flush_interval / poll_timeout) 行数的时间限制. 这允许更频繁地中断 Kafka 消费者的读取并检查顶级流的时间限制. [#5249](https://github.com/ClickHouse/ClickHouse/pull/5249) ([Ivan](https://github.com/abyss7)) -- 将 rdkafka 与捆绑的 SASL 联系起来. 它应该允许使用 SASL SCRAM 身份验证. [#5253](https://github.com/ClickHouse/ClickHouse/pull/5253) ([Ivan](https://github.com/abyss7)) -- 用于 ALL JOINS 的 RowRefList 的批处理版本. [#5267](https://github.com/ClickHouse/ClickHouse/pull/5267) ([Artem Zuikov](https://github.com/4ertus2)) -- clickhouse-server: 提供更多信息的侦听错误消息. [#5268](https://github.com/ClickHouse/ClickHouse/pull/5268) ([proller](https://github.com/proller)) -- 支持 clickhouse-copier 中的字典用于 `` 中的函数. [#5270](https://github.com/ClickHouse/ClickHouse/pull/5270) ([proller](https://github.com/proller)) -- 添加新设置 `kafka_commit_every_batch` 以规范 Kafka 提交策略. - 它允许设置提交模式: 在处理每批消息之后, 或在整个块写入存储之后. 这是在某些极端情况下丢失某些消息或读取它们两次之间的权衡. [#5308](https://github.com/ClickHouse/ClickHouse/pull/5308) ([Ivan](https://github.com/abyss7)) -- 使 `windowFunnel` 支持其他无符号整数类型. [#5320](https://github.com/ClickHouse/ClickHouse/pull/5320) ([sundyli](https://github.com/sundy-li)) -- 允许在 Merge 引擎中隐藏虚拟列 `_table`. [#5325](https://github.com/ClickHouse/ClickHouse/pull/5325) ([Ivan](https://github.com/abyss7)) -- 使 `sequenceMatch` 聚合函数支持其他无符号整数类型. [#5339](https://github.com/ClickHouse/ClickHouse/pull/5339) ([sundyli](https://github.com/sundy-li)) -- 如果校验和不匹配最有可能是由硬件故障引起的, 则会提供更好的错误消息. [#5355](https://github.com/ClickHouse/ClickHouse/pull/5355) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 检查底层表是否支持对 `StorageMerge` 的采样 [#5366](https://github.com/ClickHouse/ClickHouse/pull/5366) ([Ivan](https://github.com/abyss7)) -- 在外部字典中使用后关闭 MySQL 连接. 它与问题 #893 有关. [#5395](https://github.com/ClickHouse/ClickHouse/pull/5395) ([Clément Rodriguez](https://github.com/clemrodriguez)) -- MySQL Wire 协议的改进. 将格式名称更改为 MySQLWire. 使用 RAII 调用 RSA_free. 如果无法创建上下文, 则禁用 SSL. [#5419](https://github.com/ClickHouse/ClickHouse/pull/5419) ([Yuriy Baranov](https://github.com/yurriy)) -- 允许使用不可访问的历史文件运行 (只读,没有磁盘空间,文件是目录,...). [#5431](https://github.com/ClickHouse/ClickHouse/pull/5431) ([proller](https://github.com/proller)) -- 尊重异步 INSERT 中的查询设置到分布式表. [#4936](https://github.com/ClickHouse/ClickHouse/pull/4936) ([TCeason](https://github.com/TCeason)) -- 将函数 `leastSqr` 重命名为 `simpleLinearRegression`,将 `LinearRegression` 重命名为 `linearRegression`,将 `LogisticRegression` 重命名为 `logisticRegression`. [#5391](https://github.com/ClickHouse/ClickHouse/pull/5391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### 性能改进 {#performance-improvements} - -- 在 ALTER MODIFY 查询中并行处理部分非复制 MergeTree 表. [#4639](https://github.com/ClickHouse/ClickHouse/pull/4639) ([Ivan Kush](https://github.com/IvanKush)) -- 正则表达式提取优化. [#5193](https://github.com/ClickHouse/ClickHouse/pull/5193) [#5191](https://github.com/ClickHouse/ClickHouse/pull/5191) ([Danila Kutenin](https://github.com/danlark1)) -- 如果仅在连接部分中使用, 请勿添加右连接键列以连接结果. [#5260](https://github.com/ClickHouse/ClickHouse/pull/5260) ([Artem Zuikov](https://github.com/4ertus2)) -- 在第一个空响应后冻结 Kafka 缓冲区. 它避免了多次调用 `ReadBuffer::next()` 以获得一些行解析流中的空结果. [#5283](https://github.com/ClickHouse/ClickHouse/pull/5283) ([Ivan](https://github.com/abyss7)) -- 多参数的 `concat` 函数优化. [#5357](https://github.com/ClickHouse/ClickHouse/pull/5357) ([Danila Kutenin](https://github.com/danlark1)) -- 查询优化. 允许在将逗号/交叉连接重写为内部语句时下推 IN 语句. [#5396](https://github.com/ClickHouse/ClickHouse/pull/5396) ([Artem Zuikov](https://github.com/4ertus2)) -- 用参考文件1升级您的LZ4实现以获得更快的解压. [#5070](https://github.com/ClickHouse/ClickHouse/pull/5070) ([Danila Kutenin](https://github.com/danlark1)) -- 实现了 MSD 基数排序 (基于 kxsort) 和部分排序. [#5129](https://github.com/ClickHouse/ClickHouse/pull/5129) ([Evgenii Pravda](https://github.com/kvinty)) - -#### Bug 修复 {#bug-fixes} - -- 修复推送需要加入的列. [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) -- 修复了当 ClickHouse 由 systemd 运行时, 命令 `sudo service clickhouse-server forcerestart` 没有按预期工作的问题. [#5204](https://github.com/ClickHouse/ClickHouse/pull/5204) ([proller](https://github.com/proller)) -- 复 DataPartsExchange 中的 http 错误代码 (9009 端口上的服务器间 http 服务器始终返回代码 200, 即使出现错误). [#5216](https://github.com/ClickHouse/ClickHouse/pull/5216) ([proller](https://github.com/proller)) -- 修复长度超过 MAX_SMALL_STRING_SIZE 的字符串的 SimpleAggregateFunction [#5311](https://github.com/ClickHouse/ClickHouse/pull/5311) ([Azat Khuzhin](https://github.com/azat)) -- 修复 IN 中 `Decimal` 到 `Nullable(Decimal)` 的转换错误. 支持其他十进制到十进制的转换 (包括不同的尺度). [#5350](https://github.com/ClickHouse/ClickHouse/pull/5350) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了 simdjson 库中的 FPU 破坏导致 `uniqHLL` 和 `uniqCombined` 聚合函数和数学函数 (如 `log`) 计算错误的问题. [#5354](https://github.com/ClickHouse/ClickHouse/pull/5354) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 JSON 函数中混合常量/非常量情况的处理. [#5435](https://github.com/ClickHouse/ClickHouse/pull/5435) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复 `retention` 功能. 现在在一行数据中满足的所有条件都添加到数据状态中. [#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) -- 用小数修复 `quantileExact` 的结果类型. [#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 文档 {#documentation} - -- 将 `CollapsingMergeTree` 的文档翻译成中文. [#5168](https://github.com/ClickHouse/ClickHouse/pull/5168) ([张风啸](https://github.com/AlexZFX)) -- 将一些关于表引擎的文档翻译成中文. [#5134](https://github.com/ClickHouse/ClickHouse/pull/5134) [#5328](https://github.com/ClickHouse/ClickHouse/pull/5328) ([never lee](https://github.com/neverlee)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements} - -- 复一些显示可能在免费后使用的清除报告. [#5139](https://github.com/ClickHouse/ClickHouse/pull/5139) [#5143](https://github.com/ClickHouse/ClickHouse/pull/5143) [#5393](https://github.com/ClickHouse/ClickHouse/pull/5393) ([Ivan](https://github.com/abyss7)) -- 为方便起见, 将性能测试移出单独的目录. [#5158](https://github.com/ClickHouse/ClickHouse/pull/5158) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复不正确的性能测试. [#5255](https://github.com/ClickHouse/ClickHouse/pull/5255) ([alesapin](https://github.com/alesapin)) -- 增加了一个工具来计算由位转换引起的校验和来调试硬件问题. [#5334](https://github.com/ClickHouse/ClickHouse/pull/5334) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使运行脚本更有用. [#5340](https://github.com/ClickHouse/ClickHouse/pull/5340)[#5360](https://github.com/ClickHouse/ClickHouse/pull/5360) ([filimonov](https://github.com/filimonov)) -- 添加如何编写性能测试的小说明. [#5408](https://github.com/ClickHouse/ClickHouse/pull/5408) ([alesapin](https://github.com/alesapin)) -- 添加在性能测试中的创建、填充和删除查询中进行替换的能力. [#5367](https://github.com/ClickHouse/ClickHouse/pull/5367) ([Olga Khvostikova](https://github.com/stavrolia)) - -## ClickHouse 版本 19.7 {#clickhouse-release-19-7} - -### ClickHouse 版本 19.7.5.29, 2019-07-05 {#clickhouse-release-19-7-5-29-2019-07-05} - -#### Bug 修复 {#bug-fix-25} - -- 使用 JOIN 修复某些查询中的性能回归. [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) - -### ClickHouse 版本 19.7.5.27, 2019-06-09 {#clickhouse-release-19-7-5-27-2019-06-09} - -#### 新特征 {#new-features-1} - -- 添加位图相关函数 `bitmapHasAny` 和 `bitmapHasAll` 类似于数组的 `hasAny` 和 `hasAll` 函数. [#5279](https://github.com/ClickHouse/ClickHouse/pull/5279) ([Sergi Vladykin](https://github.com/svladykin)) - -#### Bug 修复 {#bug-fixes-1} - -- 使用 Null 值修复 `minmax` INDEX 上的段错误. [#5246](https://github.com/ClickHouse/ClickHouse/pull/5246) ([Nikita Vasilev](https://github.com/nikvas0)) -- 将 LIMIT BY 中的所有输入列标记为所需输出。 它修复了某些分布式查询中的 `Not found column` 错误. [#5407](https://github.com/ClickHouse/ClickHouse/pull/5407) ([Constantin S. Pan](https://github.com/kvap)) -- 在带有 DEFAULT 的列上修复 `SELECT .. PREWHERE` 中的 `Column '0' already exists` 错误. [#5397](https://github.com/ClickHouse/ClickHouse/pull/5397) ([proller](https://github.com/proller)) -- 修复 `ReplicatedMergeTree` 上的 `ALTER MODIFY TTL` 查询. [#5539](https://github.com/ClickHouse/ClickHouse/pull/5539/commits) ([Anton Popov](https://github.com/CurtizJ)) -- 当 Kafka 消费者启动失败时不要让服务器崩溃. [#5285](https://github.com/ClickHouse/ClickHouse/pull/5285) ([Ivan](https://github.com/abyss7)) -- 修复位图函数产生错误结果. [#5359](https://github.com/ClickHouse/ClickHouse/pull/5359) ([Andy Yang](https://github.com/andyyzh)) -- 修复哈希字典的 element_count (不包括重复项). [#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) -- 使用环境变量 TZ 的内容作为时区的名称. 在某些情况下有助于正确检测默认时区. [#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) -- 不要尝试在 `dictGetT` 函数中转换整数, 因为它不能正常工作. 改为抛出异常. [#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 ExternalData HTTP 请求中的设置. [#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila - Kutenin](https://github.com/danlark1)) -- 修复了仅从 FS 中删除部件而不将它们从 Zookeeper 中删除的错误. [#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) -- 修复了 `bitmapHasAny` 函数中的分段错误. [#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) -- 修复了复制连接池不重试解析主机时的错误, 即使 DNS 缓存已删除. [#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) -- 修复了 `DROP INDEX IF EXISTS` 查询. 现在 `ALTER TABLE ... DROP INDEX IF EXISTS ...` 如果提供的索引不存在, 查询不会引发异常. [#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) -- 修复联合所有超类型列. 存在数据和结果列的列类型不一致的情况. [#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) -- 在 DDL 查询处理期间跳过 ZNONODE. 在另一个节点删除任务队列中的 znode 之前, 没有处理它但已经获得孩子列表的节点将终止 DDLWorker 线程. [#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) -- 使用 MATERIALIZED 列将 INSERT 修复到 Distributed() 表中. [#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) - -### ClickHouse 版本 19.7.3.9, 2019-05-30 {#clickhouse-release-19-7-3-9-2019-05-30} - -#### 新特征 {#new-features-2} - -- 允许限制用户可以指定的设置范围. 这些约束可以在用户设置配置文件中设置. [#4931](https://github.com/ClickHouse/ClickHouse/pull/4931) ([Vitaly Baranov](https://github.com/vitlibar)) -- 添加函数 `groupUniqArray` 的第二个版本, 并带有一个可选的 `max_size` 参数, 用于限制结果数组的大小, 这种行为类似于 `groupArray(max_size)(x)` 函数. [#5026](https://github.com/ClickHouse/ClickHouse/pull/5026) ([Guillaume Tassery](https://github.com/YiuRULE)) -- 对于 TSVWithNames/CSVWithNames 输入文件格式, 现在可以根据文件标题确定列顺序. 这由 `input_format_with_names_use_header` 参数控制. - [#5081](https://github.com/ClickHouse/ClickHouse/pull/5081) ([Alexander](https://github.com/Akazz)) - -#### Bug 修复 {#bug-fixes-2} - -- uncompressed_cache + JOIN在合并期间崩溃. (#5197)[#5133](https://github.com/ClickHouse/ClickHouse/pull/5133) ([Danila Kutenin](https://github.com/danlark1)) -- clickhouse 客户端查询系统表时出现分段错误. #5066. [#5127](https://github.com/ClickHouse/ClickHouse/pull/5127) ([Ivan](https://github.com/abyss7)) -- 通过 KafkaEngine 重载数据丢失. (#4736) [#5080](https://github.com/ClickHouse/ClickHouse/pull/5080) ([Ivan](https://github.com/abyss7)) -- 修复了在使用 UNION ALL 执行涉及至少两个来自 system.columns、system.tables、system.parts、system.parts_tables 或 Merge 系列表的查询并执行相关列的 ALTER 时可能发生的非常罕见的数据竞争条件并发表. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 性能改进 {#performance-improvements-1} - -- 使用基数排序按 `ORDER BY` 中的单个数字列进行排序, 而无需 `LIMIT`. [#5106](https://github.com/ClickHouse/ClickHouse/pull/5106), [#4439](https://github.com/ClickHouse/ClickHouse/pull/4439) ([Evgenii Pravda](https://github.com/kvinty), [alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 文档 {#documentation-1} - -- 将一些表引擎的文档翻译成中文. [#5107](https://github.com/ClickHouse/ClickHouse/pull/5107), [#5094](https://github.com/ClickHouse/ClickHouse/pull/5094), [#5087](https://github.com/ClickHouse/ClickHouse/pull/5087) ([张风啸](https://github.com/AlexZFX)), [#5068](https://github.com/ClickHouse/ClickHouse/pull/5068) ([never lee](https://github.com/neverlee)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements-1} - -- 在 `clickhouse-test` 中正确打印 UTF-8 字符. [#5084](https://github.com/ClickHouse/ClickHouse/pull/5084) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 为 clickhouse-client 添加命令行参数以始终加载建议数据. [#5102](https://github.com/ClickHouse/ClickHouse/pull/5102) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 解决部分 PVS-Studio 警告. [#5082](https://github.com/ClickHouse/ClickHouse/pull/5082) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 更新 LZ4. [#5040](https://github.com/ClickHouse/ClickHouse/pull/5040) ([Danila Kutenin](https://github.com/danlark1)) -- 添加 gperf 以构建即将到来的拉取请求的需求 #5030. [#5110](https://github.com/ClickHouse/ClickHouse/pull/5110) ([proller](https://github.com/proller)) - -## ClickHouse 版本 19.6 {#clickhouse-release-19-6} - -### ClickHouse 版本 19.6.3.18, 2019-06-13 {#clickhouse-release-19-6-3-18-2019-06-13} - -#### Bug 修复 {#bug-fixes-3} - -- 修复了来自表函数 `mysql` 和 `odbc` 以及相应表引擎的查询的 IN 条件下推. 这修复了 #3540 和 #2384. [#5313](https://github.com/ClickHouse/ClickHouse/pull/5313) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 Zookeeper 中的死锁. [#5297](https://github.com/ClickHouse/ClickHouse/pull/5297) ([github1youlc](https://github.com/github1youlc)) -- 允许在 CSV 中引用小数. [#5284](https://github.com/ClickHouse/ClickHouse/pull/5284) ([Artem Zuikov](https://github.com/4ertus2) -- 禁止从浮点 Inf/NaN 转换为小数 (抛出异常). [#5282](https://github.com/ClickHouse/ClickHouse/pull/5282) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复重命名查询中的数据竞争. [#5247](https://github.com/ClickHouse/ClickHouse/pull/5247) ([Winter Zhang](https://github.com/zhang2014)) -- 暂时禁用 LFAlloc. 使用 LFAlloc 可能会导致在分配 UncompressedCache 时出现大量 MAP_FAILED 并导致高负载服务器上的查询崩溃. [cfdba93](https://github.com/ClickHouse/ClickHouse/commit/cfdba938ce22f16efeec504f7f90206a515b1280)([Danila Kutenin](https://github.com/danlark1)) - -### ClickHouse 版本 19.6.2.11, 2019-05-13 {#clickhouse-release-19-6-2-11-2019-05-13} - -#### 新特征 {#new-features-3} - -- 列和表的 TTL 表达式. [#4212](https://github.com/ClickHouse/ClickHouse/pull/4212) ([Anton Popov](https://github.com/CurtizJ)) -- 添加了对 HTTP 响应的 `brotli` 压缩的支持 (接受编码:br). [#4388](https://github.com/ClickHouse/ClickHouse/pull/4388) ([Mikhail](https://github.com/fandyushin)) -- 添加了新函数 `isValidUTF8` 用于检查一组字节是否正确 utf-8 编码. [#4934](https://github.com/ClickHouse/ClickHouse/pull/4934) ([Danila Kutenin](https://github.com/danlark1)) -- 添加新的负载平衡策略 `first_or_random` , 它将查询发送到第一个指定的主机, 如果无法访问, 则将查询发送到分片的随机主机. 对交叉复制拓扑设置有用. [#5012](https://github.com/ClickHouse/ClickHouse/pull/5012) ([nvartolomei](https://github.com/nvartolomei)) - -#### 实验功能 {#experimental-features-1} - -- 为 MergeTree\* 表族添加设置 `index_granularity_bytes` (自适应索引粒度). [#4826](https://github.com/ClickHouse/ClickHouse/pull/4826) ([alesapin](https://github.com/alesapin)) - -#### 改进 {#improvements-1} - -- 添加了对函数 `substringUTF8` 的非常量和负大小和长度参数的支持. [#4989](https://github.com/ClickHouse/ClickHouse/pull/4989) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在左连接中禁用右表下推, 右连接中禁用左表, 以及完全连接中的两个表. 这在某些情况下修复了错误的 JOIN 结果. [#4846](https://github.com/ClickHouse/ClickHouse/pull/4846) ([Ivan](https://github.com/abyss7)) -- `clickhouse-copier`: 从`--task-file`选项自动上传任务配置. [#4876](https://github.com/ClickHouse/ClickHouse/pull/4876) ([proller](https://github.com/proller)) -- 为存储工厂和表函数工厂添加了错别字处理程序. [#4891](https://github.com/ClickHouse/ClickHouse/pull/4891) ([Danila Kutenin](https://github.com/danlark1)) -- 支持无子查询的多个连接的星号和合格星号. [#4898](https://github.com/ClickHouse/ClickHouse/pull/4898) ([Artem Zuikov](https://github.com/4ertus2)) -- 使缺失列错误消息更加用户友好. [#4915](https://github.com/ClickHouse/ClickHouse/pull/4915) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 性能改进 {#performance-improvements-2} - -- ASOF JOIN 的显着加速. [#4924](https://github.com/ClickHouse/ClickHouse/pull/4924) ([Martijn Bakker](https://github.com/Gladdy)) - -#### 向后不兼容更改 {#backward-incompatible-changes} - -- HTTP 标头 `Query-Id` 重命名为 `X-ClickHouse-Query-Id` 以保持一致性. [#4972](https://github.com/ClickHouse/ClickHouse/pull/4972) ([Mikhail](https://github.com/fandyushin)) - -#### Bug 修复 {#bug-fixes-4} - -- 修复了 `clickhouse-copier` 中潜在的空指针取消引用. [#4900](https://github.com/ClickHouse/ClickHouse/pull/4900) ([proller](https://github.com/proller)) -- 修复了 JOIN + ARRAY JOIN 查询错误. [#4938](https://github.com/ClickHouse/ClickHouse/pull/4938) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了当字典通过带有 engine=Dictionary 的数据库依赖另一个字典时服务器启动时挂起的问题. [#4962](https://github.com/ClickHouse/ClickHouse/pull/4962) ([Vitaly Baranov](https://github.com/vitlibar)) -- 部分修复了 `distributed_product_mode = local` . 可以通过表别名允许本地表的列在 `where/have/order by/...` 中. 如果表没有别名, 则抛出异常. 目前还无法访问没有表别名的列. [#4986](https://github.com/ClickHouse/ClickHouse/pull/4986) ([Artem Zuikov](https://github.com/4ertus2)) -- 使用 `JOIN` 修复 `SELECT DISTINCT` 的潜在错误结果. [#5001](https://github.com/ClickHouse/ClickHouse/pull/5001) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了在使用 UNION ALL 执行涉及至少两个来自 system.columns、system.tables、system.parts、system.parts_tables 或 Merge 系列表的查询并执行相关列的 ALTER 时可能发生的非常罕见的数据竞争条件并发表. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements-2} - -- 修复了在不同主机上运行 clickhouse-server 时的测试失败. [#4713](https://github.com/ClickHouse/ClickHouse/pull/4713) ([Vasily Nemkov](https://github.com/Enmk)) -- clickhouse-test: 在非 tty 环境中禁用颜色控制序列. [#4937](https://github.com/ClickHouse/ClickHouse/pull/4937) ([alesapin](https://github.com/alesapin)) -- clickhouse-test: 允许使用任何测试数据库 (尽可能删除 `test.` 限定) [#5008](https://github.com/ClickHouse/ClickHouse/pull/5008) ([proller](https://github.com/proller)) -- 修复 ubsan 错误. [#5037](https://github.com/ClickHouse/ClickHouse/pull/5037) ([Vitaly Baranov](https://github.com/vitlibar)) -- 在 ClickHouse 中添加了 Yandex LFAlloc 以不同方式分配 MarkCache 和 UncompressedCache 数据以更可靠地捕获段错误. [#4995](https://github.com/ClickHouse/ClickHouse/pull/4995) ([Danila Kutenin](https://github.com/danlark1)) -- Python util 可帮助进行向后移植和更改日志. [#4949](https://github.com/ClickHouse/ClickHouse/pull/4949) ([Ivan](https://github.com/abyss7)) - -## ClickHouse 版本 19.5 {#clickhouse-release-19-5} - -### ClickHouse 版本 19.5.4.22, 2019-05-13 {#clickhouse-release-19-5-4-22-2019-05-13} - -#### Bug 修复 {#bug-fixes-5} - -- 修复了位图\* 函数中可能发生的崩溃. [#5220](https://github.com/ClickHouse/ClickHouse/pull/5220) [#5228](https://github.com/ClickHouse/ClickHouse/pull/5228) ([Andy Yang](https://github.com/andyyzh)) -- 修复了在使用 UNION ALL 执行涉及至少两个来自 system.columns、system.tables、system.parts、system.parts_tables 或 Merge 系列表的查询并执行相关列的 ALTER 时可能发生的非常罕见的数据竞争条件并发表. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `Set for IN is not created yet in case of using single LowCardinality column in the left part of IN`. 如果 LowCardinality 列是主键的一部分,则会发生此错误. #5031 [#5154](https://github.com/ClickHouse/ClickHouse/pull/5154) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 保留函数修改: 如果一行同时满足第一个和第NTH个条件, 则只将第一个满足的条件添加到数据状态中. 现在在一行数据中满足的所有条件都添加到数据状态中. [#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) - -### ClickHouse 版本 19.5.3.8, 2019-04-18 {#clickhouse-release-19-5-3-8-2019-04-18} - -#### Bug 修复 {#bug-fixes-6} - -- 固定类型的设置 `max_partitions_per_insert_block` 从布尔值到 UInt64. [#5028](https://github.com/ClickHouse/ClickHouse/pull/5028) ([Mohammad Hossein Sekhavat](https://github.com/mhsekhavat)) - -### ClickHouse 版本 19.5.2.6, 2019-04-15 {#clickhouse-release-19-5-2-6-2019-04-15} - -#### 新特征 {#new-features-4} - -- [Hyperscan](https://github.com/intel/hyperscan) 添加了多个正则表达式匹配 (函数`multiMatchAny`、`multiMatchAnyIndex`、`multiFuzzyMatchAny`、`multiFuzzyMatchAnyIndex`). [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780), [#4841](https://github.com/ClickHouse/ClickHouse/pull/4841) ([Danila Kutenin](https://github.com/danlark1)) -- 添加了 `multiSearchFirstPosition` 功能. [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) -- 表实现每行预定义的表达式过滤器. [#4792](https://github.com/ClickHouse/ClickHouse/pull/4792) ([Ivan](https://github.com/abyss7)) -- 一种基于布隆过滤器的新型数据跳过索引 (可用于`equal`、`in`和`like`函数). [#4499](https://github.com/ClickHouse/ClickHouse/pull/4499) ([Nikita Vasilev](https://github.com/nikvas0)) -- 添加了`ASOF JOIN`, 它允许运行连接到已知最新值的查询. [#4774](https://github.com/ClickHouse/ClickHouse/pull/4774) [#4867](https://github.com/ClickHouse/ClickHouse/pull/4867) [#4863](https://github.com/ClickHouse/ClickHouse/pull/4863) [#4875](https://github.com/ClickHouse/ClickHouse/pull/4875) ([Martijn Bakker](https://github.com/Gladdy), [Artem Zuikov](https://github.com/4ertus2)) -- 将多个 `COMMA JOIN` 重写为 `CROSS JOIN`. 如果可能, 然后将它们重写为 `INNER JOIN` . [#4661](https://github.com/ClickHouse/ClickHouse/pull/4661) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 改进 {#improvement-9} - -- `topK` 和 `topK Weighted` 现在支持自定义 `load Factor`. (fixes issue [#4252](https://github.com/ClickHouse/ClickHouse/issues/4252)). [#4634](https://github.com/ClickHouse/ClickHouse/pull/4634) ([Kirill Danshin](https://github.com/kirillDanshin)) -- 即使对于没有采样的表, 也允许使用 `parallel_replicas_count > 1` (该设置被简单地忽略). 在以前的版本中, 它导致异常. [#4637](https://github.com/ClickHouse/ClickHouse/pull/4637) ([Alexey Elymanov](https://github.com/digitalist)) -- 支持 `CREATE OR REPLACE VIEW`. 允许在单个语句中创建视图或设置新定义. [#4654](https://github.com/ClickHouse/ClickHouse/pull/4654) ([Boris Granveaud](https://github.com/bgranvea)) -- `Buffer` 表引擎现在支持 `PREWHERE`. [#4671](https://github.com/ClickHouse/ClickHouse/pull/4671) ([Yangkuan Liu](https://github.com/LiuYangkuan)) -- 添加在 `readonly` 模式下在zookeeper中启动没有元数据的复制表的能力. [#4691](https://github.com/ClickHouse/ClickHouse/pull/4691) ([alesapin](https://github.com/alesapin)) -- 修复了 clickhouse-client 中进度条的闪烁. 将 `FORMAT Null` 与流查询一起使用时, 该问题最为明显. [#4811](https://github.com/ClickHouse/ClickHouse/pull/4811) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 允许在每个用户的基础上禁用带有 `hyperscan` 库的功能, 以限制可能过度和不受控制的资源使用. [#4816](https://github.com/ClickHouse/ClickHouse/pull/4816) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加版本号记录所有错误. [#4824](https://github.com/ClickHouse/ClickHouse/pull/4824) ([proller](https://github.com/proller)) -- 向 `multiMatch` 函数添加了限制, 该限制要求字符串大小适合 `unsigned int`。 还为 `multiSearch` 函数添加了参数数量限制. [#4834](https://github.com/ClickHouse/ClickHouse/pull/4834) ([Danila Kutenin](https://github.com/danlark1)) -- 改进了 Hyperscan 中暂存空间的使用和错误处理. [#4866](https://github.com/ClickHouse/ClickHouse/pull/4866) ([Danila Kutenin](https://github.com/danlark1)) -- 从 `*GraphiteMergeTree` 引擎表的表配置中填充 `system.graphite_detentions`. [#4584](https://github.com/ClickHouse/ClickHouse/pull/4584) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- 将 `trigramDistance` 函数重命名为 `ngramDistance` 并添加更多带有 `CaseInsensitive` 和 `UTF` 的函数. [#4602](https://github.com/ClickHouse/ClickHouse/pull/4602) ([Danila Kutenin](https://github.com/danlark1)) -- 改进的数据跳跃指数计算. [#4640](https://github.com/ClickHouse/ClickHouse/pull/4640) ([Nikita Vasilev](https://github.com/nikvas0)) -- 将普通、`DEFAULT`、`MATERIALIZED` 和`ALIAS` 列保留在一个列表中 (修复问题 [#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) - -#### Bug 修复 {#bug-fix-26} - -- 在内存分配失败的情况下避免使用 `std::terminate`. 现在按预期抛出 `std::bad_alloc` 异常. [#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了从缓冲区读取 capnproto 的问题. 有时文件未通过 HTTP 成功加载. [#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) -- 在 `OPTIMIZE TABLE FINAL` 查询后修复错误 `Unknown log entry type: 0` . [#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos Bird](https://github.com/amosbird)) -- `hasAny` 或 `hasAll` 函数的错误参数可能会导致段错误. [#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 执行 `DROP DATABASE dictionary` 查询时可能发生死锁. [#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `median` 和 `quantile` 函数中未定义的行为. [#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([hcz](https://github.com/hczhcz)) -- 修复 `network_compression_method` 为小写时的压缩级别检测. 在 v19.1 中损坏. [#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([proller](https://github.com/proller)) -- 修复了对 `UTC` 设置的无知 (修复了问题 [#4658](https://github.com/ClickHouse/ClickHouse/issues/4658). [#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([proller](https://github.com/proller)) -- 使用 `Distributed` 表修复 `histogram` 函数行为. [#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([olegkv](https://github.com/olegkv)) -- 修复了 tsan 报告 `destroy of a locked mutex` 的问题. [#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了由于系统日志使用中的竞争条件而导致关闭的 TSan 报告。 修复了启用 part_log 时关闭时潜在的释放后使用问题. [#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `ReplicatedMergeTreeAlterThread` 中的重新检查部分以防出错. [#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 中间聚合函数状态的算术运算不适用于常量参数 (例如子查询结果). [#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 始终在元数据中反引用列名称. 否则不可能创建一个名为`index`的列的表 (由于元数据中的`ATTACH`查询格式错误, 服务器不会重启). [#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `Distributed` 表上的 `ALTER ... MODIFY ORDER BY` 崩溃. [#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([TCeason](https://github.com/TCeason)) -- 使用启用的 `enable_optimize_predicate_expression` 修复 `JOIN ON` 中的段错误. [#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Winter Zhang](https://github.com/zhang2014)) -- 使用来自 Kafka 的 protobuf 消息后通过添加无关行来修复错误. [#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复 `JOIN` 在不可为空与可为空列上的崩溃. 修复 `ANY JOIN` + `join_use_nulls` 中右键中的 `NULLs`. [#4815](https://github.com/ClickHouse/ClickHouse/pull/4815) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 `clickhouse-copier` 中的分段错误. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -- 如果表被重命名或同时更改, 则修复了来自 `system.tables` 的 `SELECT` 中的竞争条件. [#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在获取已经过时的数据部分时修复了数据竞争. [#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 MergeTree 系列的 `RENAME` 表中可能发生的罕见数据竞争. [#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了函数 `arrayIntersect` 中的分段错误. 如果使用混合常量和普通参数调用函数, 可能会发生分段错误. [#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) -- 当列包含一长串空数组时, 修复了在极少数情况下从 `Array(LowCardinality)` 列读取的问题. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 当我们加入可空与不可空时,, 修复 `FULL/RIGHT JOIN` 中的崩溃. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) -- 在副本之间获取部分时修复 `No message received` 异常. [#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) -- 修复了在单个数组中有多个重复值的情况下`arrayIntersect`函数错误结果. [#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复并发 `ALTER COLUMN` 查询期间可能导致服务器崩溃的竞争条件 (修复问题 [#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Alex Zatelepin](https://github.com/ztlpn)) -- 使用 const 列修复 `FULL/RIGHT JOIN` 中的错误结果. [#4723](https://github.com/ClickHouse/ClickHouse/pull/4723) ([Artem Zuikov](https://github.com/4ertus2)) -- 用星号修复 `GLOBAL JOIN` 中的重复项. [#4705](https://github.com/ClickHouse/ClickHouse/pull/4705) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复未指定列类型时列 `CODEC` 的 `ALTER MODIFY` 中的参数推导. [#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) -- 函数 `cutQueryStringAndFragment()` 和 `queryStringAndFragment()` 现在可以在 `URL` 包含片段且没有查询时正常工作. [#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复了设置 `min_bytes_to_use_direct_io` 大于零时的罕见错误,当线程必须在列文件中向后寻找时会发生这种情况. [#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) -- 用 `LowCardinality` 参数修复聚合函数的错误参数类型 (修复问题 [#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复`GLOBAL JOIN`中的错误名称限定. [#4969](https://github.com/ClickHouse/ClickHouse/pull/4969) ([Artem Zuikov](https://github.com/4ertus2)) -- 修正函数 `toISOWeek` 结果为1970年. [#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `DROP`、`TRUNCATE` 和 `OPTIMIZE` 查询重复, 当在 `ON CLUSTER` 上为 `ReplicatedMergeTree*` 表族执行时. [#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) - -#### 向后不兼容更新 {#backward-incompatible-change-8} - -- 将设置 `insert_sample_with_metadata` 重命名为设置 `input_format_defaults_for_omited_fields`. [#4771](https://github.com/ClickHouse/ClickHouse/pull/4771) ([Artem Zuikov](https://github.com/4ertus2)) -- 添加了设置 `max_partitions_per_insert_block` (默认值为 100). 如果插入的块包含更多的分区, 则抛出异常. 如果要取消限制, 请将其设置为 0 (不推荐). [#4845](https://github.com/ClickHouse/ClickHouse/pull/4845) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 多重搜索功能被重命名 (`multiPosition` 到 `multiSearchAllPositions`, `multiSearch` 到 `multiSearchAny`, `firstMatch` 到 `multiSearchFirstIndex`). [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) - -#### 性能改进 {#performance-improvement-6} - -- 通过内联优化 Volnitsky 搜索器, 为具有许多针或许多相似二元组的查询提供大约 5-10% 的搜索改进. [#4862](https://github.com/ClickHouse/ClickHouse/pull/4862) ([Danila Kutenin](https://github.com/danlark1)) -- 修复了当设置 `use_uncompressed_cache` 大于零时的性能问题, 当所有读取包含在缓存中的数据时出现 [#4913](https://github.com/ClickHouse/ClickHouse/pull/4913) ([alesapin](https://github.com/alesapin)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-10} - -- 强化调试构建: 更细粒度的内存映射和 ASLR; 为标记缓存和索引添加内存保护. 这允许在 ASan 和 MSan 无法执行的情况下找到更多内存踩踏错误. [#4632](https://github.com/ClickHouse/ClickHouse/pull/4632) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加对 cmake 变量 `ENABLE_PROTOBUF`、`ENABLE_PARQUET` 和`ENABLE_BROTLI` 的支持, 允许启用/禁用上述功能 (与我们对 librdkafka、mysql 等所做的一样). [#4669](https://github.com/ClickHouse/ClickHouse/pull/4669) ([Silviu Caragea](https://github.com/silviucpp)) -- 如果在测试运行后挂起某些查询, 则添加打印所有线程的进程列表和堆栈跟踪的功能. [#4675](https://github.com/ClickHouse/ClickHouse/pull/4675) ([alesapin](https://github.com/alesapin)) -- 在 `clickhouse-test` 中添加对 `Connection loss` 错误的重试. [#4682](https://github.com/ClickHouse/ClickHouse/pull/4682) ([alesapin](https://github.com/alesapin)) -- 使用 vagrant 添加 freebsd 构建并使用线程清理器构建打包脚本. [#4712](https://github.com/ClickHouse/ClickHouse/pull/4712) [#4748](https://github.com/ClickHouse/ClickHouse/pull/4748) ([alesapin](https://github.com/alesapin)) -- 现在用户在安装过程中要求用户 `default` 的密码. [#4725](https://github.com/ClickHouse/ClickHouse/pull/4725) ([proller](https://github.com/proller)) -- 抑制 `rdkafka` 库中的警告. [#4740](https://github.com/ClickHouse/ClickHouse/pull/4740) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 允许在没有 ssl 的情况下构建. [#4750](https://github.com/ClickHouse/ClickHouse/pull/4750) ([proller](https://github.com/proller)) -- 添加一种从自定义用户启动 clickhouse-server 图像的方法. [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- 将 contrib 升级到 1.69. [#4793](https://github.com/ClickHouse/ClickHouse/pull/4793) ([proller](https://github.com/proller)) -- 使用 Thread Sanitizer 编译时禁用 `mremap`. 令人惊讶的是, TSan 没有拦截导致误报的 `mremap` (尽管它确实拦截了 `mmap`、`munmap`). 修复了状态测试中的 TSan 报告. [#4859](https://github.com/ClickHouse/ClickHouse/pull/4859) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 通过 HTTP 接口使用格式模式添加测试检查. [#4864](https://github.com/ClickHouse/ClickHouse/pull/4864) ([Vitaly Baranov](https://github.com/vitlibar)) - -## ClickHouse 版本 19.4 {#clickhouse-release-19-4} - -### ClickHouse 版本 19.4.4.33, 2019-04-17 {#clickhouse-release-19-4-4-33-2019-04-17} - -#### Bug 修复 {#bug-fixes-7} - -- 在内存分配失败的情况下避免使用 `std::terminate`。 现在按预期抛出 `std::bad_alloc` 异常. [#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了从缓冲区读取 capnproto 的问题. 有时文件未通过 HTTP 成功加载. [#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) -- 在 `OPTIMIZE TABLE FINAL` 查询后修复错误 `Unknown log entry type: 0` . [#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos Bird](https://github.com/amosbird)) -- `hasAny` 或 `hasAll` 函数的错误参数可能会导致段错误. [#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 执行 `DROP DATABASE dictionary` 查询时可能发生死锁. [#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `median` 和 `quantile` 函数中未定义的行为. [#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([hcz](https://github.com/hczhcz)) -- 修复 `network_compression_method` 为小写时的压缩级别检测. 在 v19.1 中损坏. [#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([proller](https://github.com/proller)) -- 修复了对 `UTC` 设置的无知 (修复了问题 [#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([proller](https://github.com/proller)) -- 使用 `Distributed` 表修复 `histogram` 函数行为. [#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([olegkv](https://github.com/olegkv)) -- 修复了 tsan 报告 `destroy of a locked mutex` 的问题. [#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了由于系统日志使用中的竞争条件而导致关闭的 TSan 报告. 修复了启用 part_log 时关闭时潜在的释放后使用问题. [#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `ReplicatedMergeTreeAlterThread` 中的重新检查部分以防出错. [#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 中间聚合函数状态的算术运算不适用于常量参数 (例如子查询结果). [#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 始终在元数据中反引用列名称. 否则不可能创建一个名为 `index` 的列的表 (由于元数据中的`ATTACH`查询格式错误, 服务器不会重启). [#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `Distributed` 表上的 `ALTER ... MODIFY ORDER BY` 崩溃. [#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([TCeason](https://github.com/TCeason)) -- 修复了启用 `enable_optimize_predicate_expression` 的 `JOIN ON` 中段错误. [#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Winter Zhang](https://github.com/zhang2014)) -- 使用来自 Kafka 的 protobuf 消息后通过添加无关行来修复错误. [#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复 `clickhouse-copier` 中的分段错误. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -- 如果表被重命名或同时更改, 则修复了来自 `system.tables` 的 `SELECT` 中的竞争条件. [#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在获取已经过时的数据部分时修复了数据竞争. [#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 MergeTree 系列的 `RENAME` 表中可能发生的罕见数据竞争. [#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了函数 `arrayIntersect` 中的分段错误. Segmentation fault could happen if function was called with mixed constant and ordinary arguments. [#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) -- 当列包含一长串空数组时, 修复了在极少数情况下从 `Array(LowCardinality)` 列读取的问题. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 在副本之间获取部分时修复 `No message received` 异常. [#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) -- 修复了在单个数组中有多个重复值的情况下`arrayIntersect`函数错误结果. [#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复并发 `ALTER COLUMN` 查询期间可能导致服务器崩溃的竞争条件 (修复问题 [#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Alex Zatelepin](https://github.com/ztlpn)) -- 修复未指定列类型时列 `CODEC` 的 `ALTER MODIFY` 中的参数推导. [#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) -- 函数 `cutQueryStringAndFragment()` 和 `queryStringAndFragment()` 现在可以在 `URL` 包含片段且没有查询时正常工作. [#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) -- 修复了设置 `min_bytes_to_use_direct_io` 大于零时的罕见错误,当线程必须在列文件中向后寻找时会发生这种情况. [#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) -- 使用 `LowCardinality` 参数修复聚合函数的错误参数类型 (修复问题 [#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修正函数 `toISOWeek` 结果为1970年. [#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 `DROP`、`TRUNCATE` 和 `OPTIMIZE` 查询重复,当在 `ON CLUSTER` 上为 `ReplicatedMergeTree*` 表族执行时. [#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) - -#### 改进 {#improvements-2} - -- 将普通、`DEFAULT`、`MATERIALIZED` 和`ALIAS` 列保留在一个列表中 (修复问题 [#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) - -### ClickHouse 版本 19.4.3.11, 2019-04-02 {#clickhouse-release-19-4-3-11-2019-04-02} - -#### Bug 修复 {#bug-fixes-8} - -- 当我们加入可空与不可空时, 修复 `FULL/RIGHT JOIN` 中的崩溃. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 `clickhouse-copier` 中的分段错误. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-11} - -- Add a way to launch clickhouse-server image from a custom user. [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) - -### ClickHouse 版本 19.4.2.7, 2019-03-30 {#clickhouse-release-19-4-2-7-2019-03-30} - -#### Bug 修复 {#bug-fixes-9} - -- 当列包含一长串空数组时, 修复了在极少数情况下从 `Array(LowCardinality)` 列读取的问题. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -### ClickHouse 版本 19.4.1.3, 2019-03-19 {#clickhouse-release-19-4-1-3-2019-03-19} - -#### Bug 修复 {#bug-fixes-10} - -- 修复了同时包含`LIMIT BY` 和`LIMIT` 的远程查询. 以前, 如果使用 `LIMIT BY` 和 `LIMIT` 进行远程查询, `LIMIT`可能发生在`LIMIT BY`之前, 导致过滤结果过分. [#4708](https://github.com/ClickHouse/ClickHouse/pull/4708) ([Constantin S. Pan](https://github.com/kvap)) - -### ClickHouse 版本 19.4.0.49, 2019-03-09 {#clickhouse-release-19-4-0-49-2019-03-09} - -#### 新特征 {#new-features-5} - -- 添加了对 `Protobuf` 格式的完全支持 (输入和输出, 嵌套数据结构). [#4174](https://github.com/ClickHouse/ClickHouse/pull/4174) [#4493](https://github.com/ClickHouse/ClickHouse/pull/4493) ([Vitaly Baranov](https://github.com/vitlibar)) -- 使用 Roaring Bitmaps 添加位图功能. [#4207](https://github.com/ClickHouse/ClickHouse/pull/4207) ([Andy Yang](https://github.com/andyyzh)) [#4568](https://github.com/ClickHouse/ClickHouse/pull/4568) ([Vitaly Baranov](https://github.com/vitlibar)) -- Parquet 格式支持. [#4448](https://github.com/ClickHouse/ClickHouse/pull/4448) ([proller](https://github.com/proller)) -- 添加了 N-gram 距离以进行模糊字符串比较. 它类似于 R 语言中的 q-gram 度量. [#4466](https://github.com/ClickHouse/ClickHouse/pull/4466) ([Danila Kutenin](https://github.com/danlark1)) -- 从专用的聚合和保留模式中组合石墨聚合规则. [#4426](https://github.com/ClickHouse/ClickHouse/pull/4426) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -- 添加了 `max_execution_speed` 和 `max_execution_speed_bytes` 以限制资源使用. 添加了`min_execution_speed_bytes`设置以补充`min_execution_speed`. [#4430](https://github.com/ClickHouse/ClickHouse/pull/4430) ([Winter Zhang](https://github.com/zhang2014)) -- 实现的函数`flatten`. [#4555](https://github.com/ClickHouse/ClickHouse/pull/4555) [#4409](https://github.com/ClickHouse/ClickHouse/pull/4409) ([alexey-milovidov](https://github.com/alexey-milovidov), [kzon](https://github.com/kzon)) -- 添加了函数 `arrayEnumerateDenseRanked` 和 `arrayEnumerateUniqRanked` (类似于 `arrayEnumerateUniq`, 但允许微调数组深度以查看多维数组内部). [#4475](https://github.com/ClickHouse/ClickHouse/pull/4475) ([proller](https://github.com/proller)) [#4601](https://github.com/ClickHouse/ClickHouse/pull/4601) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 有一些限制的多个 JOINS: 没有星号, 在 ON/WHERE/GROUP BY/... 中没有复杂的别名. [#4462](https://github.com/ClickHouse/ClickHouse/pull/4462) ([Artem Zuikov](https://github.com/4ertus2)) - -#### Bug 修复 {#bug-fixes-11} - -- 此版本还包含 19.3 和 19.1 的所有错误修复. -- 修复了数据跳过索引中的错误: INSERT 后的颗粒顺序不正确. [#4407](https://github.com/ClickHouse/ClickHouse/pull/4407) ([Nikita Vasilev](https://github.com/nikvas0)) -- 修复了 `Nullable` 和 `LowCardinality` 列的 `set` 索引. 在此之前, 带有 `Nullable` 或 `LowCardinality` 列的 `set` 索引在选择时导致错误 `Data type must be deserialized with multiple streams` . [#4594](https://github.com/ClickHouse/ClickHouse/pull/4594) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 在完整的 `executable` 字典更新时正确设置 update_time. [#4551](https://github.com/ClickHouse/ClickHouse/pull/4551) ([Tema Novikov](https://github.com/temoon)) -- 修复 19.3 中损坏的进度条. [#4627](https://github.com/ClickHouse/ClickHouse/pull/4627) ([filimonov](https://github.com/filimonov)) -- 修复了在某些情况下内存区域缩小时 MemoryTracker 不一致的值. [#4619](https://github.com/ClickHouse/ClickHouse/pull/4619) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 ThreadPool 中未定义的行为. [#4612](https://github.com/ClickHouse/ClickHouse/pull/4612) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了一个非常罕见的崩溃, 消息 `mutex lock failed: Invalid argument` 可能发生在 MergeTree 表与 SELECT 同时删除时. [#4608](https://github.com/ClickHouse/ClickHouse/pull/4608) ([Alex Zatelepin](https://github.com/ztlpn)) -- ODBC 驱动程序与 `LowCardinality` 数据类型兼容. [#4381](https://github.com/ClickHouse/ClickHouse/pull/4381) ([proller](https://github.com/proller)) -- FreeBSD: `AIOcontextPool: Found io_event with unknown id 0` 错误的修复. [#4438](https://github.com/ClickHouse/ClickHouse/pull/4438) ([urgordeadbeef](https://github.com/urgordeadbeef)) -- `system.part_log` 表的创建与配置无关. [#4483](https://github.com/ClickHouse/ClickHouse/pull/4483) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 复缓存字典的 `dictIsIn` 函数中未定义的行为. [#4515](https://github.com/ClickHouse/ClickHouse/pull/4515) ([alesapin](https://github.com/alesapin)) -- 修复了当 SELECT 查询多次锁定同一个表 (例如来自不同线程或执行多个子查询时) 并且存在并发 DDL 查询时的死锁. [#4535](https://github.com/ClickHouse/ClickHouse/pull/4535) ([Alex Zatelepin](https://github.com/ztlpn)) -- 默认情况下禁用 compile_expressions 直到我们获得自己的 `llvm` contrib 并且可以使用 `clang` 和 `asan` 对其进行测试. [#4579](https://github.com/ClickHouse/ClickHouse/pull/4579) ([alesapin](https://github.com/alesapin)) -- `clickhouse` 外部字典源的 `invalidate_query` 返回错误的结果集 (空或多于一行或多于一列) 时, 防止 `std::terminate`. 修复了每五秒执行一次 `invalidate_query` 的问题, 而不管 `lifetime`. [#4583](https://github.com/ClickHouse/ClickHouse/pull/4583) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 当具有 `clickhouse` 源的字典的 `invalidate_query` 涉及 `system.dictionaries` 表或 `Dictionaries` 数据库时, 避免死锁 (罕见情况). [#4599](https://github.com/ClickHouse/ClickHouse/pull/4599) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 WHERE 为空的 CROSS JOIN. [#4598](https://github.com/ClickHouse/ClickHouse/pull/4598) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了传递常量参数时函数 `replicate` 中的段错误. [#4603](https://github.com/ClickHouse/ClickHouse/pull/4603) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用谓词优化器修复 lambda 函数. [#4408](https://github.com/ClickHouse/ClickHouse/pull/4408) ([Winter Zhang](https://github.com/zhang2014)) -- 多个连接多个修复. [#4595](https://github.com/ClickHouse/ClickHouse/pull/4595) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 改进 {#improvements-3} - -- 支持右表列的 JOIN ON 部分中的别名. [#4412](https://github.com/ClickHouse/ClickHouse/pull/4412) ([Artem Zuikov](https://github.com/4ertus2)) -- 多个 JOIN 的结果需要在子选择中使用正确的结果名称. 在结果中用源名称替换平面别名. [#4474](https://github.com/ClickHouse/ClickHouse/pull/4474) ([Artem Zuikov](https://github.com/4ertus2)) -- 改进连接语句的下推逻辑. [#4387](https://github.com/ClickHouse/ClickHouse/pull/4387) ([Ivan](https://github.com/abyss7)) - -#### 性能改进 {#performance-improvements-3} - -- 改进了 `move to PREWHERE` 优化的启发式方法. [#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用适当的查找表, 使用 HashTable 的 API 获取 8 位和 16 位密钥. [#4536](https://github.com/ClickHouse/ClickHouse/pull/4536) ([Amos Bird](https://github.com/amosbird)) -- 改进了字符串比较的性能. [#4564](https://github.com/ClickHouse/ClickHouse/pull/4564) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在单独的线程中清理分布式 DDL 队列, 使其不会减慢处理分布式 DDL 任务的主循环. [#4502](https://github.com/ClickHouse/ClickHouse/pull/4502) ([Alex Zatelepin](https://github.com/ztlpn)) -- 当 `min_bytes_to_use_direct_io` 设置为 1 时, 并不是每个文件都以 O_DIRECT 模式打开, 因为读取的数据大小有时会被一个压缩块的大小低估. [#4526](https://github.com/ClickHouse/ClickHouse/pull/4526) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-12} - -- 添加了对 clang-9 的支持. [#4604](https://github.com/ClickHouse/ClickHouse/pull/4604) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复错误的 `__asm__` 指令 (再次). [#4621](https://github.com/ClickHouse/ClickHouse/pull/4621) ([Konstantin Podshumok](https://github.com/podshumok)) -- 添加从命令行指定 `clickhouse-performance-test` 设置的功能. [#4437](https://github.com/ClickHouse/ClickHouse/pull/4437) ([alesapin](https://github.com/alesapin)) -- 将字典测试添加到集成测试. [#4477](https://github.com/ClickHouse/ClickHouse/pull/4477) ([alesapin](https://github.com/alesapin)) -- 将来自网站基准测试的查询添加到自动化性能测试中. [#4496](https://github.com/ClickHouse/ClickHouse/pull/4496) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `xxhash.h` 在外部 lz4 中不存在, 因为它是一个实现细节, 它的符号使用 `XXH_NAMESPACE` 宏命名. 当 lz4 是外部的时, xxHash 也必须是外部的, 并且依赖项必须链接到它. [#4495](https://github.com/ClickHouse/ClickHouse/pull/4495) ([Orivej Desh](https://github.com/orivej)) -- 修复了可以使用负数或浮点参数调用 `quantileTiming` 聚合函数的情况 (这修复了使用未定义行为清理器的模糊测试). [#4506](https://github.com/ClickHouse/ClickHouse/pull/4506) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 拼写错误更正. [#4531](https://github.com/ClickHouse/ClickHouse/pull/4531) ([sdk2](https://github.com/sdk2)) -- 修复 Mac 上的编译. [#4371](https://github.com/ClickHouse/ClickHouse/pull/4371) ([Vitaly Baranov](https://github.com/vitlibar)) -- 为 FreeBSD 和各种不寻常的构建配置构建修复. [#4444](https://github.com/ClickHouse/ClickHouse/pull/4444) ([proller](https://github.com/proller)) - -## ClickHouse 版本 19.3 {#clickhouse-release-19-3} - -### ClickHouse 版本 19.3.9.1, 2019-04-02 {#clickhouse-release-19-3-9-1-2019-04-02} - -#### Bug 修复 {#bug-fixes-12} - -- 当我们加入可空与不可空时, 修复 `FULL/RIGHT JOIN` 中的崩溃. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复 `clickhouse-copier` 中的分段错误. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -- 当列包含一长串空数组时, 修复了在极少数情况下从 `Array(LowCardinality)` 列读取的问题. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvement-13} - -- 添加一种从自定义用户启动 clickhouse-server 图像的方法. [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) - -### ClickHouse 版本 19.3.7, 2019-03-12 {#clickhouse-release-19-3-7-2019-03-12} - -#### Bug 修复 {#bug-fixes-13} - -- 修复了 #3920 中的错误。 此错误表现为随机缓存损坏 (消息 `Unknown codec family code`, `Cannot seek through file` ) 和段错误. 此错误首先出现在 19.1 版本中, 并存在于 19.1.10 和 19.3.6 之前的版本中. [#4623](https://github.com/ClickHouse/ClickHouse/pull/4623) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.3.6, 2019-03-02 {#clickhouse-release-19-3-6-2019-03-02} - -#### Bug 修复 {#bug-fixes-14} - -- 当线程池中的线程超过1000个时, 线程退出可能会发生`std::terminate`. [Azat Khuzhin](https://github.com/azat) [#4485](https://github.com/ClickHouse/ClickHouse/pull/4485) [#4505](https://github.com/ClickHouse/ClickHouse/pull/4505) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 现在可以创建带有注释的 `ReplicatedMergeTree*` 表,没有默认值的列和带有没有注释和默认值的列编解码器的表. 还修复了编解码器的比较. [#4523](https://github.com/ClickHouse/ClickHouse/pull/4523) ([alesapin](https://github.com/alesapin)) -- 修复了使用数组或元组 JOIN 时的崩溃. [#4552](https://github.com/ClickHouse/ClickHouse/pull/4552) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了 clickhouse-copier 崩溃并显示消息 `ThreadStatus not created` . [#4540](https://github.com/ClickHouse/ClickHouse/pull/4540) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了使用分布式ddl时服务器关闭时的挂起问题. [#4472](https://github.com/ClickHouse/ClickHouse/pull/4472) ([Alex Zatelepin](https://github.com/ztlpn)) -- 错误消息中打印了不正确的列号, 该错误消息是对编号大于 10 的列进行文本格式解析. [#4484](https://github.com/ClickHouse/ClickHouse/pull/4484) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements-3} - -- 修复了启用 AVX 的构建. [#4527](https://github.com/ClickHouse/ClickHouse/pull/4527) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 用基于已知版本而不是编译它的内核的扩展记帐和 IO 记帐. [#4541](https://github.com/ClickHouse/ClickHouse/pull/4541) ([nvartolomei](https://github.com/nvartolomei)) -- 用基于已知版本而不是合成它的知识的扩展记录帐和 IO. [#4473](https://github.com/ClickHouse/ClickHouse/pull/4473) ([proller](https://github.com/proller)) -- 删除了 `Field.cpp` 中 `void readBinary(...)` 的 `inline` 标签. 还合并了冗余的 `namespace DB` 块. [#4530](https://github.com/ClickHouse/ClickHouse/pull/4530) ([hcz](https://github.com/hczhcz)) - -### ClickHouse 版本 19.3.5, 2019-02-21 {#clickhouse-release-19-3-5-2019-02-21} - -#### Bug 修复 {#bug-fixes-15} - -- 修复了大型 http 插入查询处理的错误. [#4454](https://github.com/ClickHouse/ClickHouse/pull/4454) ([alesapin](https://github.com/alesapin)) -- 修复了由于 `send_logs_level` 设置的错误实现而导致的与旧版本的向后不兼容. [#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了使用列注释引入的表函数`remote` 向后不兼容的问题. [#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.3.4, 2019-02-16 {#clickhouse-release-19-3-4-2019-02-16} - -#### 改进 {#improvements-4} - -- 当执行 `ATTACH Table` 查询时, 表索引大小不考虑内存限制. 避免了表被分离后无法被附加的可能性. [#4396](https://github.com/ClickHouse/ClickHouse/pull/4396) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 稍微提高了从 ZooKeeper 接收的最大字符串和数组大小的限制. 它允许继续使用 ZooKeeper 上增加的 `CLIENT_JVMFLAGS=-Djute.maxbuffer=...` 大小. [#4398](https://github.com/ClickHouse/ClickHouse/pull/4398) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 允许修复废弃的副本, 即使它的队列中已经有大量节点. [#4399](https://github.com/ClickHouse/ClickHouse/pull/4399) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 向`SET`索引添加一个必需参数 (最大存储行数). [#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) - -#### Bug 修复 {#bug-fixes-16} - -- 修复了按单个 `LowCardinality` 键分组的 `WITH ROLLUP` 结果. [#4384](https://github.com/ClickHouse/ClickHouse/pull/4384) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 修复了集合索引中的错误 (如果包含超过 `max_rows` 行,则删除一个粒度). [#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) -- 许多 FreeBSD 构建修复. [#4397](https://github.com/ClickHouse/ClickHouse/pull/4397) ([proller](https://github.com/proller)) -- 使用包含相同别名的子查询修复了查询中的别名替换 (问题 [#4110](https://github.com/ClickHouse/ClickHouse/issues/4110)). [#4351](https://github.com/ClickHouse/ClickHouse/pull/4351) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements-4} - -- 添加在 docker 镜像中运行 `clickhouse-server` 以进行无状态测试的能力. [#4347](https://github.com/ClickHouse/ClickHouse/pull/4347) ([Vasily Nemkov](https://github.com/Enmk)) - -### ClickHouse 版本 19.3.3, 2019-02-13 {#clickhouse-release-19-3-3-2019-02-13} - -#### 新特征 {#new-features-6} - -- 添加了 `KILL MUTATION` 语句,允许删除由于某些原因卡住的突变。 将 `latest_failed_part`、`latest_fail_time`、`latest_fail_reason` 字段添加到 `system.mutations` 表中,以便于故障排除. [#4287](https://github.com/ClickHouse/ClickHouse/pull/4287) ([Alex Zatelepin](https://github.com/ztlpn)) -- 添加了计算信息熵的聚合函数 `entropy` . [#4238](https://github.com/ClickHouse/ClickHouse/pull/4238) ([Quid37](https://github.com/Quid37)) -- 添加了向服务器发送查询 `INSERT INTO tbl VALUES (....)` 的功能, 而无需拆分 `query` 和 `data` 部分. [#4301](https://github.com/ClickHouse/ClickHouse/pull/4301) ([alesapin](https://github.com/alesapin)) -- 添加了 `arrayWithConstant` 函数的通用实现. [#4322](https://github.com/ClickHouse/ClickHouse/pull/4322) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 实现了`NOT BETWEEN`比较运算符. [#4228](https://github.com/ClickHouse/ClickHouse/pull/4228) ([Dmitry Naumov](https://github.com/nezed)) -- 实现 `sumMapFiltered` 以便能够限制 `sumMap` 对其值求和的键的数量. [#4129](https://github.com/ClickHouse/ClickHouse/pull/4129) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) -- 在 `mysql` 表函数中添加了对 `Nullable` 类型的支持. [#4198](https://github.com/ClickHouse/ClickHouse/pull/4198) ([Emmanuel Donin de Rosière](https://github.com/edonin)) -- 支持 `LIMIT` 子句中的任意常量表达式. [#4246](https://github.com/ClickHouse/ClickHouse/pull/4246) ([k3box](https://github.com/k3box)) -- 添加了 `topKWeighted` 聚合函数, 该函数采用带 (无符号整数) 权重的附加参数. [#4245](https://github.com/ClickHouse/ClickHouse/pull/4245) ([Andrew Golman](https://github.com/andrewgolman)) -- `StorageJoin` 现在支持 `join_any_take_last_row` 设置, 允许覆盖相同键的现有值. [#3973](https://github.com/ClickHouse/ClickHouse/pull/3973) ([Amos Bird](https://github.com/amosbird) -- 添加函数 `toStartOfInterval` . [#4304](https://github.com/ClickHouse/ClickHouse/pull/4304) ([Vitaly Baranov](https://github.com/vitlibar)) -- 添加了 `RowBinaryWithNamesAndTypes` 格式. [#4200](https://github.com/ClickHouse/ClickHouse/pull/4200) ([Oleg V. Kozlyuk](https://github.com/DarkWanderer)) -- 添加了 `IPv4` 和 `IPv6` 数据类型. 更有效地实现 `IPv*` 功能. [#3669](https://github.com/ClickHouse/ClickHouse/pull/3669) ([Vasily Nemkov](https://github.com/Enmk)) -- 添加函数 `toStartOfTenMinutes()`. [#4298](https://github.com/ClickHouse/ClickHouse/pull/4298) ([Vitaly Baranov](https://github.com/vitlibar)) -- 添加了 `Protobuf` 输出格式. [#4005](https://github.com/ClickHouse/ClickHouse/pull/4005) [#4158](https://github.com/ClickHouse/ClickHouse/pull/4158) ([Vitaly Baranov](https://github.com/vitlibar)) -- 为数据导入的 HTTP 接口添加了 brotli 支持 (插入). [#4235](https://github.com/ClickHouse/ClickHouse/pull/4235) ([Mikhail](https://github.com/fandyushin)) -- 添加了用户在函数名称中输入拼写错误或在命令行客户端中输入时的提示. [#4239](https://github.com/ClickHouse/ClickHouse/pull/4239) ([Danila Kutenin](https://github.com/danlark1)) -- 将 `Query-Id` 添加到服务器的 HTTP 响应标头. [#4231](https://github.com/ClickHouse/ClickHouse/pull/4231) ([Mikhail](https://github.com/fandyushin)) - -#### 实验功能 {#experimental-features-2} - -- 为 MergeTree 表引擎系列添加了 `minmax` 和 `set` 数据跳过索引. [#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) -- 如果可能, 添加了 `CROSS JOIN` 到 `INNER JOIN` 的转换. [#4221](https://github.com/ClickHouse/ClickHouse/pull/4221) [#4266](https://github.com/ClickHouse/ClickHouse/pull/4266) ([Artem Zuikov](https://github.com/4ertus2)) - -#### Bug 修复 {#bug-fixes-17} - -- 修复了 `JOIN ON` 部分中重复列的 `Not found column` . [#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) -- 使 `START REPLICATED SENDS` 命令开始复制发送. [#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) -- 使用 `Array(LowCardinality)` 参数修复了聚合函数的执行. [#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 修复了在执行 `INSERT ... SELECT ... FROM file(...)` 查询和文件具有 `CSVWithNames` 或 `TSVWIthNames` 格式并且缺少第一个数据行时的错误行为. [#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了字典不可用时重新加载字典时崩溃的问题. 这个bug出现在19.1.6. [#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([proller](https://github.com/proller)) -- 修复了右表中有重复项的 `ALL JOIN` . [#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了 `use_uncompressed_cache=1` 的分段错误和错误未压缩大小的异常. 这个bug出现在19.1.6. [#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) -- 修复了 `compile_expressions` 与大 (超过 int16) 日期比较的错误. [#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) -- 从表函数 `numbers(0)` 中选择时修复了无限循环. [#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 暂时禁用 `ORDER BY` 的谓词优化. [#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Winter Zhang](https://github.com/zhang2014)) -- 修复了在旧 CPU 上使用 base64 函数时的 `Illegal instruction` 错误. 仅在使用 gcc-8 编译 ClickHouse 时才重现此错误. [#4275](https://github.com/ClickHouse/ClickHouse/pull/4275) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了通过 TLS 连接与 PostgreSQL ODBC 驱动程序交互时的 `No message received` 错误. 还修复了使用 MySQL ODBC 驱动程序时的段错误. [#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在条件运算符 (函数 `if`) 的分支中使用 `Date` 和 `DateTime` 参数时不正确的结果. 为函数 `if` 添加了通用情况. [#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- ClickHouse 词典现在在 `clickhouse` 进程中加载. [#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在 `No such file or directory` 错误后重试带有 `File` 引擎的表中的 `SELECT` 时的死锁. [#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了从 `system.tables` 中选择时的竞争条件可能会出现 `table does not exist` 错误. [#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 如果以交互模式运行, `clickhouse-client` 可以在退出时加载命令行建议数据时出现段错误. [#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了执行包含 `IN` 运算符的突变时产生错误结果的错误. [#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Alex Zatelepin](https://github.com/ztlpn)) -- 修复错误: 如果有一个带有 `Dictionary` 引擎的数据库, 所有词典都在服务器启动时强制加载, 如果有来自localhost的带有ClickHouse源的词典, 则该词典无法加载. [#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在服务器关闭时尝试再次创建系统日志时的错误. [#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 正确返回正确的类型并正确处理 `joinGet` 函数中的锁. [#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) -- 添加了 `sumMapWithOverflow` 函数. [#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) -- 使用 `allow_experimental_multiple_joins_emulation` 修复段错误. [52de2c](https://github.com/ClickHouse/ClickHouse/commit/52de2cd927f7b5257dd67e175f0a5560a48840d0) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了错误的 `Date` 和`DateTime` 比较错误. [#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) -- 修复了未定义行为消毒剂下的模糊测试: 添加了对 `quantile*Weighted` 系列函数的参数类型检查. [#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了删除旧数据部分可能会因 `File not found` 错误而失败时的罕见竞争条件. [#4378](https://github.com/ClickHouse/ClickHouse/pull/4378) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复缺少 /etc/clickhouse-server/config.xml 的安装包. [#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements-5} - -- Debian 软件包: 根据配置更正 /etc/clickhouse-server/preprocessed 链接. [#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) -- FreeBSD 的各种构建修复. [#4225](https://github.com/ClickHouse/ClickHouse/pull/4225) ([proller](https://github.com/proller)) -- 添加了在 perftest 中创建、填充和删除表格的功能. [#4220](https://github.com/ClickHouse/ClickHouse/pull/4220) ([alesapin](https://github.com/alesapin)) -- 添加了一个脚本来检查重复的包含. [#4326](https://github.com/ClickHouse/ClickHouse/pull/4326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了在性能测试中按索引运行查询的功能. [#4264](https://github.com/ClickHouse/ClickHouse/pull/4264) ([alesapin](https://github.com/alesapin)) -- 建议安装带有调试符号的包. [#4274](https://github.com/ClickHouse/ClickHouse/pull/4274) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 重构性能测试. 更好的日志记录和信号处理. [#4171](https://github.com/ClickHouse/ClickHouse/pull/4171) ([alesapin](https://github.com/alesapin)) -- 向匿名化的 Yandex.Metrika 数据集添加了文档. [#4164](https://github.com/ClickHouse/ClickHouse/pull/4164) ([alesapin](https://github.com/alesapin)) -- Аdded 工具, 用于将旧的月份分区部分转换为自定义分区格式. [#4195](https://github.com/ClickHouse/ClickHouse/pull/4195) ([Alex Zatelepin](https://github.com/ztlpn)) -- 在 s3 中添加了关于两个数据集的文档. [#4144](https://github.com/ClickHouse/ClickHouse/pull/4144) ([alesapin](https://github.com/alesapin)) -- 添加了从拉取请求描述创建变更日志的脚本. [#4169](https://github.com/ClickHouse/ClickHouse/pull/4169) [#4173](https://github.com/ClickHouse/ClickHouse/pull/4173) ([KochetovNicolai](https://github.com/KochetovNicolai)) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 为 ClickHouse 添加了 puppet 模块. [#4182](https://github.com/ClickHouse/ClickHouse/pull/4182) ([Maxim Fedotov](https://github.com/MaxFedotov)) -- 添加了一组未记录功能的文档. [#4168](https://github.com/ClickHouse/ClickHouse/pull/4168) ([Winter Zhang](https://github.com/zhang2014)) -- ARM 构建修复. [#4210](https://github.com/ClickHouse/ClickHouse/pull/4210)[#4306](https://github.com/ClickHouse/ClickHouse/pull/4306) [#4291](https://github.com/ClickHouse/ClickHouse/pull/4291) ([proller](https://github.com/proller)) ([proller](https://github.com/proller)) -- 字典测试现在可以从 `ctest` 运行. [#4189](https://github.com/ClickHouse/ClickHouse/pull/4189) ([proller](https://github.com/proller)) -- 现在 `/etc/ssl` 被用作 SSL 证书的默认目录. [#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 在开始时添加了检查 SSE 和 AVX 指令. [#4234](https://github.com/ClickHouse/ClickHouse/pull/4234) ([Igr](https://github.com/igron99)) -- 初始化脚本将等待服务器直到启动. [#4281](https://github.com/ClickHouse/ClickHouse/pull/4281) ([proller](https://github.com/proller)) - -#### 向后不兼容更改 {#backward-incompatible-changes-1} - -- 删除了 `allow_experimental_low_cardinality_type` 设置. `LowCardinality` 数据类型是生产就绪的. [#4323](https://github.com/ClickHouse/ClickHouse/pull/4323) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 根据可用内存量减少标记缓存大小和未压缩缓存大小. [#4240](https://github.com/ClickHouse/ClickHouse/pull/4240) ([Lopatin Konstantin](https://github.com/k-lopatin) -- 在 `CREATE TABLE` 查询中添加了关键字 `INDEX` . 名称为 `index` 的列必须用反引号或双引号引用: `` `index` ``. [#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) -- `sumMap` 现在提升结果类型而不是溢出. 旧的 `sumMap` 行为可以通过使用 `sumMapWithOverflow` 函数获得. [#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) - -#### 性能改进 {#performance-improvements-4} - -- `std::sort` 替换为 `pdqsort` 用于没有 `LIMIT` 的查询. [#4236](https://github.com/ClickHouse/ClickHouse/pull/4236) ([Evgenii Pravda](https://github.com/kvinty)) -- 现在服务器重用来自全局线程池的线程. 这会影响某些极端情况下的性能. [#4150](https://github.com/ClickHouse/ClickHouse/pull/4150) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 改进 {#improvements-5} - -- 为 FreeBSD 实现 AIO 支持. [#4305](https://github.com/ClickHouse/ClickHouse/pull/4305) ([urgordeadbeef](https://github.com/urgordeadbeef)) -- `SELECT * FROM a JOIN b USING a, b` 现在只从左表返回 `a` 和 `b` 列. [#4141](https://github.com/ClickHouse/ClickHouse/pull/4141) ([Artem Zuikov](https://github.com/4ertus2)) -- 允许客户端的`-C` 选项作为`-c` 选项工作. [#4232](https://github.com/ClickHouse/ClickHouse/pull/4232) ([syominsergey](https://github.com/syominsergey)) -- 现在不带值使用的选项 `--password` 需要来自 stdin 的密码. [#4230](https://github.com/ClickHouse/ClickHouse/pull/4230) ([BSD_Conqueror](https://github.com/bsd-conqueror)) -- 在包含 `LIKE` 表达式或正则表达式的字符串文字中添加了未转义元字符的突出显示. [#4327](https://github.com/ClickHouse/ClickHouse/pull/4327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 如果客户端套接字消失, 则添加取消 HTTP 只读查询. [#4213](https://github.com/ClickHouse/ClickHouse/pull/4213) ([nvartolomei](https://github.com/nvartolomei)) -- 现在服务器报告进度以保持客户端连接有效. [#4215](https://github.com/ClickHouse/ClickHouse/pull/4215) ([Ivan](https://github.com/abyss7)) -- 启用了 `optimize_throw_if_noop` 设置的 OPTIMIZE 查询原因稍微好一点的消息. [#4294](https://github.com/ClickHouse/ClickHouse/pull/4294) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 为 clickhouse 服务器添加了对 `--version` 选项的支持. [#4251](https://github.com/ClickHouse/ClickHouse/pull/4251) ([Lopatin Konstantin](https://github.com/k-lopatin)) -- 为 `clickhouse-server` 添加了 `--help/-h` 选项. [#4233](https://github.com/ClickHouse/ClickHouse/pull/4233) ([Yuriy Baranov](https://github.com/yurriy)) -- 添加了对具有聚合函数状态结果的标量子查询的支持. [#4348](https://github.com/ClickHouse/ClickHouse/pull/4348) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -- 改进了服务器关闭时间和 ALTERs 等待时间. [#4372](https://github.com/ClickHouse/ClickHouse/pull/4372) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 向 system.replicas 添加了关于 replicad_can_become_leader 设置的信息, 如果副本不会尝试成为领导者,则添加日志记录. [#4379](https://github.com/ClickHouse/ClickHouse/pull/4379) ([Alex Zatelepin](https://github.com/ztlpn)) - -## ClickHouse 版本 19.1 {#clickhouse-release-19-1} - -### ClickHouse 版本 19.1.14, 2019-03-14 {#clickhouse-release-19-1-14-2019-03-14} - -- 修复了在将 `GLOBAL JOIN` 与 `SELECT *` (罕见情况) 一起使用的情况下, 如果设置 `asterisk_left_columns_only` 设置为 1 时可能发生的错误 `Column ... queried more than once`. 该问题在 19.3 及更新版本中不存在. [6bac7d8d](https://github.com/ClickHouse/ClickHouse/pull/4692/commits/6bac7d8d11a9b0d6de0b32b53c47eb2f6f8e7062) ([Artem Zuikov](https://github.com/4ertus2)) - -### ClickHouse 版本 19.1.13, 2019-03-12 {#clickhouse-release-19-1-13-2019-03-12} - -此版本包含与 19.3.7 完全相同的补丁集. - -### ClickHouse 版本 19.1.10, 2019-03-03 {#clickhouse-release-19-1-10-2019-03-03} - -此版本包含与 19.3.6 完全相同的补丁集 - -## ClickHouse 版本 19.1 {#clickhouse-release-19-1-1} - -### ClickHouse 版本 19.1.9, 2019-02-21 {#clickhouse-release-19-1-9-2019-02-21} - -#### Bug 修复 {#bug-fixes-18} - -- 修复了由于 `send_logs_level` 设置的错误实现而导致的与旧版本的向后不兼容. [#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了使用列注释引入的表函数 `remote` 向后不兼容的问题. [#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.1.8, 2019-02-16 {#clickhouse-release-19-1-8-2019-02-16} - -#### Bug 修复 {#bug-fixes-19} - -- 修复缺少 /etc/clickhouse-server/config.xml 的安装包. [#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) - -## ClickHouse 版本 19.1 {#clickhouse-release-19-1-2} - -### ClickHouse 版本 19.1.7, 2019-02-15 {#clickhouse-release-19-1-7-2019-02-15} - -#### Bug 修复 {#bug-fixes-20} - -- 正确返回正确的类型并正确处理`joinGet`函数中的锁. [#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) -- 修复了在服务器关闭时尝试再次创建系统日志时的错误. [#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复错误: 如果有一个带有 `Dictionary` 引擎的数据库, 所有词典都在服务器启动时强制加载, 如果有来自localhost的带有ClickHouse源的词典, 则该词典无法加载. [#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了执行包含 `IN` 运算符的突变时产生错误结果的错误. [#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Alex Zatelepin](https://github.com/ztlpn)) -- 如果以交互模式运行, `clickhouse-client` 可以在退出时加载命令行建议数据时出现段错误. [#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 从 `system.tables` 中选择时固定的竞争条件可能会给出“表不存在”错误. [#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了在 `No such file or directory` 错误后重试带有 `File` 引擎的表中的 `SELECT` 时的死锁. [#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了一个问题: 本地 ClickHouse 词典通过 TCP 加载, 但应在进程内加载. [#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了通过 TLS 连接与 PostgreSQL ODBC 驱动程序交互时的 `No message received` 错误. 还修复了使用 MySQL ODBC 驱动程序时的段错误. [#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 暂时禁用`ORDER BY` 的谓词优化. [#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Winter Zhang](https://github.com/zhang2014)) -- 从表函数 `numbers(0)` 中选择时修复了无限循环. [#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `compile_expressions` 与大 (超过 int16) 日期比较的错误. [#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) -- 修复了 `uncompressed_cache=1` 的分段错误和错误的未压缩大小的异常. [#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) -- 修复了右表中有重复项的 `ALL JOIN`. [#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) -- 修复了在执行 `INSERT ... SELECT ... FROM file(...)` 查询和文件具有 `CSVWithNames` 或 `TSVWIthNames` 格式并且缺少第一个数据行时的错误行为. [#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用 `Array(LowCardinality)` 参数修复了聚合函数的执行. [#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- Debian 软件包: 根据配置更正 /etc/clickhouse-server/preprocessed 链接. [#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) -- 修复了未定义行为消毒剂下的模糊测试: 添加了对 `quantile*Weighted` 系列函数的参数类型检查. [#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使 `START REPLICATED SENDS` 命令开始复制发送. [#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) -- 修复了 JOIN ON 部分中重复列的 `Not found column`. [#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) -- 现在 `/etc/ssl` 被用作 SSL 证书的默认目录. [#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了字典不可用时重新加载字典时崩溃的问题. [#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([proller](https://github.com/proller)) -- 修复了错误的 `Date` 和`DateTime` 比较错误. [#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) -- 修复了在条件运算符(函数 `if`) 的分支中使用 `Date` 和 `DateTime` 参数时不正确的结果. 为函数 `if` 添加了通用情况. [#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### ClickHouse 版本 19.1.6, 2019-01-24 {#clickhouse-release-19-1-6-2019-01-24} - -#### 新特征 {#new-features-7} - -- 表的自定义每列压缩编解码器. [#3899](https://github.com/ClickHouse/ClickHouse/pull/3899) [#4111](https://github.com/ClickHouse/ClickHouse/pull/4111) ([alesapin](https://github.com/alesapin), [Winter Zhang](https://github.com/zhang2014), [Anatoly](https://github.com/Sindbag)) -- 添加了压缩编解码器 `Delta`. [#4052](https://github.com/ClickHouse/ClickHouse/pull/4052) ([alesapin](https://github.com/alesapin)) -- 允许`ALTER`压缩编解码器. [#4054](https://github.com/ClickHouse/ClickHouse/pull/4054) ([alesapin](https://github.com/alesapin)) -- 添加函数`left`、`right`、`trim`、`ltrim`、`rtrim`、`timestampadd`、`timestampsub` 以实现SQL标准兼容性. [#3826](https://github.com/ClickHouse/ClickHouse/pull/3826) ([Ivan Blinkov](https://github.com/blinkov)) -- 支持写入`HDFS`表和`hdfs`表功能. [#4084](https://github.com/ClickHouse/ClickHouse/pull/4084) ([alesapin](https://github.com/alesapin)) -- 添加了从大型 haystack 中搜索多个常量字符串的函数: `multiPosition`、`multiSearch`、`firstMatch` 以及 `-UTF8`、`-CaseInsensitive` 和 `-CaseInsensitiveUTF8` 变体. [#4053](https://github.com/ClickHouse/ClickHouse/pull/4053) ([Danila Kutenin](https://github.com/danlark1)) -- 如果`SELECT` 查询通过分片键进行过滤, 则修剪未使用的分片 (设置`optimize_skip_unused_shards`). [#3851](https://github.com/ClickHouse/ClickHouse/pull/3851) ([Gleb Kanterov](https://github.com/kanterov), [Ivan](https://github.com/abyss7)) -- 允许 `Kafka` 引擎忽略每个块的一些解析错误. [#4094](https://github.com/ClickHouse/ClickHouse/pull/4094) ([Ivan](https://github.com/abyss7)) -- 添加了对`CatBoost` 多类模型评估的支持。 函数`modelEvaluate` 返回带有多类模型的每类原始预测的元组。 `libcatboostmodel.so` 应该使用 [#607](https://github.com/catboost/catboost/pull/607) 构建. [#3959](https://github.com/ClickHouse/ClickHouse/pull/3959) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 添加函数 `filesystemAvailable`、`filesystemFree`、`filesystemCapacity`. [#4097](https://github.com/ClickHouse/ClickHouse/pull/4097) ([Boris Granveaud](https://github.com/bgranvea)) -- 添加了散列函数 `xxHash64` 和 `xxHash32`. [#3905](https://github.com/ClickHouse/ClickHouse/pull/3905) ([filimonov](https://github.com/filimonov)) -- 添加了 `gccMurmurHash` 散列函数 (GCC 风味的 Murmur 散列), 它使用与 [gcc] (https://github.com/gcc-mirror/gcc/blob/41d6b10e96a1de98e90a7c0378437c3255814b16/libstdc%2B%2B-v3/include/bits/functional_hash.h#L191)相同的散列种子. [#4000](https://github.com/ClickHouse/ClickHouse/pull/4000) ([sundyli](https://github.com/sundy-li)) -- 添加了散列函数 `javaHash`、`hiveHash`. [#3811](https://github.com/ClickHouse/ClickHouse/pull/3811) ([shangshujie365](https://github.com/shangshujie365)) -- 添加了表功能`remoteSecure`. 功能作为 `remote` 工作,但使用安全连接. [#4088](https://github.com/ClickHouse/ClickHouse/pull/4088) ([proller](https://github.com/proller)) - -#### 实验功能 {#experimental-features-3} - -- 添加了多个 JOIN 仿真 (`allow_experimental_multiple_joins_emulation` 设置). [#3946](https://github.com/ClickHouse/ClickHouse/pull/3946) ([Artem Zuikov](https://github.com/4ertus2)) - -#### Bug 修复 {#bug-fixes-21} - -- 将 `compiled_expression_cache_size` 设置限制为默认以降低内存消耗. [#4041](https://github.com/ClickHouse/ClickHouse/pull/4041) ([alesapin](https://github.com/alesapin)) -- 修复导致执行复制表 ALTER 的线程和从 ZooKeeper 更新配置的线程挂起的错误. [#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [#3891](https://github.com/ClickHouse/ClickHouse/issues/3891) [#3934](https://github.com/ClickHouse/ClickHouse/pull/3934) ([Alex Zatelepin](https://github.com/ztlpn)) -- 修复了执行分布式 ALTER 任务时的竞争条件. 竞争条件导致多个副本尝试执行任务, 所有副本除了一个因 ZooKeeper 错误而失败. [#3904](https://github.com/ClickHouse/ClickHouse/pull/3904) ([Alex Zatelepin](https://github.com/ztlpn)) -- 修复了在对 ZooKeeper 的请求超时后不刷新 `from_zk` 配置元素的错误. [#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [#3947](https://github.com/ClickHouse/ClickHouse/pull/3947) ([Alex Zatelepin](https://github.com/ztlpn)) -- 修复错误前缀的错误 IPv4 子网掩码. [#3945](https://github.com/ClickHouse/ClickHouse/pull/3945) ([alesapin](https://github.com/alesapin)) -- 修复了由于资源耗尽而无法创建新线程的极少数情况下的崩溃 (`std::terminate`). [#3956](https://github.com/ClickHouse/ClickHouse/pull/3956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复在 `getStructureOfRemoteTable` 中使用错误限制时在 `remote` 表函数执行中的错误. [#4009](https://github.com/ClickHouse/ClickHouse/pull/4009) ([alesapin](https://github.com/alesapin)) -- 修复 netlink 套接字泄漏. 它们被放置在一个永远不会被删除的池中, 并且当所有当前套接字都在使用时, 在新线程开始时创建新套接字. [#4017](https://github.com/ClickHouse/ClickHouse/pull/4017) ([Alex Zatelepin](https://github.com/ztlpn)) -- 修复在分支 `odbc-bridge` 子进程后, 在从 `/proc` 读取所有 fds 之前关闭 `/proc/self/fd` 目录的错误. [#4120](https://github.com/ClickHouse/ClickHouse/pull/4120) ([alesapin](https://github.com/alesapin)) -- 在主键中使用字符串的情况下修复了字符串到 UInt 的单调转换. [#3870](https://github.com/ClickHouse/ClickHouse/pull/3870) ([Winter Zhang](https://github.com/zhang2014)) -- 修复整数转换函数单调性计算错误. [#3921](https://github.com/ClickHouse/ClickHouse/pull/3921) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `arrayEnumerateUniq`、`arrayEnumerateDense` 函数中某些无效参数的段错误. [#3909](https://github.com/ClickHouse/ClickHouse/pull/3909) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复 StorageMerge 中的 UB. [#3910](https://github.com/ClickHouse/ClickHouse/pull/3910) ([Amos Bird](https://github.com/amosbird)) -- 修复了函数 `addDays`、`subtractDays` 中的段错误. [#3913](https://github.com/ClickHouse/ClickHouse/pull/3913) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复错误: 函数`round`、`floor`、`trunc`、`ceil` 在整数参数和大的负比例上执行时可能返回虚假结果. [#3914](https://github.com/ClickHouse/ClickHouse/pull/3914) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了由 `kill query sync` 引起的导致核心转储的错误. [#3916](https://github.com/ClickHouse/ClickHouse/pull/3916) ([muVulDeePecker](https://github.com/fancyqlx)) -- 修复空复制队列后长时间延迟的错误. [#3928](https://github.com/ClickHouse/ClickHouse/pull/3928) [#3932](https://github.com/ClickHouse/ClickHouse/pull/3932) ([alesapin](https://github.com/alesapin)) -- 修复了使用 `LowCardinality` 主键插入表时过多的内存使用. [#3955](https://github.com/ClickHouse/ClickHouse/pull/3955) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 在空数组的情况下修复了`Native`格式的`LowCardinality`序列化. [#3907](https://github.com/ClickHouse/ClickHouse/issues/3907) [#4011](https://github.com/ClickHouse/ClickHouse/pull/4011) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 修复了使用单 LowCardinality 数字列不同时的错误结果. [#3895](https://github.com/ClickHouse/ClickHouse/issues/3895) [#4012](https://github.com/ClickHouse/ClickHouse/pull/4012) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 使用 LowCardinality 键修复了专门的聚合 (如果启用了 `compile` 设置). [#3886](https://github.com/ClickHouse/ClickHouse/pull/3886) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 修复复制表查询的用户和密码转发. [#3957](https://github.com/ClickHouse/ClickHouse/pull/3957) ([alesapin](https://github.com/alesapin)) ([小路](https://github.com/nicelulu)) -- 修复了在重新加载字典时列出字典数据库中的表时可能发生的非常罕见的竞争条件. [#3970](https://github.com/ClickHouse/ClickHouse/pull/3970) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了当 HAVING 与 ROLLUP 或 CUBE 一起使用时的错误结果. [#3756](https://github.com/ClickHouse/ClickHouse/issues/3756) [#3837](https://github.com/ClickHouse/ClickHouse/pull/3837) ([Sam Chou](https://github.com/reflection)) -- 使用 `JOIN ON` 语法和分布式表进行查询的固定列别名. [#3980](https://github.com/ClickHouse/ClickHouse/pull/3980) ([Winter Zhang](https://github.com/zhang2014)) -- 修复了 `quantileTDigest` (由 Artem Vakhrushev 发现) 的内部实现中的错误. 此错误从未发生在 ClickHouse 中, 仅与直接使用 ClickHouse 代码库作为库的人相关. [#3935](https://github.com/ClickHouse/ClickHouse/pull/3935) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 改进 {#improvements-6} - -- 支持 `ALTER TABLE ADD COLUMN` 语句中的 `IF NOT EXISTS` 以及 `DROP/MODIFY/CLEAR/COMMENT COLUMN` 中的 `IF EXISTS`. [#3900](https://github.com/ClickHouse/ClickHouse/pull/3900) ([Boris Granveaud](https://github.com/bgranvea)) -- 函数 `parseDateTimeBestEffort`: 支持`DD.MM.YYYY`、`DD.MM.YY`、`DD-MM-YYYY`、`DD-Mon-YYYY`、`DD/Month/YYYY`等格式. [#3922](https://github.com/ClickHouse/ClickHouse/pull/3922) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- `CapnProtoInputStream` 现在支持锯齿状结构. [#4063](https://github.com/ClickHouse/ClickHouse/pull/4063) ([Odin Hultgren Van Der Horst](https://github.com/Miniwoffer)) -- 可用性改进: 添加了检查服务器进程是否从数据目录的所有者启动. 如果数据属于非 root 用户, 则不允许从 root 启动服务器. [#3785](https://github.com/ClickHouse/ClickHouse/pull/3785) ([sergey-v-galtsev](https://github.com/sergey-v-galtsev)) -- 在使用 JOIN 分析查询期间检查所需列的更好逻辑. [#3930](https://github.com/ClickHouse/ClickHouse/pull/3930) ([Artem Zuikov](https://github.com/4ertus2)) -- 在单个服务器中有大量分布式表的情况下减少连接数. [#3726](https://github.com/ClickHouse/ClickHouse/pull/3726) ([Winter Zhang](https://github.com/zhang2014)) -- 支持 ODBC 驱动程序 `WITH TOTALS` 查询的总计行. [#3836](https://github.com/ClickHouse/ClickHouse/pull/3836) ([Maksim Koritckiy](https://github.com/nightweb)) -- 允许在 if 函数中使用 `Enum` 作为整数. [#3875](https://github.com/ClickHouse/ClickHouse/pull/3875) ([Ivan](https://github.com/abyss7)) -- 添加了 `low_cardinality_allow_in_native_format` 设置. 如果禁用, 请勿使用 `Native` 格式的 `LowCadrinality` 类型. [#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) ([KochetovNicolai](https://github.com/KochetovNicolai)) -- 从已编译的表达式缓存中删除了一些冗余对象以降低内存使用率. [#4042](https://github.com/ClickHouse/ClickHouse/pull/4042) ([alesapin](https://github.com/alesapin)) -- 添加检查 `SET send_logs_level = 'value'` 查询接受适当的值. [#3873](https://github.com/ClickHouse/ClickHouse/pull/3873) ([Sabyanin Maxim](https://github.com/s-mx)) -- 类型转换函数中的固定数据类型检查. [#3896](https://github.com/ClickHouse/ClickHouse/pull/3896) ([Winter Zhang](https://github.com/zhang2014)) - -#### 性能改进 {#performance-improvements-5} - -- 添加 MergeTree 设置 `use_minimalistic_part_header_in_zookeeper`. 如果启用,复制表将在单个部分 znode 中存储压缩部分元数据. 这可以显着减少 ZooKeeper 快照大小 (特别是如果表有很多列). 请注意, 启用此设置后, 您将无法降级到不支持它的版本. [#3960](https://github.com/ClickHouse/ClickHouse/pull/3960) ([Alex Zatelepin](https://github.com/ztlpn)) -- 为函数 `sequenceMatch` 和 `sequenceCount` 添加基于 DFA 的实现,以防模式不包含时间. [#4004](https://github.com/ClickHouse/ClickHouse/pull/4004) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) -- 整数序列化的性能改进. [#3968](https://github.com/ClickHouse/ClickHouse/pull/3968) ([Amos Bird](https://github.com/amosbird)) -- 零左填充 PODArray 以便 -1 元素始终有效并归零. 它用于偏移量的无分支计算. [#3920](https://github.com/ClickHouse/ClickHouse/pull/3920) ([Amos Bird](https://github.com/amosbird)) -- 恢复了导致性能下降的 `jemalloc` 版本. [#4018](https://github.com/ClickHouse/ClickHouse/pull/4018) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 向后不兼容更改 {#backward-incompatible-changes-2} - -- 删除了未记录的特性 `ALTER MODIFY PRIMARY KEY`, 因为它被 `ALTER MODIFY ORDER BY` 命令取代. [#3887](https://github.com/ClickHouse/ClickHouse/pull/3887) ([Alex Zatelepin](https://github.com/ztlpn)) -- 删除了函数 `shardByHash`. [#3833](https://github.com/ClickHouse/ClickHouse/pull/3833) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 禁止使用类型为 `AggregateFunction` 的标量子查询. [#3865](https://github.com/ClickHouse/ClickHouse/pull/3865) ([Ivan](https://github.com/abyss7)) - -#### 构建/测试/打包改进 {#buildtestingpackaging-improvements-6} - -- 添加了对 PowerPC (`ppc64le`) 构建的支持. [#4132](https://github.com/ClickHouse/ClickHouse/pull/4132) ([Danila Kutenin](https://github.com/danlark1)) -- 在公共可用数据集上运行状态功能测试. [#3969](https://github.com/ClickHouse/ClickHouse/pull/3969) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了当服务器无法以 Docker 或 systemd-nspawn 中的 `bash: /usr/bin/clickhouse-extract-from-config: Operation not allowed` 消息启动时的错误. [#4136](https://github.com/ClickHouse/ClickHouse/pull/4136) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将 `rdkafka` 库更新到 v1.0.0-RC5. 使用 cppkafka 而不是原始 C 接口. [#4025](https://github.com/ClickHouse/ClickHouse/pull/4025) ([Ivan](https://github.com/abyss7)) -- 更新了 `mariadb-client` 库. 修复了 UBSan 发现的问题之一. [#3924](https://github.com/ClickHouse/ClickHouse/pull/3924) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- UBSan 版本的一些修复. [#3926](https://github.com/ClickHouse/ClickHouse/pull/3926) [#3021](https://github.com/ClickHouse/ClickHouse/pull/3021) [#3948](https://github.com/ClickHouse/ClickHouse/pull/3948) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用 UBSan 构建添加了每次提交的测试运行. -- 添加了 PVS-Studio 静态分析器的每次提交运行. -- 修复了 PVS-Studio 发现的错误. [#4013](https://github.com/ClickHouse/ClickHouse/pull/4013) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 glibc 兼容性问题. [#4100](https://github.com/ClickHouse/ClickHouse/pull/4100) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 将 Docker 镜像移至 18.10 并添加 glibc \>= 2.28 的兼容性文件. [#3965](https://github.com/ClickHouse/ClickHouse/pull/3965) ([alesapin](https://github.com/alesapin)) -- 如果用户不想在服务器 Docker 映像中 chown 目录,请添加 env 变量. [#3967](https://github.com/ClickHouse/ClickHouse/pull/3967) ([alesapin](https://github.com/alesapin)) -- 在 clang 中启用了来自 `-Weverything` 的大部分警告. 启用 `-Wpedantic` . [#3986](https://github.com/ClickHouse/ClickHouse/pull/3986) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 添加了一些仅在 clang 8 中可用的警告. [#3993](https://github.com/ClickHouse/ClickHouse/pull/3993) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 使用共享链接时链接到 `libLLVM` 而不是单个 LLVM 库. [#3989](https://github.com/ClickHouse/ClickHouse/pull/3989) ([Orivej Desh](https://github.com/orivej)) -- 测试图像添加了清除变量. [#4072](https://github.com/ClickHouse/ClickHouse/pull/4072) ([alesapin](https://github.com/alesapin)) -- `clickhouse-server` debian 包会推荐 `libcap2-bin` 包使用 `setcap` 工具设置功能. 这是可选的. [#4093](https://github.com/ClickHouse/ClickHouse/pull/4093) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 改进了编译时间, 修复了包含. [#3898](https://github.com/ClickHouse/ClickHouse/pull/3898) ([proller](https://github.com/proller)) -- 添加了哈希函数的性能测试. [#3918](https://github.com/ClickHouse/ClickHouse/pull/3918) ([filimonov](https://github.com/filimonov)) -- 修复了循环库依赖. [#3958](https://github.com/ClickHouse/ClickHouse/pull/3958) ([proller](https://github.com/proller)) -- 改进了低可用内存的编译. [#4030](https://github.com/ClickHouse/ClickHouse/pull/4030) ([proller](https://github.com/proller)) -- 添加了测试脚本以重现 `jemalloc` 中的性能下降. [#4036](https://github.com/ClickHouse/ClickHouse/pull/4036) ([alexey-milovidov](https://github.com/alexey-milovidov)) -- 修复了 `dbms` 下注释和字符串中的拼写错误. [#4122](https://github.com/ClickHouse/ClickHouse/pull/4122) ([maiha](https://github.com/maiha)) -- 修正了评论中的错别字. [#4089](https://github.com/ClickHouse/ClickHouse/pull/4089) ([Evgenii Pravda](https://github.com/kvinty)) - -## [Changelog for 2018](../../whats-new/changelog/2018.md#clickhouse-release-18-16) {#changelog-for-2018} diff --git a/docs/zh/whats-new/changelog/2020.md b/docs/zh/whats-new/changelog/2020.md deleted file mode 100644 index 30531d74030..00000000000 --- a/docs/zh/whats-new/changelog/2020.md +++ /dev/null @@ -1,3529 +0,0 @@ ---- -slug: /zh/whats-new/changelog/2020 ---- -### ClickHouse 版本 20.12 - -### ClickHouse 版本 v20.12.5.14-stable, 2020-12-28 - -#### Bug 修复 - -* 在合并期间禁用 AIO 写入, 因为它会导致合并期间主键列的数据损坏极为罕见. [#18481](https://github.com/ClickHouse/ClickHouse/pull/18481) ([alesapin](https://github.com/alesapin)). -* 修复了在使用类型为 `Nullable(String)` 的参数执行 `toType(...)` 函数 (`toDate`、`toUInt32` 等)时出现的 `value is too short` 错误. 现在这些函数在解析错误时返回 `NULL` 而不是抛出异常. 修复 [#18445](https://github.com/ClickHouse/ClickHouse/pull/18445) ([tavplubix](https://github.com/tavplubix)). -* 限制从宽部分合并到紧凑部分. 在垂直合并的情况下, 它会导致结果部分损坏. [#18381](https://github.com/ClickHouse/ClickHouse/pull/18381) ([Anton Popov](https://github.com/CurtizJ)). -* 修复填充表 `system.settings_profile_elements`. 此 PR 修复了 [#18231](https://github.com/ClickHouse/ClickHouse/issues/18231). [#18379](https://github.com/ClickHouse/ClickHouse/pull/18379) ([Vitaly Baranov](https://github.com/vitlibar)). -* 使用组合子 `Distinct` 修复聚合函数中可能的崩溃, 同时使用两级聚合. 修复 [#17682](https://github.com/ClickHouse/ClickHouse/issues/17682). [#18365](https://github.com/ClickHouse/ClickHouse/pull/18365) ([Anton Popov](https://github.com/CurtizJ)). -* 修复查询 `MODIFY COLUMN ... REMOVE TTL` 并没有真正删除列 TTL 时的错误. [#18130](https://github.com/ClickHouse/ClickHouse/pull/18130) ([alesapin](https://github.com/alesapin)). - -#### 构建/测试/打包改进 - -* 将时区信息更新为 2020e. [#18531](https://github.com/ClickHouse/ClickHouse/pull/18531) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse 版本 v20.12.4.5-stable, 2020-12-24 - -#### Bug 修复 - -* 修复了在具有双 IPv4/IPv6 堆栈的机器上服务器无法访问 `clickhouse-odbc-bridge` 进程的问题; - 修复了使用格式错误的查询执行 ODBC 字典更新和/或导致崩溃的问题; 可能关闭 [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* 修复了 Enum 和 Int 类型之间的键比较. 这修复了 [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `MaterializeMySQL` 数据库引擎中唯一键转换崩溃的问题. 这修复了 [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) 并修复了 [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 S3 URL 解析中的 `std::out_of_range: basic_string`. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复了由于 MaterializeMySQL 不支持转换 MySQL 前缀索引而导致某些表无法从 MySQL 同步到 ClickHouse 的问题. 这修复了 [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) 并修复了 [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了查询包含 `ARRAY JOIN` 时查询优化产生错误结果的问题. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* 修复了 `topK` 聚合函数中可能出现的段错误. 这将关闭 [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* 修复了服务器在守护进程模式下运行时空的 `system.stack_trace` 表. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). - - -### ClickHouse 版本 v20.12.3.3-stable, 2020-12-13 - -#### 向后不兼容变更 - -* 默认启用`use_compact_format_in_distributed_parts_names` (参考文档). [#16728](https://github.com/ClickHouse/ClickHouse/pull/16728) ([Azat Khuzhin](https://github.com/azat)). -* 创建使用 `File` 引擎的表时, 在 `SETTINGS` 子句中接受与文件格式 (例如 `format_csv_delimiter` ) 相关的用户设置, 并在所有 `INSERT` 和 `SELECT` 中使用这些设置. 当前用户会话中或 DML 查询本身的 `SETTINGS` 子句中更改的文件格式设置不再影响查询. [#16591](https://github.com/ClickHouse/ClickHouse/pull/16591) ([Alexander Kuzmenkov](https://github.com/akuzm)). - -#### 新特征 - -* 添加 `*.xz` 压缩/解压支持。它允许在 `file()` 函数中使用 `*.xz`. 这将关闭 [#8828](https://github.com/ClickHouse/ClickHouse/issues/8828). [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)). -* 引入查询`ALTER TABLE ... DROP|DETACH PART 'part_name'`. [#15511](https://github.com/ClickHouse/ClickHouse/pull/15511) ([nvartolomei](https://github.com/nvartolomei)). -* 添加了新的 ALTER UPDATE/DELETE IN PARTITION 语法. [#13403](https://github.com/ClickHouse/ClickHouse/pull/13403) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 允许在使用 JSON 输入/输出格式时将命名元组格式化为 JSON 对象, 由 `output_format_json_named_tuples_as_objects` 设置控制, 默认禁用. [#17175](https://github.com/ClickHouse/ClickHouse/pull/17175) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 在TSV和CSV格式默认情况下添加一种可能的输入枚举值作为它的id. [#16834](https://github.com/ClickHouse/ClickHouse/pull/16834) ([Kruglov Pavel](https://github.com/Avogar)). -* 添加对 Nullable、LowCardinality、Array 和 Tuple 的 COLLATE 支持,其中嵌套类型为 String. 同时重构ColumnString.cpp中与collations相关的代码. [#16273](https://github.com/ClickHouse/ClickHouse/pull/16273) ([Kruglov Pavel](https://github.com/Avogar)). -* 新的 `tcpPort` 函数返回此服务器监听的 TCP 端口. [#17134](https://github.com/ClickHouse/ClickHouse/pull/17134) ([Ivan](https://github.com/abyss7)). -* 添加新的数学函数:`acosh`、`asinh`、`atan2`、`atanh`、`cosh`、`hypot`、`log1p`、`sinh`. [#16636](https://github.com/ClickHouse/ClickHouse/pull/16636) ([Konstantin Malanchev](https://github.com/hombit)). -* 在不同副本之间分配合并的可能性. 引入 `execute_merges_on_single_replica_time_threshold` 合并树设置. [#16424](https://github.com/ClickHouse/ClickHouse/pull/16424) ([filimonov](https://github.com/filimonov)). -* 添加设置 `aggregate_functions_null_for_empty` 以实现 SQL 标准兼容性. 此选项将重写查询中的所有聚合函数, 并向它们添加 -OrNull 后缀. 实现 [10273](https://github.com/ClickHouse/ClickHouse/issues/10273). [#16123](https://github.com/ClickHouse/ClickHouse/pull/16123) ([flynn](https://github.com/ucasFL)). -* 更新了 DateTime、DateTime64 解析以接受字符串日期文字格式. [#16040](https://github.com/ClickHouse/ClickHouse/pull/16040) ([Maksim Kita](https://github.com/kitaisreal)). -* 可以使用 `--history_file` 参数在 `clickhouse-client` 中更改历史文件的路径. [#15960](https://github.com/ClickHouse/ClickHouse/pull/15960) ([Maksim Kita](https://github.com/kitaisreal)). - -#### Bug 修复 - -* 修复在极少数情况下服务器可以停止接受连接的问题. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([Amos Bird](https://github.com/amosbird)). -* 修复了在 Linux 的 Windows 子系统上运行的 ClickHouse 在 `Atomic` 数据库中执行 `RENAME` 查询时出现的 `Function not implementation` 错误. 修复 [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). -* 如果禁用了`in_memory_parts_enable_wal`, 则不要从 WAL 恢复部件. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* 使用 `min_compress_block_size` 修复 MergeTreeWriterSettings 的 `max_compress_block_size` 的错误初始化. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). -* 有关要删除的最大表大小的异常消息显示不正确. [#17764](https://github.com/ClickHouse/ClickHouse/pull/17764) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了插入 `Distributed` 表时空间不足时可能出现的段错误. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). -* 修复了 ClickHouse 无法恢复与 MySQL 服务器的连接时的问题. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* 当 `pool_size` > 1 时, 由于竞争条件, 在执行 `ON CLUSTER` 查询时, 可能会错误地确定集群是否为循环 (交叉) 复制. 已修复. [#17640](https://github.com/ClickHouse/ClickHouse/pull/17640) ([tavplubix](https://github.com/tavplubix)). -* 异常 `fmt::v7::format_error` 可以记录在 MergeTree 表的后台. 这修复了 [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当 clickhouse-client 在交互模式下与多行查询一起使用时, 单行注释被错误地扩展到查询结束. 这修复了 [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当相应的突变在不同的副本上被杀死时, 修复更改查询挂起. 修复 [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* 修复了 clickhouse 低估了标记缓存大小的问题. 当有很多带有标记的小文件时可能会发生. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* 启用设置 `optimize_redundant_functions_in_order_by` 来修复 `ORDER BY`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 `DISTINCT` 之后由于不正确优化而可能出现的重复项. 修复 [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复从具有 `LowCardinality` 类型的 `JOIN` 表读取时崩溃的问题. 修复 [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `toInt256(inf)` 堆栈溢出. Int256 是一项实验性功能. 已关闭 [#17235](https://github.com/ClickHouse/ClickHouse/issues/17235). [#17257](https://github.com/ClickHouse/ClickHouse/pull/17257) ([flynn](https://github.com/ucasFL)). -* 使用 `LIMIT` 修复分布式查询中可能记录的 `Unexpected packet Data received from client` 错误. [#17254](https://github.com/ClickHouse/ClickHouse/pull/17254) ([Azat Khuzhin](https://github.com/azat)). -* 修复子查询中存在 const 列时设置索引失效的问题. 这修复了 [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* 修复索引比较类型不同时可能出现的错误索引分析. 这修复了 [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). -* 修复 ColumnConst 比较导致崩溃. 这个修复了 [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088). [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* MaterializeMySQL (实验功能) 的多个修复. 修复 [#16923](https://github.com/ClickHouse/ClickHouse/issues/16923) 修复 [#15883](https://github.com/ClickHouse/ClickHouse/issues/15883) 修复 MaterializeMySQL 同步失败时修改 MySQL binlog_checksum. [#17091](https://github.com/ClickHouse/ClickHouse/pull/17091) ([Winter Zhang](https://github.com/zhang2014)). -* 修复非领导者 ReplicatedMergeTree 表的 `ON CLUSTER` 查询可能永远挂起的错误. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* 修复了在创建 `some_table` 时 `CREATE TABLE ... AS some_table` 查询崩溃 `AS table_function()` 修复. [#17072](https://github.com/ClickHouse/ClickHouse/pull/17072) ([tavplubix](https://github.com/tavplubix)). -* 功能模糊位的错误未完成的实现, 相关问题: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* 在 CFA 寄存器为 RAX 的情况下修复 LLVM 的 libunwind. 这是 [LLVM's libunwind](https://github.com/llvm/llvm-project/tree/master/libunwind) 中的 [bug](https://bugs.llvm.org/show_bug.cgi?id=48186) ). 我们已经有针对此错误的解决方法. [#17046](https://github.com/ClickHouse/ClickHouse/pull/17046) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 避免远程查询的不必要的网络错误, 这些错误在执行时可能会被取消, 例如使用 `LIMIT` 的查询. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* 修复 `optimize_distributed_group_by_sharding_key` 设置 (默认情况下禁用) 以仅使用 OFFSET 进行查询. [#16996](https://github.com/ClickHouse/ClickHouse/pull/16996) ([Azat Khuzhin](https://github.com/azat)). -* 修复了使用 JOIN 在分布式表上合并表的问题. [#16993](https://github.com/ClickHouse/ClickHouse/pull/16993) ([Azat Khuzhin](https://github.com/azat)). -* 修复了从 double 转换时大整数 (128、256 位) 的错误结果。 大整数支持是实验性的. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). -* 修复在 `ALTER TABLE ... MODIFY COLUMN ... NewType` 之后可能发生的服务器崩溃, 当 `SELECT` 在更改列上具有 `WHERE` 表达式并且更改尚未完成时. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* `clickhouse-git-import` 中没有正确计算批判信息. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过使用单调函数进行优化来修复顺序. 修复 [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). -* 通过启用 `optimize_aggregators_of_group_by_keys` 和 join 设置来修复组的优化. 修复 [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* 修复使用 `ORDER BY` 的查询可能出现的错误 `Illegal type of argument` . 修复 [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 InterpreterShowAccessQuery 中的奇怪代码. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). -* 使用函数 `timeSeriesGroupSum` 时防止clickhouse 服务器崩溃. 该功能已从较新的 ClickHouse 版本中删除. [#16865](https://github.com/ClickHouse/ClickHouse/pull/16865) ([filimonov](https://github.com/filimonov)). -* 当查询分析器打开并且 ClickHouse 安装在操作系统上时, 修复罕见的无声崩溃, glibc 版本已经 (据说) 损坏了某些功能的异步展开表. 这修复了 [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复在不带任何参数的情况下使用 `any` 时的崩溃。 这是针对 [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803)的 . cc @azat. [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* 如果在磁盘上写入表元数据时无法分配内存, 则可以写入损坏的元数据文件. [#16772](https://github.com/ClickHouse/ClickHouse/pull/16772) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用分区谓词修复琐碎的查询优化. [#16767](https://github.com/ClickHouse/ClickHouse/pull/16767) ([Azat Khuzhin](https://github.com/azat)). -* 在启用了 `transform_null_in` 设置的情况下修复多列和元组上的 `IN` 运算符. 修复 [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* 通过 MySQL 协议返回 INSERT 查询的受影响行数. 以前 ClickHouse 过去总是返回 0, 现在已修复. 修复 [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* 修复使用 `if` 后缀聚合函数时远程查询失败的问题. 修复 [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) Fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). -* 修复由 `select_sequential_consistency` 引起的不一致行为, 用于优化的琐碎计数查询和 system.tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). - -#### 改进 - -* 删除被 TTL、变异或折叠合并算法修剪后的空部分. [#16895](https://github.com/ClickHouse/ClickHouse/pull/16895) ([Anton Popov](https://github.com/CurtizJ)). -* 为分布式表中的异步发送启用紧凑格式的目录: `use_compact_format_in_distributed_parts_names` 默认设置为 1. [#16788](https://github.com/ClickHouse/ClickHouse/pull/16788) ([Azat Khuzhin](https://github.com/azat)). -* 如果没有数据写入 S3, 则中止分段上传. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* 重新解析 `format_avro_schema_registry_url` 的IP. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* 在 system.distribution_queue 中的 data_path 中屏蔽密码. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). -* 使用列转换器替换不存在的列时抛出错误. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). -* 当没有足够的内存供所有线程同时工作时关闭并行解析. 当有人试图插入非常大的行 (> min_chunk_bytes_for_parallel_parsing) 时, 也可能有 `Memory limit exceeded` 之类的例外, 因为要解析的每个部分都必须是独立的一组字符串 (一个或多个). [#16721](https://github.com/ClickHouse/ClickHouse/pull/16721) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 安装脚本应始终在 config 文件夹中创建子目录. 这仅与使用自定义配置的 Docker 构建相关. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* 纠正 JSONEachRow、JSONCompactEachRow 和 RegexpRow 输入格式的错误消息中的语法. [#17205](https://github.com/ClickHouse/ClickHouse/pull/17205) ([nico piderman](https://github.com/sneako)). -* 将 `SOURCE(CLICKHOUSE(...))` 的默认 `host` 和 `port` 参数设置为当前实例,并将默认的 `user` 值设置为 `default`. [#16997](https://github.com/ClickHouse/ClickHouse/pull/16997) ([vdimir](https://github.com/vdimir)). -* 执行`ATTACH/DETACH TABLE ` 时抛出信息性错误消息. 在此 PR 之前, `detach table ` 可以工作, 但会导致内存中元数据格式错误. [#16885](https://github.com/ClickHouse/ClickHouse/pull/16885) ([Amos Bird](https://github.com/amosbird)). -* 添加 cutToFirstSignificantSubdomainWithWWW(). [#16845](https://github.com/ClickHouse/ClickHouse/pull/16845) ([Azat Khuzhin](https://github.com/azat)). -* 如果给出错误的配置, 服务器拒绝启动并显示异常消息 (缺少`metric_log`.`collect_interval_milliseconds`). [#16815](https://github.com/ClickHouse/ClickHouse/pull/16815) ([Ivan](https://github.com/abyss7)). -* 不存在分布式 DDL 配置时更好的异常消息. 这修复了 [#5075](https://github.com/ClickHouse/ClickHouse/issues/5075). [#16769](https://github.com/ClickHouse/ClickHouse/pull/16769) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 可用性改进:当 `CODEC` 表达式在 `CREATE TABLE` 查询中错位时, 在语法错误消息中提供更好的建议. 这修复了 [#12493](https://github.com/ClickHouse/ClickHouse/issues/12493). [#16768](https://github.com/ClickHouse/ClickHouse/pull/16768) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在分布式引擎启动时删除异步插入的空目录. [#16729](https://github.com/ClickHouse/ClickHouse/pull/16729) ([Azat Khuzhin](https://github.com/azat)). -* 将 S3 与 nginx 服务器一起用作代理的解决方法. Nginx 当前不接受像 `http://domain.com?delete` 这样的空路径 url, 但是 vanilla aws-sdk-cpp 会产生这种 url. 此提交使用修补过的 aws-sdk-cpp 版本, 在这种情况下, 它使 url 以 "/" 作为路径, 例如 `http://domain.com/?delete`. [#16709](https://github.com/ClickHouse/ClickHouse/pull/16709) ([ianton-ru](https://github.com/ianton-ru)). -* 允许 `reinterpretAs*` 函数处理相同大小的整数和浮点数. 实现 [16640](https://github.com/ClickHouse/ClickHouse/issues/16640). [#16657](https://github.com/ClickHouse/ClickHouse/pull/16657) ([flynn](https://github.com/ucasFL)). -* 现在,`` 配置可以在 `config.xml` 中更改并在不启动服务器的情况下重新加载. [#16627](https://github.com/ClickHouse/ClickHouse/pull/16627) ([Amos Bird](https://github.com/amosbird)). -* 在到远程资源的 https 连接中支持 SNI. 这将允许连接到需要 SNI 的 Cloudflare 服务器. 这修复了 [#10055](https://github.com/ClickHouse/ClickHouse/issues/10055). [#16252](https://github.com/ClickHouse/ClickHouse/pull/16252) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 可以连接到需要 SNI 的 `clickhouse-server` 安全端点. 当 `clickhouse-server` 托管在 TLS 代理之后是可能的. [#16938](https://github.com/ClickHouse/ClickHouse/pull/16938) ([filimonov](https://github.com/filimonov)). -* 如果创建了物化视图循环, 则修复可能的堆栈溢出. 这将关闭 [#15732](https://github.com/ClickHouse/ClickHouse/issues/15732). [#16048](https://github.com/ClickHouse/ClickHouse/pull/16048) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 简化 MergeTree 表引擎系列的后台任务处理的实现. 用户应该没有明显的变化. [#15983](https://github.com/ClickHouse/ClickHouse/pull/15983) ([alesapin](https://github.com/alesapin)). -* MaterializeMySQL (实验功能) 的改进. 当 MySQL 同步用户有错误权限时抛出关于正确同步权限的异常. [#15977](https://github.com/ClickHouse/ClickHouse/pull/15977) ([TCeason](https://github.com/TCeason)). -* 使用 BloomFilter 生成 `indexOf()` . [#14977](https://github.com/ClickHouse/ClickHouse/pull/14977) ([achimbab](https://github.com/achimbab)). - -#### 性能改进 - -* 使用 Floyd-Rivest 算法, 它是 ClickHouse 部分排序用例的最佳选择. 基准在 https://github.com/danlark1/miniselect 和 [here](https://drive.google.com/drive/folders/1DHEaeXgZuX6AJ9eByeZ8iQVQv0ueP8XM). [#16825](https://github.com/ClickHouse/ClickHouse/pull/16825) ([Danila Kutenin](https://github.com/danlark1)). -* 现在, `ReplicatedMergeTree` 树引擎系列使用单独的线程池进行复制提取. 通过设置“background_fetches_pool_size”限制池的大小, 可以通过服务器重启进行调整. 设置的默认值是 3, 这意味着最大并行提取量等于 3 (允许使用 10G 网络) . 修复 #520. [#16390](https://github.com/ClickHouse/ClickHouse/pull/16390) ([alesapin](https://github.com/alesapin)). -* 修复了 `quantileTDigest` 状态的不受控制的增长. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). -* 将 `VIEW` 子查询描述添加到 `EXPLAIN`. 限制`VIEW`的下推优化. 将 `Distributed` 的本地副本添加到查询计划中. [#14936](https://github.com/ClickHouse/ClickHouse/pull/14936) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 max_threads > 0 和 ORDER BY 中的表达式修复 optimize_read_in_order/optimize_aggregation_in_order. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* 修复从 `Merge` 表读取大量 `MergeTree` 表的性能. 修复 [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). -* 现在我们可以安全地修剪具有精确匹配的分区. 有用的案例: 假设表由 `intHash64(x) % 100` 分区, 并且查询的条件是逐字的, 而不是 x. [#16253](https://github.com/ClickHouse/ClickHouse/pull/16253) ([Amos Bird](https://github.com/amosbird)). - -#### 实验功能 - -* 添加 `Embedded RocksDB` 表引擎 (可用于字典). [#15073](https://github.com/ClickHouse/ClickHouse/pull/15073) ([sundyli](https://github.com/sundy-li)). - -#### 构建/测试/打包改进 - -* 测试覆盖率构建图像的改进. [#17233](https://github.com/ClickHouse/ClickHouse/pull/17233) ([alesapin](https://github.com/alesapin)). -* 将嵌入的时区数据更新到 2020d 版 (同时将 cctz 更新到最新的 master). [#17204](https://github.com/ClickHouse/ClickHouse/pull/17204) ([filimonov](https://github.com/filimonov)). -* 修复 Poco 中的 UBSan 报告. 这将关闭 [#12719](https://github.com/ClickHouse/ClickHouse/issues/12719). [#16765](https://github.com/ClickHouse/ClickHouse/pull/16765) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 不要使用 UBSan 检测第三方库. [#16764](https://github.com/ClickHouse/ClickHouse/pull/16764) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复缓存字典中的 UBSan 报告. 这将关闭 [#12641](https://github.com/ClickHouse/ClickHouse/issues/12641). [#16763](https://github.com/ClickHouse/ClickHouse/pull/16763) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在尝试将无限浮点数转换为整数时修复 UBSan 报告. 这将关闭 [#14190](https://github.com/ClickHouse/ClickHouse/issues/14190). [#16677](https://github.com/ClickHouse/ClickHouse/pull/16677) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -## ClickHouse 版本 20.11 - -### ClickHouse 版本 v20.11.7.16-stable, 2021-03-02 - -#### 改进 - -* 将 clickhouse 用户和组的 uid / gid 显式设置为 clickhouse-server 图像中的固定值 (101). [#19096](https://github.com/ClickHouse/ClickHouse/pull/19096) ([filimonov](https://github.com/filimonov)). - -#### Bug 修复 - -* BloomFilter 索引崩溃修复. 修复 [#19757](https://github.com/ClickHouse/ClickHouse/issues/19757). [#19884](https://github.com/ClickHouse/ClickHouse/pull/19884) ([Maksim Kita](https://github.com/kitaisreal)). -* 如果启用了 system.text_log, 则可能出现死锁. 这修复了 [#19874](https://github.com/ClickHouse/ClickHouse/issues/19874). [#19875](https://github.com/ClickHouse/ClickHouse/pull/19875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在以前的版本中, 函数 arrayEnumerateUniq 的异常参数可能会导致崩溃或无限循环. 这关闭 [#19787](https://github.com/ClickHouse/ClickHouse/issues/19787). [#19788](https://github.com/ClickHouse/ClickHouse/pull/19788) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了使用算术类型与字符串类型的准确比较时的堆栈溢出. [#19773](https://github.com/ClickHouse/ClickHouse/pull/19773) ([tavplubix](https://github.com/tavplubix)). -* 修复 `bitmapAndnot` 函数中的分段错误. 修复 [#19668](https://github.com/ClickHouse/ClickHouse/issues/19668). [#19713](https://github.com/ClickHouse/ClickHouse/pull/19713) ([Maksim Kita](https://github.com/kitaisreal)). -* 一些具有大整数的函数可能会导致段错误. 大整数是实验性功能. 这关闭 [#19667](https://github.com/ClickHouse/ClickHouse/issues/19667). [#19672](https://github.com/ClickHouse/ClickHouse/pull/19672) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `LowCardinality` 参数的 `neighbor` 函数的错误结果. 修复 [#10333](https://github.com/ClickHouse/ClickHouse/issues/10333). [#19617](https://github.com/ClickHouse/ClickHouse/pull/19617) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复断开连接后连接中 CompressedWriteBuffer 的 use-after-free. [#19599](https://github.com/ClickHouse/ClickHouse/pull/19599) ([Azat Khuzhin](https://github.com/azat)). -* `DROP/DETACH TABLE table ON CLUSTER cluster SYNC` 查询可能会挂起, 已修复. 修复 [#19568](https://github.com/ClickHouse/ClickHouse/issues/19568). [#19572](https://github.com/ClickHouse/ClickHouse/pull/19572) ([tavplubix](https://github.com/tavplubix)). -* 查询 CREATE DICTIONARY id 表达式修复. [#19571](https://github.com/ClickHouse/ClickHouse/pull/19571) ([Maksim Kita](https://github.com/kitaisreal)). -* 使用 merge_tree_min_rows_for_concurrent_read/merge_tree_min_bytes_for_concurrent_read=0/UINT64_MAX 修复 SIGSEGV. [#19528](https://github.com/ClickHouse/ClickHouse/pull/19528) ([Azat Khuzhin](https://github.com/azat)). -* 如果使用专门设计的参数调用 `addMonth` 函数, 则可能会发生缓冲区溢出 (在内存读取时). 这修复了 [#19441](https://github.com/ClickHouse/ClickHouse/issues/19441). This fixes [#19413](https://github.com/ClickHouse/ClickHouse/issues/19413). [#19472](https://github.com/ClickHouse/ClickHouse/pull/19472) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果文件之一中有空数据块,则将分布式批处理标记为已损坏. [#19449](https://github.com/ClickHouse/ClickHouse/pull/19449) ([Azat Khuzhin](https://github.com/azat)). -* 修复 Uber H3 库中可能的缓冲区溢出. 请参阅 https://github.com/uber/h3/issues/392. 这将关闭 [#19219](https://github.com/ClickHouse/ClickHouse/issues/19219). [#19383](https://github.com/ClickHouse/ClickHouse/pull/19383) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 system.parts _state 列 (查询此列时出现 LOGICAL_ERROR,由于顺序不正确). [#19346](https://github.com/ClickHouse/ClickHouse/pull/19346) ([Azat Khuzhin](https://github.com/azat)). -* 修复错误 `Cannot convert column now64() because it is constant but values of constants are different in source and result` . 续 [#7156](https://github.com/ClickHouse/ClickHouse/issues/7156). [#19316](https://github.com/ClickHouse/ClickHouse/pull/19316) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复在处理 ReplicatedMergeTree 表时并发 `ALTER` 和 `DROP` 查询可能挂起的错误. [#19237](https://github.com/ClickHouse/ClickHouse/pull/19237) ([alesapin](https://github.com/alesapin)). -* 修复了从 `ORC` 格式的文件中无限读取的问题 (在 [#10580](https://github.com/ClickHouse/ClickHouse/issues/10580) 中引入). 修复 [#19095](https://github.com/ClickHouse/ClickHouse/issues/19095). [#19134](https://github.com/ClickHouse/ClickHouse/pull/19134) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 clickhouse 无法从 `LowCardinality(Nullable(...))` 读取压缩编解码器并抛出异常 `Attempt to read after EOF` 时的启动错误. 修复 [#18340](https://github.com/ClickHouse/ClickHouse/issues/18340). [#19101](https://github.com/ClickHouse/ClickHouse/pull/19101) ([alesapin](https://github.com/alesapin)). -* 修复了使用 `Template` 或 `CustomSeparated` 格式通过 http 接口插入数据时出现的 `There no checkpoint` 错误. 修复 [#19021](https://github.com/ClickHouse/ClickHouse/issues/19021). [#19072](https://github.com/ClickHouse/ClickHouse/pull/19072) ([tavplubix](https://github.com/tavplubix)). -* 限制对以旧语法创建的 `MergeTree` 表的 `MODIFY TTL` 查询. 之前查询成功了, 但实际上没有效果. [#19064](https://github.com/ClickHouse/ClickHouse/pull/19064) ([Anton Popov](https://github.com/CurtizJ)). -* 确保 `groupUniqArray` 为 Enum 类型的参数返回正确的类型. 这将关闭 [#17875](https://github.com/ClickHouse/ClickHouse/issues/17875). [#19019](https://github.com/ClickHouse/ClickHouse/pull/19019) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果将函数 `ignore` 与 `LowCardinality` 参数一起使用, 则修复可能的错误 `Expected single dictionary argument for function` . 修复 [#14275](https://github.com/ClickHouse/ClickHouse/issues/14275). [#19016](https://github.com/ClickHouse/ClickHouse/pull/19016) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `TinyLog` 引擎修复将 `LowCardinality` 列插入到表中的问题. 修复 [#18629](https://github.com/ClickHouse/ClickHouse/issues/18629). [#19010](https://github.com/ClickHouse/ClickHouse/pull/19010) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 禁用 `optimize_move_functions_out_of_any` 因为优化并不总是正确的. 这将关闭 [#18051](https://github.com/ClickHouse/ClickHouse/issues/18051). 这将关闭 [#18973](https://github.com/ClickHouse/ClickHouse/issues/18973). [#18981](https://github.com/ClickHouse/ClickHouse/pull/18981) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了非常罕见的关机死锁. [#18977](https://github.com/ClickHouse/ClickHouse/pull/18977) ([tavplubix](https://github.com/tavplubix)). -* 修复某些转义文本突变时的错误 (例如 `ALTER ... UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1')` 序列化不正确). 修复 [#18878](https://github.com/ClickHouse/ClickHouse/issues/18878). [#18944](https://github.com/ClickHouse/ClickHouse/pull/18944) ([alesapin](https://github.com/alesapin)). -* 附加分区应该重置突变. [#18804](https://github.com/ClickHouse/ClickHouse/issues/18804). [#18935](https://github.com/ClickHouse/ClickHouse/pull/18935) ([fastio](https://github.com/fastio)). -* 修复 clickhouse-local 关机时可能挂起的问题. 这修复了 [#18891](https://github.com/ClickHouse/ClickHouse/issues/18891). [#18893](https://github.com/ClickHouse/ClickHouse/pull/18893) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 *If 具有一元函数和 Nullable 类型的组合器. [#18806](https://github.com/ClickHouse/ClickHouse/pull/18806) ([Azat Khuzhin](https://github.com/azat)). -* 如果设置`network_compression_method`全局设置为非默认值,则服务器可以拒绝异步分布式插入. 这修复了 [#18741](https://github.com/ClickHouse/ClickHouse/issues/18741). [#18776](https://github.com/ClickHouse/ClickHouse/pull/18776) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在尝试将 `CAST` `NULL` 从 `Nullable(String)` 转换为 `Nullable(Decimal(P, S))` 时的 `Attempt to read after eof` 错误. 现在函数`CAST` 在它不能从可为空的字符串解析十进制时返回`NULL`. 修复 [#7690](https://github.com/ClickHouse/ClickHouse/issues/7690). [#18718](https://github.com/ClickHouse/ClickHouse/pull/18718) ([Winter Zhang](https://github.com/zhang2014)). -* 使用不匹配的 arg 大小修复 Logger. [#18717](https://github.com/ClickHouse/ClickHouse/pull/18717) ([sundyli](https://github.com/sundy-li)). -* 添加 FixedString 数据类型支持. 将数据从 MySQL 复制到 ClickHouse 时, 我会收到此异常 `Code: 50, e.displayText() = DB::Exception: Unsupported type FixedString(1)` . 此补丁修复了错误 [#18450](https://github.com/ClickHouse/ClickHouse/issues/18450) 还修复了 [#6556](https://github.com/ClickHouse/ClickHouse/issues/6556). [#18553](https://github.com/ClickHouse/ClickHouse/pull/18553) ([awesomeleo](https://github.com/awesomeleo)). -* 在使用 `RIGHT` 或 FULL` 连接进行子查询后使用 `ORDER BY` 时修复可能的 `Pipeline stuck` 错误. [#18550](https://github.com/ClickHouse/ClickHouse/pull/18550) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复在相应的突变杀死后可能导致 `ALTER` 查询挂起的错误. 由线程模糊器发现. [#18518](https://github.com/ClickHouse/ClickHouse/pull/18518) ([alesapin](https://github.com/alesapin)). -* 在合并期间禁用AIO写, 因为这可能导致合并期间主键列的数据损坏. [#18481](https://github.com/ClickHouse/ClickHouse/pull/18481) ([alesapin](https://github.com/alesapin)). -* 当无法计算结果时, 在分析阶段禁用子查询的常量折叠. [#18446](https://github.com/ClickHouse/ClickHouse/pull/18446) ([Azat Khuzhin](https://github.com/azat)). -* 修复了在使用类型为 Nullable(String) 的参数执行 `toType(...)` 函数 (`toDate`、`toUInt32` 等) 时出现的 `value is too short` 错误. 现在这些函数在解析错误时返回 `NULL` 而不是抛出异常. 修复 [#7673](https://github.com/ClickHouse/ClickHouse/issues/7673). [#18445](https://github.com/ClickHouse/ClickHouse/pull/18445) ([tavplubix](https://github.com/tavplubix)). -* 限制从宽部分合并到紧凑部分. 在垂直合并的情况下, 它会导致结果部分损坏. [#18381](https://github.com/ClickHouse/ClickHouse/pull/18381) ([Anton Popov](https://github.com/CurtizJ)). -* 修复填充表 `system.settings_profile_elements` . 这个 PR 修复了 [#18231](https://github.com/ClickHouse/ClickHouse/issues/18231). [#18379](https://github.com/ClickHouse/ClickHouse/pull/18379) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复带有常量参数的二元函数的索引分析, 导致错误的查询结果. 这修复了 [#18364](https://github.com/ClickHouse/ClickHouse/issues/18364). [#18373](https://github.com/ClickHouse/ClickHouse/pull/18373) ([Amos Bird](https://github.com/amosbird)). -* 使用组合子 `Distinct` 修复聚合函数中可能的崩溃,同时使用两级聚合. 修复 [#17682](https://github.com/ClickHouse/ClickHouse/issues/17682). [#18365](https://github.com/ClickHouse/ClickHouse/pull/18365) ([Anton Popov](https://github.com/CurtizJ)). -* 如果只能从 `table` 中选择任意一列, 现在可以执行 `SELECT count() FROM table` . 此 PR 修复 [#10639](https://github.com/ClickHouse/ClickHouse/issues/10639). [#18233](https://github.com/ClickHouse/ClickHouse/pull/18233) ([Vitaly Baranov](https://github.com/vitlibar)). -* `SELECT JOIN` 现在需要对每个连接表的 `SELECT` 权限. 此 PR 修复 [#17654](https://github.com/ClickHouse/ClickHouse/issues/17654). [#18232](https://github.com/ClickHouse/ClickHouse/pull/18232) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复在读取退避的情况下从 `MergeTree*` 读取时可能不完整的查询结果 (消息`Debug: MergeTreeReadPool: Will lower number of threads in logs). 在 [#16423](https://github.com/ClickHouse/ClickHouse/issues/16423) 中引入. 修复 [#18137](https://github.com/ClickHouse/ClickHouse/issues/18137). [#18216](https://github.com/ClickHouse/ClickHouse/pull/18216) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复查询 `MODIFY COLUMN ... REMOVE TTL` 并没有真正删除列 TTL 时的错误. [#18130](https://github.com/ClickHouse/ClickHouse/pull/18130) ([alesapin](https://github.com/alesapin)). -* 使用谓词优化器修复不确定函数. 这修复了 [#17244](https://github.com/ClickHouse/ClickHouse/issues/17244). [#17273](https://github.com/ClickHouse/ClickHouse/pull/17273) ([Winter Zhang](https://github.com/zhang2014)). -* Mutation 可能会在 `MOVE` 或 `REPLACE PARTITION` 之后等待一些不存在的部分, 或者在极少数情况下, 在 `DETACH` 或 `DROP PARTITION` 之后挂起. 是固定的. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). - -#### 构建/测试/打包改进 - -* 将时区信息更新为 2020e. [#18531](https://github.com/ClickHouse/ClickHouse/pull/18531) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse 版本 v20.11.6.6-stable, 2020-12-24 - -#### Bug 修复 - -* 修复了在具有双 `IPv4/IPv6 stack` 的机器上的服务器无法访问 `clickhouse-odbc-bridge` 进程时的问题, 并修复了使用格式错误的查询执行 ODBC 字典更新和/或导致崩溃时的问题. 这可能会关闭 [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* 修复了 Enum 和 Int 类型之间的键比较. 这修复了 [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `MaterializeMySQL` 数据库引擎中唯一键转换崩溃的问题. 这修复了 [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) 并修复了 [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 S3 URL 解析中的 `std::out_of_range: basic_string`. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复了由于 MaterializeMySQL 不支持转换 MySQL 前缀索引而导致某些表无法从 MySQL 同步到 ClickHouse 的问题. 这修复了 [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) 并修复了 [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了查询包含`ARRAY JOIN`时查询优化产生错误结果的问题. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* 修复 `topK` 聚合函数中可能出现的段错误. 这将关闭 [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* 如果禁用了`in_memory_parts_enable_wal`, 则不要从 WAL 恢复部件. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* 修复了 ClickHouse 无法恢复与 MySQL 服务器的连接时的问题. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复了 `optimize_trivial_count_query` 与分区谓词不一致的行为. [#17644](https://github.com/ClickHouse/ClickHouse/pull/17644) ([Azat Khuzhin](https://github.com/azat)). -* 修复了服务器在守护进程模式下运行时空的 `system.stack_trace` 表. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). -* 修复了当 xxception `fmt::v7::format_error` 可以登录到 MergeTree 表的后台时的行为. 这修复了 [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 clickhouse-client 在交互模式下使用多行查询和单行注释错误地扩展到查询结束时的行为. 这修复了 [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了服务器在极少数情况下停止接受连接的问题. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在不同副本上杀死相应突变时更改查询挂起的问题. 这修复了 [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* 修复了 clickhouse 低估标记缓存大小时的错误. 当有很多带有标记的小文件时可能会发生. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* 修复了启用设置 `optimize_redundant_functions_in_order_by` 的 `ORDER BY`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `DISTINCT` 之后由于不正确优化而可能出现的重复项. 这修复了 [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了从具有 `LowCardinality` 类型的 `JOIN` 表读取时崩溃的问题. 这修复了 [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了子查询中有 const 列时设置索引失效的问题. 这修复了 [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* 修复索引比较类型不同时可能出现的错误索引分析. 这修复了 [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `ColumnConst` 比较导致崩溃. 这修复了 [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* 修复了非领导者 `ReplicatedMergeTreeTables` 的 `ON CLUSTER` 查询可能永远挂起的错误. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* 修复了函数 `fuzz Bits` 中模糊器发现的错误. 这修复了 [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* 避免远程查询的不必要的网络错误, 这些错误在执行时可能会被取消, 例如使用 `LIMIT` 的查询. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* 修复了从 double 转换时大整数 (128、256 位) 的错误结果. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). -* 出现错误时解析 `format_avro_schema_registry_url` 的IP. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* 修复了在 `ALTER TABLE ... MODIFY COLUMN ... NewType` 之后, 当 `SELECT` 在更改列上具有 `WHERE` 表达式并且更改尚未完成时可能出现的服务器崩溃. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* `clickhouse-git-import` 中没有正确计算批判信息. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过单调函数优化来固定顺序. 修复 [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了启用设置 `optimize_aggregators_of_group_by_keys` 和 joins 的 group by 优化. 这修复了 [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* 安装脚本应始终在 config 文件夹中创建子目录. 这仅与使用自定义配置的 Docker 构建相关. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* 修复了使用 `ORDER BY` 的查询可能出现的错误 `Illegal type of argument` . 这修复了 [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果没有数据写入 WriteBufferFromS3, 则中止分段上传. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复了在不带任何参数的情况下使用 `any` 时崩溃的问题. 这修复了 [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* 修复了 ClickHouse 过去总是返回 0 而不是通过 MySQL 协议进行 `INSERT` 查询的受影响行数时的行为. 这修复了 [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 TDigest 不受控制的增长. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). -* 修复了在 Aggregate 函数中使用后缀 `if` 时远程查询失败的问题. 这修复了 [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了由 `select_sequential_consistency` 导致优化的琐碎计数查询和 system.tables 的不一致行为. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). -* 使用 ColumnTransformer 替换不存在的列时抛出错误. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). - - -### ClickHouse 版本 v20.11.3.3-stable, 2020-11-13 - -#### Bug 修复 - -* 当查询分析器打开并且 ClickHouse 安装在操作系统上时, 修复罕见的无声崩溃, glibc 版本已经 (据说) 损坏了某些功能的异步展开表. 这修复了 [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse 版本 v20.11.2.1, 2020-11-11 - -#### 向后不兼容变更 - -* 如果在 `distributed_ddl` 配置部分指定了一些 `profile`, 那么这个配置文件可以在服务器启动时覆盖 `default` 配置文件的设置. 已修复, 现在分布式 DDL 查询的设置不应影响全局服务器设置. [#16635](https://github.com/ClickHouse/ClickHouse/pull/16635) ([tavplubix](https://github.com/tavplubix)). -* 限制在键 (排序键、主键、分区键等) 中使用不可比较的数据类型 (如 `AggregateFunction`). [#16601](https://github.com/ClickHouse/ClickHouse/pull/16601) ([alesapin](https://github.com/alesapin)). -* 删除 `ANALYZE` 和 `AST` 查询,并使设置 `enable_debug_queries` 过时, 因为它现在是全功能 `EXPLAIN` 查询的一部分. [#16536](https://github.com/ClickHouse/ClickHouse/pull/16536) ([Ivan](https://github.com/abyss7)). -* 聚合函数 `boundingRatio`、`rankCorr`、`retention`、`timeSeriesGroupSum`、`timeSeriesGroupRateSum`、`windowFunnel` 被错误地设为不区分大小写. 现在他们的名字按照设计区分大小写. 只有在 SQL 标准中指定的函数或为了与其他 DBMS 兼容而制作的函数或与这些函数类似的函数才应不区分大小写. [#16407](https://github.com/ClickHouse/ClickHouse/pull/16407) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使 `rankCorr` 函数在数据不足时返回 nan [#16124](https://github.com/ClickHouse/ClickHouse/issues/16124). [#16135](https://github.com/ClickHouse/ClickHouse/pull/16135) ([hexiaoting](https://github.com/hexiaoting)). -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重启, 旧版本在新版本存在的情况下启动, 可能会导致 到 `Part ... intersects previous part` 错误. 为防止出现此错误, 首先在所有集群节点上安装更新的 clickhouse-server 软件包, 然后重新启动 (因此,当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* 添加了对 LDAP 作为本地不存在用户的用户目录的支持. [#12736](https://github.com/ClickHouse/ClickHouse/pull/12736) ([Denis Glazachev](https://github.com/traceon)). -* 添加 `system.replicated_fetches` 表, 显示当前正在运行的后台提取. [#16428](https://github.com/ClickHouse/ClickHouse/pull/16428) ([alesapin](https://github.com/alesapin)). -* 添加设置`date_time_output_format`. [#15845](https://github.com/ClickHouse/ClickHouse/pull/15845) ([Maksim Kita](https://github.com/kitaisreal)). -* 为 ClickHouse 添加了最小的 Web UI. [#16158](https://github.com/ClickHouse/ClickHouse/pull/16158) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许一次读/写单个 protobuf 消息 (无长度分隔符). [#15199](https://github.com/ClickHouse/ClickHouse/pull/15199) ([filimonov](https://github.com/filimonov)). -* 添加了初始 OpenTelemetry 支持. ClickHouse 现在通过 Native 和 HTTP 协议接受 OpenTelemetry traceparent 标头, 并在某些情况下向下游传递它们. 执行查询的跟踪跨度保存到 `system.opentelemetry_span_log` 表中. [#14195](https://github.com/ClickHouse/ClickHouse/pull/14195) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 允许在 `CREATE TABLE` 查询的列列表中指定主键. 这是与其他 SQL 方言兼容所必需的. [#15823](https://github.com/ClickHouse/ClickHouse/pull/15823) ([Maksim Kita](https://github.com/kitaisreal)). -* 实现 `OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES}` 在带有 ORDER BY 的 SELECT 查询中. 这是指定 `LIMIT` 的SQL标准方式. [#15855](https://github.com/ClickHouse/ClickHouse/pull/15855) ([hexiaoting](https://github.com/hexiaoting)). -* `errorCodeToName` 函数 - 返回错误的变量名称 (用于分析 query_log 和类似的). `system.errors` 表 - 显示错误发生的次数 (包括 `system_events_show_zero_values`). [#16438](https://github.com/ClickHouse/ClickHouse/pull/16438) ([Azat Khuzhin](https://github.com/azat)). -* 添加了函数 `untuple`, 这是一个特殊的函数, 它可以通过扩展命名元组将新列引入到 SELECT 列表中. [#16242](https://github.com/ClickHouse/ClickHouse/pull/16242) ([Nikolai Kochetov](https://github.com/KochetovNicolai), [Amos Bird](https://github.com/amosbird)). -* 现在我们可以通过查询参数提供标识符. 并且这些参数可以用作表对象或列. [#16594](https://github.com/ClickHouse/ClickHouse/pull/16594) ([Amos Bird](https://github.com/amosbird)). -* 为 MergeTree BloomFilter 索引添加了大整数 (UInt256、Int128、Int256)和 UUID 数据类型支持. 大整数是一项实验性功能. [#16642](https://github.com/ClickHouse/ClickHouse/pull/16642) ([Maksim Kita](https://github.com/kitaisreal)). -* 添加 `farmFingerprint64` 函数 (非加密字符串散列). [#16570](https://github.com/ClickHouse/ClickHouse/pull/16570) ([Jacob Hayes](https://github.com/JacobHayes)). -* 添加 `log_queries_min_query_duration_ms`, 只有比这个设置的值慢的查询才会转到 `query_log`/`query_thread_log` (例如mysql中的 `slow_query_log` ). [#16529](https://github.com/ClickHouse/ClickHouse/pull/16529) ([Azat Khuzhin](https://github.com/azat)). -* 能够在 `Alpine` 的顶部创建一个 docker 图像. 使用来自 ubuntu 20.04 的预编译二进制和 glibc 组件. [#16479](https://github.com/ClickHouse/ClickHouse/pull/16479) ([filimonov](https://github.com/filimonov)). -* 添加了 `toUUIDOrNull`、`toUUIDOrZero` 转换函数. [#16337](https://github.com/ClickHouse/ClickHouse/pull/16337) ([Maksim Kita](https://github.com/kitaisreal)). -* 添加 `max_concurrent_queries_for_all_users` 设置, 用例见 [#6636](https://github.com/ClickHouse/ClickHouse/issues/6636). [#16154](https://github.com/ClickHouse/ClickHouse/pull/16154) ([nvartolomei](https://github.com/nvartolomei)). -* 向 clickhouse-client 添加一个新选项 `print_query_id`. 它有助于使用客户端生成的当前查询 ID 生成任意字符串. 默认情况下还会在 clickhouse-client 中打印查询 ID. [#15809](https://github.com/ClickHouse/ClickHouse/pull/15809) ([Amos Bird](https://github.com/amosbird)). -* 添加 `tid` 和 `logTrace` 函数. 这将关闭 [#9434](https://github.com/ClickHouse/ClickHouse/issues/9434). [#15803](https://github.com/ClickHouse/ClickHouse/pull/15803) ([flynn](https://github.com/ucasFL)). -* 添加函数 `formatReadableTimeDelta`, 将时间增量格式化为人类可读的字符串... [#15497](https://github.com/ClickHouse/ClickHouse/pull/15497) ([Filipe Caixeta](https://github.com/filipecaixeta)). -* 在多磁盘配置中为卷添加了 `disable_merges` 选项. [#13956](https://github.com/ClickHouse/ClickHouse/pull/13956) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### 实验功能 - -* 新函数 `encrypt`、`aes_encrypt_mysql`、`decrypt`、`aes_decrypt_mysql`. 这些功能运行缓慢, 因此我们将其视为实验性功能. [#11844](https://github.com/ClickHouse/ClickHouse/pull/11844) ([Vasily Nemkov](https://github.com/Enmk)). - -#### Bug 修复 - -* 在 `system.distribution_queue` 中的data_path中屏蔽密码. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). -* 在启用了 `transform_null_in` 设置的情况下修复多个列和元组上的 `IN` 运算符. 修复 [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* 如果查询的表没有采样, 设置 `max_parallel_replicas` 将无法正常工作. 这修复了 [#5733](https://github.com/ClickHouse/ClickHouse/issues/5733). [#16675](https://github.com/ClickHouse/ClickHouse/pull/16675) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用 max_threads > 0 和 ORDER BY 中的表达式修复 optimize_read_in_order/optimize_aggregation_in_order. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* `DEFAULT` 表达式的计算涉及可能的名称冲突 (这是不太可能遇到的). 这修复了 [#9359](https://github.com/ClickHouse/ClickHouse/issues/9359). [#16612](https://github.com/ClickHouse/ClickHouse/pull/16612) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `query_thread_log.query_duration_ms` 单元. [#16563](https://github.com/ClickHouse/ClickHouse/pull/16563) ([Azat Khuzhin](https://github.com/azat)). -* 修复使用 MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine 时的错误. `MaterializeMySQL` 是一个实验性功能. [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* 带有 `Decimal` 的 `round` 函数的特别设计参数导致整数除以零. 这修复了 [#13338](https://github.com/ClickHouse/ClickHouse/issues/13338). [#16451](https://github.com/ClickHouse/ClickHouse/pull/16451) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复分布式的 DROP TABLE (使用 INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* 修复复制队列中非常大的条目的处理. 如果表结构非常大 (接近 1 MB), 则非常大的条目可能会出现在 ALTER 查询中. 这修复了 [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了由于未创建过滤集而可能删除部分返回数据时的不一致行为. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复 sharding_key 中的 dictGet (以及类似的地方, 即当函数上下文被永久存储时). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). -* 修复尝试执行 `OPTIMIZE` 命令时在 `clickhouse-local` 中抛出的异常. 修复 [#16076](https://github.com/ClickHouse/ClickHouse/issues/16076). [#16192](https://github.com/ClickHouse/ClickHouse/pull/16192) ([filimonov](https://github.com/filimonov)). -* 修复 [#15780](https://github.com/ClickHouse/ClickHouse/issues/15780) 回归, 例如 `indexOf([1, 2, 3], toLowCardinality(1))` 现在被禁止, 但不应该. [#16038](https://github.com/ClickHouse/ClickHouse/pull/16038) ([Mike](https://github.com/myrrc)). -* 修复 MySQL 数据库的错误. 当用作数据库引擎的 MySQL 服务器关闭时, 某些查询会引发异常, 因为它们试图从禁用的服务器获取表, 而这是不必要的. 例如, 查询 `SELECT ... FROM system.parts` 应该只适用于 MergeTree 表并且根本不要接触 MySQL 数据库. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). -* 现在, 当 `ALTER MODIFY COLUMN ... DEFAULT ...` 与列类型的默认值不兼容时, 将抛出异常. 修复 [#15854](https://github.com/ClickHouse/ClickHouse/issues/15854). [#15858](https://github.com/ClickHouse/ClickHouse/pull/15858) ([alesapin](https://github.com/alesapin)). -* 修复了 IPv4CIDRToRange/IPv6CIDRToRange 函数以接受常量 IP 列值. [#15856](https://github.com/ClickHouse/ClickHouse/pull/15856) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). - -#### 改进 - -* 将 `INTERVAL '1 hours'` 视为等同于 `INTERVAL 1 HOUR`, 以与 Postgres 和类似的兼容. 这修复了 [#15637](https://github.com/ClickHouse/ClickHouse/issues/15637). [#15978](https://github.com/ClickHouse/ClickHouse/pull/15978) ([flynn](https://github.com/ucasFL)). -* 启用通过 CSV、TSV 和 JSON 输入格式的数字 ID 解析枚举值. [#15685](https://github.com/ClickHouse/ClickHouse/pull/15685) ([vivarum](https://github.com/vivarum)). -* 更好地为 JBOD 架构和 `MergeTree` 存储安排读取任务. 新设置 `read_backoff_min_concurrency` 作为读取线程数的下限. [#16423](https://github.com/ClickHouse/ClickHouse/pull/16423) ([Amos Bird](https://github.com/amosbird)). -* 在 `Avro` 格式中添加对 `LowCardinality` 的缺失支持. [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* 将 `S3` 与 nginx 服务器一起用作代理的解决方法. Nginx 当前不接受像 `http://domain.com?delete` 这样的空路径 url, 但是 vanilla aws-sdk-cpp 会产生这种 url. 此提交使用修补过的 aws-sdk-cpp 版本, 在这种情况下, 它使 url 以"/"作为路径, 例如 `http://domain.com/?delete`. [#16814](https://github.com/ClickHouse/ClickHouse/pull/16814) ([ianton-ru](https://github.com/ianton-ru)). -* 更好地诊断输入数据中的解析错误. 在 `Cannot read all data` 错误上提供行号. [#16644](https://github.com/ClickHouse/ClickHouse/pull/16644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使 `mind Map` 和`maxMap` 的行为更加可取. 它不会在结果中跳过零值. 修复 [#16087](https://github.com/ClickHouse/ClickHouse/issues/16087). [#16631](https://github.com/ClickHouse/ClickHouse/pull/16631) ([Ildus Kurbangaliev](https://github.com/ildus)). -* 在运行时更好地更新 ZooKeeper 配置. [#16630](https://github.com/ClickHouse/ClickHouse/pull/16630) ([sundyli](https://github.com/sundy-li)). -* 尽早应用 SETTINGS 子句. 它允许修改查询中的更多设置. 这将关闭 [#3178](https://github.com/ClickHouse/ClickHouse/issues/3178). [#16619](https://github.com/ClickHouse/ClickHouse/pull/16619) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在 `event_time_microseconds` 字段存储在 Decimal64 中, 而不是 UInt64. [#16617](https://github.com/ClickHouse/ClickHouse/pull/16617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 现在可以在`APPLY`列转换器中使用参数化函数. [#16589](https://github.com/ClickHouse/ClickHouse/pull/16589) ([Amos Bird](https://github.com/amosbird)). -* 改进后台任务的调度, 该任务删除 `Atomic` 数据库中已删除表的数据. 如果表实际上没有数据目录, `Atomic` 数据库不会创建到表数据目录的损坏符号链接. [#16584](https://github.com/ClickHouse/ClickHouse/pull/16584) ([tavplubix](https://github.com/tavplubix)). -* `WITH` 部分 (CTE) 中的子查询可以通过名称引用 `WITH` 部分中先前的子查询. [#16575](https://github.com/ClickHouse/ClickHouse/pull/16575) ([Amos Bird](https://github.com/amosbird)). -* 将 current_database 添加到 `system.query_thread_log`. [#16558](https://github.com/ClickHouse/ClickHouse/pull/16558) ([Azat Khuzhin](https://github.com/azat)). -* 允许将当前实例中已经提交或过时的部分提取到分离目录中. 当从另一个集群迁移表并具有 N 到 1 个分片映射时, 它很有用. 它也与当前的 fetchPartition 实现一致. [#16538](https://github.com/ClickHouse/ClickHouse/pull/16538) ([Amos Bird](https://github.com/amosbird)). -* `RabbitMQ` 的多项改进: 修复了 [#16263](https://github.com/ClickHouse/ClickHouse/issues/16263) 的错误. 还最小化了事件循环生命周期. 添加了更高效的队列设置. [#16426](https://github.com/ClickHouse/ClickHouse/pull/16426) ([Kseniia Sumarokova](https://github.com/kssenii)). -* 修复 `quantileDeterministic` 函数中的调试断言. 在以前的版本中, 它还可以通过网络传输多达两倍的数据. 虽然不存在错误. 这修复了 [#15683](https://github.com/ClickHouse/ClickHouse/issues/15683). [#16410](https://github.com/ClickHouse/ClickHouse/pull/16410) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加 `TablesToDropQueueSize` 指标. 它等于正在等待后台数据删除的已删除表的数量. [#16364](https://github.com/ClickHouse/ClickHouse/pull/16364) ([tavplubix](https://github.com/tavplubix)). -* 当客户端断开连接时更好的诊断. 在以前的版本中, 服务器中记录了 `Attempt to read after EOF` 和 `Broken pipe` 异常. 在新版本中, 它的信息消息 `Client has dropped the connection, cancel the query.`. [#16329](https://github.com/ClickHouse/ClickHouse/pull/16329) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 Set/Join 表引擎添加 total_rows/total_bytes (来自 system.tables) 支持. [#16306](https://github.com/ClickHouse/ClickHouse/pull/16306) ([Azat Khuzhin](https://github.com/azat)). -* 现在可以为 MergeTree 表引擎系列指定没有 `ORDER BY` 的 `PRIMARY KEY`. 关闭 [#15591](https://github.com/ClickHouse/ClickHouse/issues/15591). [#16284](https://github.com/ClickHouse/ClickHouse/pull/16284) ([alesapin](https://github.com/alesapin)). -* 如果系统中没有 tmp 文件夹 (chroot、错误配置等) `clickhouse-local` 将在当前目录中创建临时子文件夹. [#16280](https://github.com/ClickHouse/ClickHouse/pull/16280) ([filimonov](https://github.com/filimonov)). -* 添加对嵌套数据类型 (如命名元组) 作为子类型的支持. 修复 [#15587](https://github.com/ClickHouse/ClickHouse/issues/15587). [#16262](https://github.com/ClickHouse/ClickHouse/pull/16262) ([Ivan](https://github.com/abyss7)). -* 支持`database_atomic_wait_for_drop_and_detach_synchronously`/`NO DELAY`/`SYNC` for `DROP DATABASE`. [#16127](https://github.com/ClickHouse/ClickHouse/pull/16127) ([Azat Khuzhin](https://github.com/azat)). -* 添加 `allow_nondeterministic_optimize_skip_unused_shards` (允许在分片键中使用非确定性,如 `rand()` 或 `dictGet()`). [#16105](https://github.com/ClickHouse/ClickHouse/pull/16105) ([Azat Khuzhin](https://github.com/azat)). -* 修复 `memory_profiler_step`/`max_untracked_memory` 以通过 HTTP 进行查询 (包括测试). 修复在 xml 配置中全局调整此值也无济于事的问题, 因为这些设置无论如何都不会应用, 只有默认 (4MB) 值是 [used](https://github.com/ClickHouse/ClickHouse/blob/17731245336d8c84f75e4c0894c5797ed7732190 /src/Common/ThreadStatus.h#L104). 修复 http 查询最根 ThreadStatus 的 `query_id` (通过在读取 query_id 后初始化 QueryScope). [#16101](https://github.com/ClickHouse/ClickHouse/pull/16101) ([Azat Khuzhin](https://github.com/azat)). -* 现在, 无论集群配置中的 `` 设置如何,都可以执行 `ALTER ... ON CLUSTER` 查询. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* 修复了 `clickhouse-client` 可能因加载建议而在退出时中止的罕见问题. 这修复了 [#16035](https://github.com/ClickHouse/ClickHouse/issues/16035). [#16047](https://github.com/ClickHouse/ClickHouse/pull/16047) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为具有复杂键的 `Redis` 字典添加 `cache` 布局支持. [#15985](https://github.com/ClickHouse/ClickHouse/pull/15985) ([Anton Popov](https://github.com/CurtizJ)). -* 在配置错误的情况下修复查询挂起 (无限循环) (`connections_with_failover_max_tries` 设置为 0). [#15876](https://github.com/ClickHouse/ClickHouse/pull/15876) ([Azat Khuzhin](https://github.com/azat)). -* 将某些日志消息的级别从信息更改为调试, 因此不会为每个查询显示信息消息. 这将关闭 [#5293](https://github.com/ClickHouse/ClickHouse/issues/5293). [#15816](https://github.com/ClickHouse/ClickHouse/pull/15816) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 删除 `MemoryTrackingInBackground*` 指标以避免潜在的误导性结果. 这修复了 [#15684](https://github.com/ClickHouse/ClickHouse/issues/15684). [#15813](https://github.com/ClickHouse/ClickHouse/pull/15813) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加重新连接到 `zookeeper-dump-tree` 工具. [#15711](https://github.com/ClickHouse/ClickHouse/pull/15711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许在 `CREATE TABLE table AS table_function(...)` 查询中明确指定列列表. 修复 [#9249](https://github.com/ClickHouse/ClickHouse/issues/9249) 修复 [#14214](https://github.com/ClickHouse/ClickHouse/issues/14214). [#14295](https://github.com/ClickHouse/ClickHouse/pull/14295) ([tavplubix](https://github.com/tavplubix)). - -#### 性能改进 - -* 不要在 SELECT FINAL 中跨分区合并部分. [#15938](https://github.com/ClickHouse/ClickHouse/pull/15938) ([Kruglov Pavel](https://github.com/Avogar)). -* 提高 `-OrNull` 和 `-OrDefault` 聚合函数的性能. [#16661](https://github.com/ClickHouse/ClickHouse/pull/16661) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 提高 `quantileMerge` 的性能. 在以前的版本中, 它非常慢. 这将关闭 [#1463](https://github.com/ClickHouse/ClickHouse/issues/1463). [#16643](https://github.com/ClickHouse/ClickHouse/pull/16643) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 稍微提高逻辑函数的性能. [#16347](https://github.com/ClickHouse/ClickHouse/pull/16347) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 改进了 MergeTree 表引擎中合并分配的性能. 不应该对用户可见. [#16191](https://github.com/ClickHouse/ClickHouse/pull/16191) ([alesapin](https://github.com/alesapin)). -* 通过预分配哈希表加速哈希/sparse_hashed 字典加载. [#15454](https://github.com/ClickHouse/ClickHouse/pull/15454) ([Azat Khuzhin](https://github.com/azat)). -* 现在, 琐碎的计数优化变得有点不琐碎了.包含精确分区expr的谓词也可以被优化. 这也修复了[#11092](https://github.com/ClickHouse/ClickHouse/issues/11092)当 `max_parallel_replicas > 1` 时返回错误计数的问题. [#15074](https://github.com/ClickHouse/ClickHouse/pull/15074) ([Amos Bird](https://github.com/amosbird)). - -#### 构建/测试/打包改进 - -* 为无状态测试添加薄片检查. 在功能测试被合并之前, 它将提前检测出可能不可靠的功能测试. [#16238](https://github.com/ClickHouse/ClickHouse/pull/16238) ([alesapin](https://github.com/alesapin)). -* 使用适当的版本进行 `croaring` 而不是合并. [#16285](https://github.com/ClickHouse/ClickHouse/pull/16285) ([sundyli](https://github.com/sundy-li)). -* 改进为 `ya.make` 构建系统生成构建文件 (Arcadia). [#16700](https://github.com/ClickHouse/ClickHouse/pull/16700) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 `MaterializeMySQL` 数据库引擎添加MySQL BinLog文件检查工具. `MaterializeMySQL` 是一个实验性功能. [#16223](https://github.com/ClickHouse/ClickHouse/pull/16223) ([Winter Zhang](https://github.com/zhang2014)). -* 检查非可执行文件上的可执行位. 人们经常不小心从 Windows 提交可执行文件. [#15843](https://github.com/ClickHouse/ClickHouse/pull/15843) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 检查标题中的 `#pragma once` . [#15818](https://github.com/ClickHouse/ClickHouse/pull/15818) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 libhdfs3 中非法代码样式 `&vector[idx]`. 这修复了 libcxx 调试版本。 另见 https://github.com/ClickHouse-Extras/libhdfs3/pull/8 . [#15815](https://github.com/ClickHouse/ClickHouse/pull/15815) ([Amos Bird](https://github.com/amosbird)). -* 在 Mac OS 上修复一个杂项示例工具的构建. 请注意, 我们不在我们的 CI 中在 Mac OS 上构建示例 (我们仅构建 ClickHouse 二进制文件), 因此它不会再次中断的可能性为零. 这修复了 [#15804](https://github.com/ClickHouse/ClickHouse/issues/15804). [#15808](https://github.com/ClickHouse/ClickHouse/pull/15808) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 简化 Sys/V 初始化脚本. [#14135](https://github.com/ClickHouse/ClickHouse/pull/14135) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 将 `boost::program_options` 添加到 `db_generator` 以提高其可用性. 这将关闭 [#15940](https://github.com/ClickHouse/ClickHouse/issues/15940). [#15973](https://github.com/ClickHouse/ClickHouse/pull/15973) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - - -## ClickHouse 版本 20.10 - -### ClickHouse 版本 v20.10.7.4-stable, 2020-12-24 - -#### Bug 修复 - -* 修复了 `clickhouse-odbc-bridge` 进程无法被具有双 `IPv4/IPv6` 堆栈的机器上的服务器访问的问题, 并修复了使用格式错误的查询执行 ODBC 字典更新和/或导致崩溃时的问题. 这可能会关闭 [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* 修复 Enum 和 Int 类型之间的键比较. 这修复了 [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `MaterializeMySQL` 数据库引擎中唯一键转换崩溃的问题. 这修复了 [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) 并修复了 [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 S3 URL 解析中的 `std::out_of_range: basic_string`. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复了由于 MaterializeMySQL 不支持转换 MySQL 前缀索引而导致某些表无法从 MySQL 同步到 ClickHouse 的问题. 这修复了 [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) 并修复了 [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). -* 修复 `topK` 聚合函数中可能出现的段错误. 这将关闭 [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* 如果禁用了`in_memory_parts_enable_wal`, 则不要从 `WAL` 恢复部件. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* 修复了 ClickHouse 无法恢复与 MySQL 服务器的连接时的问题. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复了服务器在守护进程模式下运行时空的 `system.stack_trace` 表. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). -* 修复了在多行查询的交互模式下使用 `clickhouse-client` 时的行为, 并且单行注释被错误地扩展到查询结束. 这修复了 [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了服务器在极少数情况下停止接受连接的问题. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在不同副本上杀死相应突变时的 `ALTER` 查询挂起. 这修复了 [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* 修复了 clickhouse 低估标记缓存大小时的错误. 当有很多带有标记的小文件时可能会发生. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* 修复了启用设置 `optimize_redundant_functions_in_order_by` 的 `ORDER BY` . [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `DISTINCT` 之后由于不正确优化而可能出现的重复项. 修复 [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了从具有 `LowCardinality` 类型的 `JOIN` 表读取时崩溃的问题. 这修复了 [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了子查询中存在 const 列时设置索引失效的问题. 这修复了 [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* 修复了导致崩溃的 `ColumnConst` 比较. 这个修复了 [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088). [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* 修复了非领导者 `ReplicatedMergeTreeTables` 的 `ON CLUSTER` 查询可能永远挂起的错误. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* 修复了函数 `fuzzBits` 中的模糊器发现错误. 这修复了. [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* 避免远程查询的不必要的网络错误, 这些错误在执行时可能会被取消, 例如使用 `LIMIT` 的查询. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* 修复了从 double 转换时大整数 (128、256 位) 的错误结果. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). -* 出现错误时解析 `format_avro_schema_registry_url` 的IP. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* 修复了在 `ALTER TABLE ... MODIFY COLUMN ... NewType` 之后, 当“SELECT”在更改列上具有 `WHERE` 表达式并且更改尚未完成时可能出现的服务器崩溃. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* `clickhouse-git-import` 中没有正确计算批判信息. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过单调函数优化来固定顺序. 这修复了 [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). -* Fixrf 通过启用设置 `optimize_aggregators_of_group_by_keys` 和连接来优化组. 这修复了 [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* 安装脚本应始终在 config 文件夹中创建子目录. 这仅与使用自定义配置的 Docker 构建相关. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* 使用 `ORDER BY` 的查询修复可能的错误 `Illegal type of argument` . 这修复了 [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果没有数据写入`WriteBufferFromS3`, 则中止分段上传. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复了在不带任何参数的情况下使用 `any` 时的崩溃问题. 这修复了 [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* 修复了 ClickHouse 过去总是通过 MySQL 协议为 `INSERT` 查询返回许多受影响行的 0 插入的行为. 这修复了 [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 `TDigest` 不受控制的增长. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). -* 修复了在 Aggregate 函数中使用后缀 `if` 时远程查询失败的问题. 这修复了 [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). - - -### ClickHouse 版本 v20.10.4.1-stable, 2020-11-13 - -#### Bug 修复 - -* 当查询分析器打开并且 ClickHouse 安装在操作系统上时, 修复罕见的无声崩溃, glibc 版本已经 (据说) 损坏了某些功能的异步展开表. 这修复了 [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301)。 这修复了 [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在启用了 `transform_null_in` 设置的情况下修复多个列和元组上的 `IN` 运算符. 修复 [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* 这将修复 optimize_read_in_order/optimize_aggregation_in_order 与 max_threads>0 和 ORDER BY 中的表达式. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* 现在, 当从输入解析 AVRO 时, LowCardinality 从类型中删除. 修复 [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* 通过适当缩小 GTID 集, 修复使用 MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine 和 MySQL Slave 上启用的 `slave_parallel_worker` 时元数据的快速增长. 这修复了 [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* 修复分布式的 DROP TABLE (使用 INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* 修复复制队列中非常大的条目的处理. 如果表结构非常大 (接近 1 MB), 则非常大的条目可能会出现在 ALTER 查询中. 这修复了 [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 MySQL 数据库的错误. 当用作数据库引擎的 MySQL 服务器关闭时, 某些查询会引发异常, 因为它们试图从禁用的服务器获取表, 而这是不必要的. 例如, 查询 `SELECT ... FROM system.parts` 应该只适用于 MergeTree 表并且根本不要接触 MySQL 数据库. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). - -#### 改进 - -* 将 S3 与 nginx 服务器一起用作代理的解决方法。 Nginx 当前不接受像 http://domain.com?delete 这样的空路径 url,但是 vanilla aws-sdk-cpp 会生成这种 url。 此提交使用修补过的 aws-sdk-cpp 版本,在这种情况下,它使 url 以“/”作为路径,例如 http://domain.com/?delete. [#16813](https://github.com/ClickHouse/ClickHouse/pull/16813) ([ianton-ru](https://github.com/ianton-ru)). - - -### ClickHouse 版本 v20.10.3.30, 2020-10-28 - -#### 向后不兼容变更 - -* 使 `multiple_joins_rewriter_version` 过时. 删除连接重写器的第一个版本. [#15472](https://github.com/ClickHouse/ClickHouse/pull/15472) ([Artem Zuikov](https://github.com/4ertus2)). -* 将 `format_regexp_escaping_rule` 设置的默认值 (它与 `Regexp` 格式相关) 更改为 `Raw` (这意味着 - 将整个子模式作为一个值读取), 使行为更符合用户的期望. [#15426](https://github.com/ClickHouse/ClickHouse/pull/15426) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 SQL 中添加对嵌套多行注释 `/* 注释 /* 注释 */ */` 的支持. 这符合SQL标准. [#14655](https://github.com/ClickHouse/ClickHouse/pull/14655) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了 MergeTree 设置 (`max_replicated_merges_with_ttl_in_queue` 和 `max_number_of_merges_with_ttl_in_pool`) 来控制后台池和复制队列中与 TTL 合并的数量. 仅当您使用删除 TTL 时, 此更改才会破坏与旧版本的兼容性. 否则, 复制将保持兼容. 如果您一次更新所有分片副本或执行 `SYSTEM STOP TTL MERGES` 直到完成所有副本的更新, 您可以避免不兼容问题. 如果你在复制队列中得到一个不兼容的条目, 首先, 执行 `SYSTEM STOP TTL MERGES` , 然后在 `ALTER TABLE ... DETACH PARTITION ...` 分配不兼容的TTL合并的分区. 将其重新连接到单个副本上. [#14490](https://github.com/ClickHouse/ClickHouse/pull/14490) ([alesapin](https://github.com/alesapin)). -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重启, 旧版本在新版本存在的情况下启动, 可能会导致 `Part ... intersects previous part` 错误. 为防止出现此错误, 首先在所有集群节点上安装更新的 clickhouse-server 软件包, 然后重新启动 (因此, 当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* 后台数据再压缩. 添加为 MergeTree 表引擎系列指定 `TTL ... RECOMPRESS codec_name` 的能力. [#14494](https://github.com/ClickHouse/ClickHouse/pull/14494) ([alesapin](https://github.com/alesapin)). -* 添加并行仲裁插入. 这将关闭 [#15601](https://github.com/ClickHouse/ClickHouse/issues/15601). [#15601](https://github.com/ClickHouse/ClickHouse/pull/15601) ([Latysheva Alexandra](https://github.com/alexelex)). -* 用于额外强制执行数据持久性的设置. 对非复制设置有用. [#11948](https://github.com/ClickHouse/ClickHouse/pull/11948) ([Anton Popov](https://github.com/CurtizJ)). -* 当复制块写入本地不存在的副本时 (尚未从副本中提取), 不要忽略它并在本地写入, 以达到与复制成功相同的效果. [#11684](https://github.com/ClickHouse/ClickHouse/pull/11684) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在我们支持 `WITH AS (subquery) ... ` 在查询上下文中引入命名子查询. 这将关闭 [#2416](https://github.com/ClickHouse/ClickHouse/issues/2416). 这将关闭 [#4967](https://github.com/ClickHouse/ClickHouse/issues/4967). [#14771](https://github.com/ClickHouse/ClickHouse/pull/14771) ([Amos Bird](https://github.com/amosbird)). -* 引入 `enable_global_with_statement` 设置, 它将第一个 select 的 `WITH` 语句传播到同一级别的其他选择查询, 并使 `WITH` 语句中的别名对子查询可见. [#15451](https://github.com/ClickHouse/ClickHouse/pull/15451) ([Amos Bird](https://github.com/amosbird)). -* 安全的集群间查询执行 (initial_user 作为当前查询用户). [#13156](https://github.com/ClickHouse/ClickHouse/pull/13156) ([Azat Khuzhin](https://github.com/azat)). [#15551](https://github.com/ClickHouse/ClickHouse/pull/15551) ([Azat Khuzhin](https://github.com/azat)). -* 添加删除列属性和表 TTL 的功能. 引入了查询 `ALTER TABLE MODIFY COLUMN col_name REMOVE what_to_remove`和`ALTER TABLE REMOVE TTL`. 这两个操作都是轻量级的, 并且在元数据级别执行. [#14742](https://github.com/ClickHouse/ClickHouse/pull/14742) ([alesapin](https://github.com/alesapin)). -* 添加格式 `RawBLOB` . 它用于输入或输出没有任何转义和分隔符的单个值. 这将关闭 [#15349](https://github.com/ClickHouse/ClickHouse/issues/15349). [#15364](https://github.com/ClickHouse/ClickHouse/pull/15364) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加 `reinterpretAsUUID` 函数, 允许将大端字节字符串转换为 UUID. [#15480](https://github.com/ClickHouse/ClickHouse/pull/15480) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 实现 `force_data_skiping_indices` 设置. [#15642](https://github.com/ClickHouse/ClickHouse/pull/15642) ([Azat Khuzhin](https://github.com/azat)). -* 添加一个设置 `output_format_pretty_row_numbers` 以计算出漂亮格式的结果. 这将关闭 [#15350](https://github.com/ClickHouse/ClickHouse/issues/15350). [#15443](https://github.com/ClickHouse/ClickHouse/pull/15443) ([flynn](https://github.com/ucasFL)). -* 添加了查询混淆工具. 它允许共享更多查询以进行更好的测试. 这将关闭 [#15268](https://github.com/ClickHouse/ClickHouse/issues/15268). [#15321](https://github.com/ClickHouse/ClickHouse/pull/15321) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加表函数 `null('structure')` . [#14797](https://github.com/ClickHouse/ClickHouse/pull/14797) ([vxider](https://github.com/Vxider)). -* 添加了 `formatReadableQuantity` 函数. 它对于人类读取大数字很有用. [#14725](https://github.com/ClickHouse/ClickHouse/pull/14725) ([Artem Hnilov](https://github.com/BooBSD)). -* 添加格式 `LineAsString` 接受由换行符分隔的一系列行, 每一行都作为一个整体解析为单个 String 字段. [#14703](https://github.com/ClickHouse/ClickHouse/pull/14703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)), [#13846](https://github.com/ClickHouse/ClickHouse/pull/13846) ([hexiaoting](https://github.com/hexiaoting)). -* 添加以字符串数组形式输出数据的 `JSONStrings` 格式. [#14333](https://github.com/ClickHouse/ClickHouse/pull/14333) ([hcz](https://github.com/hczhcz)). -* 为`Regexp` 格式添加对 `Raw` 列格式的支持。 它允许简单地将子模式作为一个整体提取,而无需任何转义规则. [#15363](https://github.com/ClickHouse/ClickHouse/pull/15363) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许 `TSV` 输出格式的可配置 `NULL` 表示. 它由设置 `output_format_tsv_null_representation` 控制, 默认情况下为 `\N`. 这将关闭 [#9375](https://github.com/ClickHouse/ClickHouse/issues/9375). 请注意, 该设置仅控制输出格式, 并且 `\N` 是 `TSV` 输入格式唯一支持的 `NULL` 表示. [#14586](https://github.com/ClickHouse/ClickHouse/pull/14586) ([Kruglov Pavel](https://github.com/Avogar)). -* 支持 `MaterializeMySQL` 的十进制数据类型。 `MaterializeMySQL` 是一个实验性功能. [#14535](https://github.com/ClickHouse/ClickHouse/pull/14535) ([Winter Zhang](https://github.com/zhang2014)). -* 添加新功能: `SHOW DATABASES LIKE 'xxx'`. [#14521](https://github.com/ClickHouse/ClickHouse/pull/14521) ([hexiaoting](https://github.com/hexiaoting)). -* 添加了将 (任意) git 存储库作为示例数据集导入到 ClickHouse 的脚本. [#14471](https://github.com/ClickHouse/ClickHouse/pull/14471) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在插入语句可以在列列表中带有带有列转换器的星号 (或变体). [#14453](https://github.com/ClickHouse/ClickHouse/pull/14453) ([Amos Bird](https://github.com/amosbird)). -* 用于分布式查询的新查询复杂度限制设置 `max_rows_to_read_leaf`、`max_bytes_to_read_leaf`, 以限制在叶节点上读取的最大行数/字节数. 限制仅适用于本地读取, *排除*根节点上的最终合并阶段. [#14221](https://github.com/ClickHouse/ClickHouse/pull/14221) ([Roman Khavronenko](https://github.com/hagen1778)). -* 允许用户在配置文件的 `` 部分为 `ReplicatedMergeTree*` 存储指定设置. 它的工作原理类似于 `` 部分. 对于 `ReplicatedMergeTree*` 存储, `` 和 `` 中的设置被一起应用, 但来自 `` 的设置具有更高的优先级. 添加了`system.replicated_merge_tree_settings`表. [#13573](https://github.com/ClickHouse/ClickHouse/pull/13573) ([Amos Bird](https://github.com/amosbird)). -* 添加 `mapPopulateSeries` 函数. [#13166](https://github.com/ClickHouse/ClickHouse/pull/13166) ([Ildus Kurbangaliev](https://github.com/ildus)). -* 支持 MySQL 类型:`decimal` (作为 ClickHouse`Decimal`) 和具有亚秒级精度的 `datetime` (作为 `DateTime64`). [#11512](https://github.com/ClickHouse/ClickHouse/pull/11512) ([Vasily Nemkov](https://github.com/Enmk)). -* 将 `event_time_microseconds` 字段引入 `system.text_log`、`system.trace_log`、`system.query_log` 和 `system.query_thread_log` 表. [#14760](https://github.com/ClickHouse/ClickHouse/pull/14760) ([Bharat Nallan](https://github.com/bharatnc)). -* 将 `event_time_microseconds` 添加到 `system.asynchronous_metric_log` 和 `system.metric_log` 表. [#14514](https://github.com/ClickHouse/ClickHouse/pull/14514) ([Bharat Nallan](https://github.com/bharatnc)). -* 将 `query_start_time_microseconds` 字段添加到 `system.query_log` 和 `system.query_thread_log` 表. [#14252](https://github.com/ClickHouse/ClickHouse/pull/14252) ([Bharat Nallan](https://github.com/bharatnc)). - -#### Bug 修复 - -* 修复无论限制如何都可以过度分配内存的情况. 这将关闭 [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `executable` 字典源挂起. 在以前的版本中, 当使用某些格式 (例如`JSONEachRow`) 时, 数据在至少输出一些东西之前不会提供给子进程. 这将关闭 [#1697](https://github.com/ClickHouse/ClickHouse/issues/1697). 这将关闭 [#2455](https://github.com/ClickHouse/ClickHouse/issues/2455). [#14525](https://github.com/ClickHouse/ClickHouse/pull/14525) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在函数 `dictGet` 出现异常时修复double free. 如果字典加载错误, 可能会发生这种情况. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用总数/汇总/多维数据集修饰符和最小/最大功能按键修复分组. 修复 [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). -* 使用 `prefer_localhost_replica=0` 和 `internal_replication` 修复异步分布式插入. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). -* 修复 TwoLevelStringHashTable 实现中一个很错误的代码,可能导致内存泄漏. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). -* 修复了在lambda中某些错误聚合情况下的段错误. [#16082](https://github.com/ClickHouse/ClickHouse/pull/16082) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 `ReplicatedVersionedCollapsingMergeTree` 的 `ALTER MODIFY ... ORDER BY` 查询挂起。 这修复了 [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). -* `MaterializeMySQL` (实验功能): 修复整理名称和字符集名称解析器并支持字符串类型的 `length = 0`. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). -* 允许对具有复杂键的字典使用 `direct` 布局. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). -* 在一段时间不活动后发生复制错误时,防止副本挂起 5-10 分钟. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). -* 在插入或从 MaterializedView 中选择并同时删除目标表时修复罕见的段错误 (适用于原子数据库引擎). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). -* 修复解析设置配置文件时的歧义: `CREATE USER ... SETTINGS profile readonly` 现在被视为使用名为 `readonly` 的配置文件, 而不是名为 `profile` 的具有只读约束的设置. 这修复了 [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). -* `MaterializeMySQL` (实验功能): 修复创建数据库失败时的崩溃. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了当表被并发重命名 (对于原子数据库引擎) 时, `DROP TABLE IF EXISTS` 失败并带有 `Table ... does not exist` 错误的问题. 修复了并发执行具有多个表的某些 DDL 查询 (如`DROP DATABASE` 和 `RENAME TABLE` ) 时罕见的死锁 - 修复了并发执行 `DROP/DETACH` 时 `Table ... does not exist` 的 `DROP/DETACH DATABASE` 失败表`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). -* 如果查询具有 `WHERE`、`PREWHERE` 和`GLOBAL IN`, 则修复来自`Distributed` 表的查询的错误空结果. 修复[#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 [#12513](https://github.com/ClickHouse/ClickHouse/issues/12513): 重新分析查询时具有相同别名的差异表达式. [#15886](https://github.com/ClickHouse/ClickHouse/pull/15886) ([Winter Zhang](https://github.com/zhang2014)). -* 修复 RBAC 实现中可能非常罕见的死锁. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复在 `ALTER MODIFY COLUMN` 查询之后执行的 `SELECT ... ORDER BY DESC` 查询中的异常 `Block structure mismatch` . 修复 [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). -* `MaterializeMySQL` (实验功能): 修复 `select count()` 不准确. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). -* 修复某些查询情况, 其中仅选择虚拟列. 以前可能会抛出 `Not found column _nothing in block` 异常. 修复 [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). -* 修复原子数据库中具有内表的物化视图的删除 (由于工作线程挂起, 由于 MV 内表的递归 DROP TABLE, 因此挂起所有后续 DROP TABLE). [#15743](https://github.com/ClickHouse/ClickHouse/pull/15743) ([Azat Khuzhin](https://github.com/azat)). -* 如果第一次尝试失败, 可以将部分移动到另一个磁盘/卷. [#15723](https://github.com/ClickHouse/ClickHouse/pull/15723) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复错误 `Cannot find column`, 如果对 `MV` 的查询包含 `ARRAY JOIN` , 则在插入 `MATERIALIZED VIEW` 时可能会发生该错误. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 `max_replicated_logs_to_keep` 设置的默认值过低, 这可能导致副本过于频繁地丢失. 通过选择要克隆的最新副本来改进丢失的副本恢复过程. 也不要从丢失的复制品中取出旧零件, 而是将它们拆下. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). -* 修复 MySQL 字典和表中罕见的竞争条件. [#15686](https://github.com/ClickHouse/ClickHouse/pull/15686) ([alesapin](https://github.com/alesapin)). -* 修复 AMQP-CPP 中的 (良性) 竞争条件. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). -* 修复错误 `Cannot add simple transform to empty Pipe` , 该错误在从与目标表结构不同的 `Buffer` 表中读取时发生. 如果目标表返回空的查询结果是可能的. 修复 [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 在使用 S3 插入 MergeTree 期间正确处理错误. S3 上的 MergeTree 是一项实验性功能. [#15657](https://github.com/ClickHouse/ClickHouse/pull/15657) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复了 S3 表功能的错误: 来自 URL 的区域未应用于 S3 客户端配置. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复查询计划的 ReadFromStorage 步骤中资源的销毁顺序. 在极少数情况下, 它可能会导致崩溃. 可能与[#15610](https://github.com/ClickHouse/ClickHouse/issues/15610)有关. [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 分离只读表时减去`ReadonlyReplica` 指标. [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). -* 修复了使用 `JSON*` 函数时会导致 `VALUES`、`LIMIT` 或 `IN` 运算符右侧的 `Element ... is not a constant expression` 错误. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). -* 如果出现异常, 查询将更快完成. 如果发生异常, 取消对远程副本的执行. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). -* 防止出现错误消息 `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call` 的可能性. 这修复了 [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `Database does not exist.` 在启动器上没有数据库时使用 IN 和分布式表的查询. [#15538](https://github.com/ClickHouse/ClickHouse/pull/15538) ([Artem Zuikov](https://github.com/4ertus2)). -* Mutation 可能会在 `MOVE` 或 `REPLACE PARTITION` 之后等待一些不存在的部分, 或者在极少数情况下, 在 `DETACH` 或 `DROP PARTITION` 之后挂起. 已修复. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* 修复如果执行具有相同模式的 `LIKE` , 则 `LIKE` 运算符不再区分大小写的错误. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). -* 修复了当选择数据中没有的列,但依赖于数据中也没有的其他列时的 `Missing columns` 错误. 修复 [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). -* 将单个参数传递给 ReplicatedMergeTree 而不是忽略它时抛出错误. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). -* 修复了 DDLWorker 中事件订阅的错误, 该错误很少会导致查询在 `ON CLUSTER` 中挂起. 引入 [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). -* 当 `boundingRatio` 聚合函数的第二个参数类型错误时报告正确的错误. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). -* 修复 [#15365](https://github.com/ClickHouse/ClickHouse/issues/15365): 附加带有 MySQL 引擎的数据库抛出异常 (无查询上下文) . [#15384](https://github.com/ClickHouse/ClickHouse/pull/15384) ([Winter Zhang](https://github.com/zhang2014)). -* 修复 select 查询中多次出现列转换器的情况. [#15378](https://github.com/ClickHouse/ClickHouse/pull/15378) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `S3` 存储中的压缩. [#15376](https://github.com/ClickHouse/ClickHouse/pull/15376) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复诸如 `SELECT toStartOfDay(today())` 之类的查询失败抱怨空 time_zone 参数的错误. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复合并树表重命名和背景清理过程中的竞争条件. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). -* 修复启用系统日志时服务器启动时罕见的竞争条件. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). -* 修复带有大量子查询的查询挂起到 `MySQL` 引擎的同一个表. 以前, 如果查询中对同一个 `MySQL` 表的子查询超过 16 个, 它将永远挂起. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 QueryLog 中的 MSan 报告. 未初始化的内存可用于字段 `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当查询在 Merge 表上有 JOIN 时,修复 GROUP BY 中的 `Unknown identifier` . [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了将 `joinGet` 与 `LowCardinality` 类型结合使用时的实例崩溃问题. 这修复了 [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). -* 修复表引擎 `Buffer` 中的错误, 该错误不允许在 `ALTER` 查询后将新结构的数据插入到 `Buffer` 中. 修复 [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). -* 调整 MySQL 列定义包中的 Decimal 字段大小. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). -* 修复了 `join_algorithm='auto'` 中的`用不同方法压缩的数据`. 在 `join_algorithm='partial_merge'` 中保持 LowCardinality 作为左表连接键的类型. [#15088](https://github.com/ClickHouse/ClickHouse/pull/15088) ([Artem Zuikov](https://github.com/4ertus2)). -* 更新 `jemalloc` 以修复带有亲和掩码的 `percpu_arena`. [#15035](https://github.com/ClickHouse/ClickHouse/pull/15035) ([Azat Khuzhin](https://github.com/azat)). [#14957](https://github.com/ClickHouse/ClickHouse/pull/14957) ([Azat Khuzhin](https://github.com/azat)). -* 我们已经在 String 和 FixedString 之间使用了填充比较 (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333) . 此 PR 将相同的逻辑应用于字段比较, 以更正 FixedString 作为主键的使用. 这修复了 [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). -* 如果使用专门设计的参数调用函数 `bar`, 则可能会发生缓冲区溢出. 这将关闭 [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在 Mac OS 上的 Docker 中运行 clickhouse-server 时, 在原子数据库中执行 DDL 查询时出现的 `Cannot rename ... errno: 22, strerror: Invalid argument` 错误. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). -* 当超过内存限制时, 使用 join_algorith='auto' 修复 RIGHT 或 FULL JOIN 中的崩溃, 我们应该使用 MergeJoin 更改 HashJoin. [#15002](https://github.com/ClickHouse/ClickHouse/pull/15002) ([Artem Zuikov](https://github.com/4ertus2)). -* 现在设置 `number_of_free_entries_in_pool_to_execute_mutation` 和 `number_of_free_entries_in_pool_to_lower_max_size_of_merge` 可以等于 `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). -* 当子查询包含`finalizeAggregation` 函数时, 修复使谓词下推工作. 修复 [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* 在 `system.asynchronous_metrics` 中发布每个逻辑核心的 CPU 频率. 这修复了 [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* `MaterializeMySQL` (实验功能): 修复了 `.metadata.tmp File exists` 错误. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). -* 修复部分调用 `extractAllGroups` 函数可能触发 `Memory limit exceeded` 错误的问题. 这修复了 [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 SIGSEGV 以尝试使用文件描述符插入 StorageFile. [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `cache` 字典中的段错误 [#14837](https://github.com/ClickHouse/ClickHouse/issues/14837). [#14879](https://github.com/ClickHouse/ClickHouse/pull/14879) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* `MaterializeMySQL` (实验功能): 修复了解析 MySQL 二进制日志事件的错误,该错误导致 `MaterializeMySQL` 数据库引擎中的 `Attempt to read after eof` 和 `Packet payload is not fully read`. [#14852](https://github.com/ClickHouse/ClickHouse/pull/14852) ([Winter Zhang](https://github.com/zhang2014)). -* 当被查询的列具有 `DEFAULT` 表达式时, 修复 `SELECT` 查询中的罕见错误, 该表达式依赖于另一列, 该列也具有 `DEFAULT` 并且不存在于选择查询中且不存在于磁盘上. 部分修复 [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* 如果必须从 ZK 获取配置文件 (使用 `from_zk` 包含选项) , 则修复服务器在与 ZooKeeper 交谈时可能会在启动时卡住的问题. 这修复了 [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复了收缩的`Int -> Int` 签名类型转换的错误单调性检测. 可能会导致查询结果不正确. 此错误在 [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513) 中公布. [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). -* `Replace` 列转换器应该用克隆的 AST 替换标识符. 这修复了 [#14695](https://github.com/ClickHouse/ClickHouse/issues/14695). [#14734](https://github.com/ClickHouse/ClickHouse/pull/14734) ([Amos Bird](https://github.com/amosbird)). -* 修复了执行 `ALTER ... MODIFY QUERY` 时物化视图元数据中丢失的默认数据库名称. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). -* 修复了当赋值表达式和常量值 (如 `UPDATE x = 42` ) 中带有 `Nullable` 列的 `ALTER UPDATE` 突变导致列或段错误中的值不正确时的错误. 修复 [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* 修复错误的十进制乘法结果导致结果列小数位数错误. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复函数 `has` 与 `LowCardinality` 的 `Nullable` . [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). -* 在为 StorageReplicatedMergeTree 引擎创建查询期间 Zookeeper 异常后清理数据目录. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复带有组合子 `-Resample` 的函数中罕见的段错误, 这可能会出现在参数非常大的溢出结果中. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了将 `Nullable(String)` 转换为 Enum 时的错误. 由 [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) 介绍. 这修复了 [#14435](https://github.com/ClickHouse/ClickHouse/issues/14435). [#14530](https://github.com/ClickHouse/ClickHouse/pull/14530) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `Nullable` 列不正确的排序顺序. 这修复了 [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复 `currentDatabase()` 函数不能在 `ON CLUSTER` ddl 查询中使用. [#14211](https://github.com/ClickHouse/ClickHouse/pull/14211) ([Winter Zhang](https://github.com/zhang2014)). -* `MaterializeMySQL` (实验功能): 修复了 `MaterializeMySQL` 数据库引擎中的 `Packet payload is not full read` 错误. [#14696](https://github.com/ClickHouse/ClickHouse/pull/14696) ([BohuTANG](https://github.com/BohuTANG)). - -#### 改进 - -* 默认情况下为新创建的数据库启用 `Atomic` 数据库引擎. [#15003](https://github.com/ClickHouse/ClickHouse/pull/15003) ([tavplubix](https://github.com/tavplubix)). -* 添加为具有子类型的列指定专用编解码器的功能, 如 `Delta`、`T64` 等. 实现 [#12551](https://github.com/ClickHouse/ClickHouse/issues/12551), 修复 [#11397](https://github.com/ClickHouse/ClickHouse/issues/11397), 修复 [#4609](https://github.com/ClickHouse/ClickHouse/issues/4609). [#15089](https://github.com/ClickHouse/ClickHouse/pull/15089) ([alesapin](https://github.com/alesapin)). -* 动态重新加载zookeeper配置. [#14678](https://github.com/ClickHouse/ClickHouse/pull/14678) ([sundyli](https://github.com/sundy-li)). -* 现在, 无论集群配置中的 `` 设置如何, 都可以执行 `ALTER ... ON CLUSTER` 查询. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* 现在 `joinGet` 支持多键查找。 续. [#13015](https://github.com/ClickHouse/ClickHouse/pull/13015) ([Amos Bird](https://github.com/amosbird)). -* 如果为 `Atomic` 数据库指定了 `NO DELAY` 或 `SYNC`, 则等待 `DROP/DETACH TABLE` 实际完成. [#15448](https://github.com/ClickHouse/ClickHouse/pull/15448) ([tavplubix](https://github.com/tavplubix)). -* 现在可以使用 `ALTER` 查询更改 `VersionedCollapsingMergeTree` 的版本列的类型. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). -* 在复制表创建时展开 `zookeeper_path` 中的 `{database}`、`{table}` 和 `{uuid}` 宏. 如果在服务器重启后可能会破坏 `zookeeper_path`, 不要允许 `RENAME TABLE`. 修复 [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). -* 函数 `now` 允许使用时区参数. 这关闭 [15264](https://github.com/ClickHouse/ClickHouse/issues/15264). [#15285](https://github.com/ClickHouse/ClickHouse/pull/15285) ([flynn](https://github.com/ucasFL)). -* 在执行 `/docker-entrypoint-initdb.d/` 中的所有脚本之前. 不允许连接到 ClickHouse 服务器. [#15244](https://github.com/ClickHouse/ClickHouse/pull/15244) ([Aleksei Kozharin](https://github.com/alekseik1)). -* 向 `EXPLAIN PLAN` 查询添加了 `optimize` 设置. 如果启用, 则应用查询计划级别优化. 默认启用. [#15201](https://github.com/ClickHouse/ClickHouse/pull/15201) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* CAST 参数数量错误的正确异常消息. 这将关闭 [#13992](https://github.com/ClickHouse/ClickHouse/issues/13992). [#15029](https://github.com/ClickHouse/ClickHouse/pull/15029) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加选项以在数据部分插入时禁用 TTL 移动. [#15000](https://github.com/ClickHouse/ClickHouse/pull/15000) ([Pavel Kovalenko](https://github.com/Jokser)). -* 进行突变时忽略关键约束. 没有这个拉取请求, 当 `force_index_by_date = 1` 或 `force_primary_key = 1` 时不可能进行突变. [#14973](https://github.com/ClickHouse/ClickHouse/pull/14973) ([Amos Bird](https://github.com/amosbird)). -* 如果之前的删除尝试由于 ZooKeeper 会话过期而失败, 则允许删除复制表. 这修复了 [#11891](https://github.com/ClickHouse/ClickHouse/issues/11891). [#14926](https://github.com/ClickHouse/ClickHouse/pull/14926) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了从分布式表中使用 SETTINGS 运行 SELECT 时过多的设置约束冲突. [#14876](https://github.com/ClickHouse/ClickHouse/pull/14876) ([Amos Bird](https://github.com/amosbird)). -* 提供一个 `load_balancing_first_offset` 查询设置来明确说明第一个副本是什么. 它与“FIRST_OR_RANDOM”负载均衡策略一起使用, 允许控制副本工作负载. [#14867](https://github.com/ClickHouse/ClickHouse/pull/14867) ([Amos Bird](https://github.com/amosbird)). -* 在 `EXPLAIN` 结果中显示 `SET` 和 `JOIN` 的子查询. [#14856](https://github.com/ClickHouse/ClickHouse/pull/14856) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 允许在存储 `Distributed` 中使用多卷存储配置. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). -* 从相同的 timespec 构造 `query_start_time` 和 `query_start_time_microseconds`. [#14831](https://github.com/ClickHouse/ClickHouse/pull/14831) ([Bharat Nallan](https://github.com/bharatnc)). -* 支持禁用 `StorageJoin` 和 `StorageSet` 的持久化, 这个特性是通过设置`disable_set_and_join_persistency`来控制的. 这个 PR 解决了问题 [#6318](https://github.com/ClickHouse/ClickHouse/issues/6318). [#14776](https://github.com/ClickHouse/ClickHouse/pull/14776) ([vxider](https://github.com/Vxider)). -* 现在 `COLUMNS` 可用于包装列列表并在之后应用列转换器. [#14775](https://github.com/ClickHouse/ClickHouse/pull/14775) ([Amos Bird](https://github.com/amosbird)). -* 将 `merge_algorithm` 添加到 `system.merges` 表以改进合并检查. [#14705](https://github.com/ClickHouse/ClickHouse/pull/14705) ([Amos Bird](https://github.com/amosbird)). -* 修复 zookeeper 存在 watch 导致的潜在内存泄漏. [#14693](https://github.com/ClickHouse/ClickHouse/pull/14693) ([hustnn](https://github.com/hustnn)). -* 允许并行执行分布式 DDL. [#14684](https://github.com/ClickHouse/ClickHouse/pull/14684) ([Azat Khuzhin](https://github.com/azat)). -* 添加 `QueryMemoryLimitExceeded` 事件计数器. 这将关闭 [#14589](https://github.com/ClickHouse/ClickHouse/issues/14589). [#14647](https://github.com/ClickHouse/ClickHouse/pull/14647) ([fastio](https://github.com/fastio)). -* 修复查询格式中的一些尾随空格. [#14595](https://github.com/ClickHouse/ClickHouse/pull/14595) ([Azat Khuzhin](https://github.com/azat)). -* ClickHouse 对分区 expr 和键 expr 的处理方式不同. 分区 expr 用于构造包含相关列的 minmax 索引, 而主键 expr 存储为 expr. 有时用户可能会在更粗略的级别对表进行分区, 例如 `partition by i / 1000`. 然而, 二元运算符不是单调的, 这个 PR 试图解决这个问题. 它也可能有益于其他用例. [#14513](https://github.com/ClickHouse/ClickHouse/pull/14513) ([Amos Bird](https://github.com/amosbird)). -* 添加一个选项以跳过对 `DiskS3` 的访问检查. `s3` 磁盘是一项实验性功能. [#14497](https://github.com/ClickHouse/ClickHouse/pull/14497) ([Pavel Kovalenko](https://github.com/Jokser)). -* 如果有正在进行的 S3 请求, 则加快服务器关闭过程. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). -* 如果重新加载失败并继续使用以前的 users.xml, `SYSTEM RELOAD CONFIG` 现在会抛出异常. 如果重新加载失败, 后台定期重新加载也会继续使用以前的 users.xml. [#14492](https://github.com/ClickHouse/ClickHouse/pull/14492) ([Vitaly Baranov](https://github.com/vitlibar)). -* 对于 `clickhouse-client` 的脚本模式下带有 VALUES 格式的内联数据的 INSERT, 除了换行外, 还支持分号作为数据终止符. 关闭 [#12288](https://github.com/ClickHouse/ClickHouse/issues/12288). [#13192](https://github.com/ClickHouse/ClickHouse/pull/13192) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 支持紧凑部分的自定义编解码器. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - -#### 性能改进 - -* 默认情况下为小零件启用紧凑零件. 这将允许稍微更有效地处理频繁插入 (4..100 次). [#11913](https://github.com/ClickHouse/ClickHouse/pull/11913) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 提高 `quantileTDigest` 性能. 这修复了 [#2668](https://github.com/ClickHouse/ClickHouse/issues/2668). [#15542](https://github.com/ClickHouse/ClickHouse/pull/15542) ([Kruglov Pavel](https://github.com/Avogar)). -* 显著减少 AggregatingInOrderTransform/optimize_aggregation_in_order 中的内存使用. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). -* 更快的 256 位乘法. [#15418](https://github.com/ClickHouse/ClickHouse/pull/15418) ([Artem Zuikov](https://github.com/4ertus2)). -* 使用 (u)int64_t 作为宽整数的基本类型提高 256 位类型的性能. 原始宽整数使用 8 位类型作为基数. [#14859](https://github.com/ClickHouse/ClickHouse/pull/14859) ([Artem Zuikov](https://github.com/4ertus2)). -* 显式使用临时磁盘存储垂直合并临时数据. [#15639](https://github.com/ClickHouse/ClickHouse/pull/15639) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). -* 在循环中使用一个 S3 DeleteObjects 请求而不是多个 DeleteObject. 没有任何功能更改, 因此由现有测试 (如集成/test_log_family_s3) 覆盖. [#15238](https://github.com/ClickHouse/ClickHouse/pull/15238) ([ianton-ru](https://github.com/ianton-ru)). -* 修复 `DateTime DateTime` 错误地选择了缓慢的通用实现. 这修复了 [#15153](https://github.com/ClickHouse/ClickHouse/issues/15153). [#15178](https://github.com/ClickHouse/ClickHouse/pull/15178) ([Amos Bird](https://github.com/amosbird)). -* 提高 `FixedString` 类型的 GROUP BY 键的性能. [#15034](https://github.com/ClickHouse/ClickHouse/pull/15034) ([Amos Bird](https://github.com/amosbird)). -* 启动 clickhouse-server 时只有 `mlock` 代码段. 在以前的版本中, 所有映射区域都被锁定在内存中, 包括调试信息. 调试信息通常被拆分为一个单独的文件, 但如果不是, 则会导致 +2..3 GiB 内存使用. [#14929](https://github.com/ClickHouse/ClickHouse/pull/14929) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 由于链接时间优化, ClickHouse 二进制文件变得更小. - -#### 构建/测试/打包改进 - -* 现在我们使用 clang-11 进行生产 ClickHouse 构建. [#15239](https://github.com/ClickHouse/ClickHouse/pull/15239) ([alesapin](https://github.com/alesapin)). -* 现在我们使用 clang-11 在 CI 中构建 ClickHouse. [#14846](https://github.com/ClickHouse/ClickHouse/pull/14846) ([alesapin](https://github.com/alesapin)). -* 将二进制构建 (Linux、Darwin、AArch64、FreeDSD) 切换到 clang-11. [#15622](https://github.com/ClickHouse/ClickHouse/pull/15622) ([Ilya Yatsishin](https://github.com/qoega)). -* 现在所有测试图像都使用 `llvm-symbolizer-11`. [#15069](https://github.com/ClickHouse/ClickHouse/pull/15069) ([alesapin](https://github.com/alesapin)). -* 允许使用 llvm-11 构建. [#15366](https://github.com/ClickHouse/ClickHouse/pull/15366) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 从 `clang-tidy-10` 切换到 `clang-tidy-11` . [#14922](https://github.com/ClickHouse/ClickHouse/pull/14922) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 默认使用 LLVM 的实验通行证管理器. [#15608](https://github.com/ClickHouse/ClickHouse/pull/15608) ([Danila Kutenin](https://github.com/danlark1)). -* 不允许任何 C++ 翻译单元构建超过 10 分钟或使用超过 10 GB 或内存. 这修复了 [#14925](https://github.com/ClickHouse/ClickHouse/issues/14925). [#15060](https://github.com/ClickHouse/ClickHouse/pull/15060) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过拆分测试运行和配置文件运行, 使性能测试更加稳定和具有代表性. [#15027](https://github.com/ClickHouse/ClickHouse/pull/15027) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 尝试使性能测试更可靠. 它是通过使用 `madvise` 动态重新映射进程的可执行内存以使用透明大页面来完成的 - 它可以降低 iTLB 未命中的数量, 这是性能测试中不稳定的主要来源. [#14685](https://github.com/ClickHouse/ClickHouse/pull/14685) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 转换为python3. 这将关闭 [#14886](https://github.com/ClickHouse/ClickHouse/issues/14886). [#15007](https://github.com/ClickHouse/ClickHouse/pull/15007) ([Azat Khuzhin](https://github.com/azat)). -* 如果服务器未能响应, 则在功能测试中尽早失败. 这将关闭 [#15262](https://github.com/ClickHouse/ClickHouse/issues/15262). [#15267](https://github.com/ClickHouse/ClickHouse/pull/15267) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许在没有配置的情况下运行 AArch64 版本的 clickhouse-server. 这有利于 [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174). [#15266](https://github.com/ClickHouse/ClickHouse/pull/15266) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* CI docker 镜像的改进: 摆脱 ZooKeeper 和用于测试配置安装的单个脚本. [#15215](https://github.com/ClickHouse/ClickHouse/pull/15215) ([alesapin](https://github.com/alesapin)). -* 在快速测试脚本中修复 CMake 选项转发. 修复了 [#14711](https://github.com/ClickHouse/ClickHouse/issues/14711) 中的错误. [#15155](https://github.com/ClickHouse/ClickHouse/pull/15155) ([alesapin](https://github.com/alesapin)). -* 添加了一个脚本以在单个命令中执行硬件基准测试. [#15115](https://github.com/ClickHouse/ClickHouse/pull/15115) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 将巨大的测试 `test_dictionaries_all_layouts_and_sources` 拆分成较小的测试. [#15110](https://github.com/ClickHouse/ClickHouse/pull/15110) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 也许修复 base64 中的 MSan 报告 (在带有 AVX-512 的服务器上). 这修复了. [#15030](https://github.com/ClickHouse/ClickHouse/pull/15030) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 重新格式化和清理所有集成测试 *.py 文件中的代码. [#14864](https://github.com/ClickHouse/ClickHouse/pull/14864) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复在 CI 中发现的 MaterializeMySQL 空事务不稳定测试用例. [#14854](https://github.com/ClickHouse/ClickHouse/pull/14854) ([Winter Zhang](https://github.com/zhang2014)). -* 尝试加快构建速度. [#14808](https://github.com/ClickHouse/ClickHouse/pull/14808) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过删除未使用的标头来加快构建速度. [#14714](https://github.com/ClickHouse/ClickHouse/pull/14714) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 OSX 中的构建失败. [#14761](https://github.com/ClickHouse/ClickHouse/pull/14761) ([Winter Zhang](https://github.com/zhang2014)). -* 如果在操作系统中找到 ccache, 则默认在 cmake 中启用. [#14575](https://github.com/ClickHouse/ClickHouse/pull/14575) ([alesapin](https://github.com/alesapin)). -* Control CI 从 ClickHouse 存储库构建配置. [#14547](https://github.com/ClickHouse/ClickHouse/pull/14547) ([alesapin](https://github.com/alesapin)). -* 在 CMake 文件中: - 将一些选项的描述部分移动到上面的注释中. - 替换 0 -> `OFF`, 1 -> `option` 的默认值中的 `ON`. - 在选项中添加了一些说明和文档链接. - 替换了 `FUZZER` 选项 (还有另一个选项 `ENABLE_FUZZING` 也可以启用相同的功能). - 删除了 `ENABLE_GTEST_LIBRARY` 选项, 因为有 `ENABLE_TESTS`. 请参阅 PR 中的完整说明: [#14711](https://github.com/ClickHouse/ClickHouse/pull/14711) ([Mike](https://github.com/myrrc)). -* 使二进制文件更小 (调试版本约为 50 Mb). [#14555](https://github.com/ClickHouse/ClickHouse/pull/14555) ([Artem Zuikov](https://github.com/4ertus2)). -* 在 ConfigProcessor 中使用 std::filesystem::path 连接文件路径. [#14558](https://github.com/ClickHouse/ClickHouse/pull/14558) ([Bharat Nallan](https://github.com/bharatnc)). -* 使用负大整数调用时修复 `bitShiftLeft()` 中的调试断言. [#14697](https://github.com/ClickHouse/ClickHouse/pull/14697) ([Artem Zuikov](https://github.com/4ertus2)). - - -## ClickHouse 版本 20.9 - -### ClickHouse 版本 v20.9.7.11-stable, 2020-12-07 - -#### 性能改进 - -* 修复从 `Merge` 表读取大量 `MergeTree` 表的性能. 修复 [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). - -#### Bug 修复 - -* 如果禁用了`in_memory_parts_enable_wal`, 则不要从 WAL 恢复部件. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* 修复了插入 `Distributed` 表时空间不足时的段错误. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). -* 修复了 ClickHouse 无法恢复与 MySQL 服务器的连接时的问题. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复了在 Linux 的 Windows 子系统上运行的 ClickHouse 在 `Atomic` 数据库中执行 `RENAME` 查询时出现的 `Function not implementation` 错误. 修复 [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). -* 当 clickhouse-client 用于多行查询的交互模式时, 单行注释被错误地扩展到查询结束. 这修复了 [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复在极少数情况下服务器可以停止接受连接的问题. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了当不同副本上的相应突变被杀死时 alter 查询挂起的问题. 修复 [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* 修复 clickhouse 低估标记缓存大小时的错误. 当有很多带有标记的小文件时可能会发生. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* 修复了启用 `optimize_redundant_functions_in_order_by` 设置的 `ORDER BY` . [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 `DISTINCT` 之后由于不正确优化而可能出现的重复项. 修复 [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了从具有 `LowCardinality` 类型的 `JOIN` 表读取时崩溃的问题. 修复 [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复子查询中存在 const 列时设置索引失效的问题. 这修复了 [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* 修复 ColumnConst 比较导致崩溃. 这个修复了 [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088). [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* 修复了在创建 `some_table` 时 `CREATE TABLE ... AS some_table` 查询崩溃 `AS table_function()` 修复 [#16944](https://github.com/ClickHouse/ClickHouse/issues/16944). [#17072](https://github.com/ClickHouse/ClickHouse/pull/17072) ([tavplubix](https://github.com/tavplubix)). -* 功能 fuzzBits 的 Bug 修复, 相关问题: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* 避免远程查询的不必要的网络错误, 这些错误在执行时可能会被取消, 例如使用 `LIMIT` 的查询. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* 备忘录. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). -* 通过 MySQL 协议返回 INSERT 查询的受影响行数. 以前 ClickHouse 过去总是返回 0, 它是固定的. 修复 [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). - -#### 构建/测试/打包改进 - -* 将嵌入的时区数据更新到 2020d 版 (同时将 cctz 更新到最新的 master). [#17204](https://github.com/ClickHouse/ClickHouse/pull/17204) ([filimonov](https://github.com/filimonov)). - - -### ClickHouse 版本 v20.9.6.14-stable, 2020-11-20 - -#### 改进 - -* 可以连接到需要 SNI 的 `clickhouse-server` 安全端点. 当 `clickhouse-server` 托管在 TLS 代理之后是可能的. [#16938](https://github.com/ClickHouse/ClickHouse/pull/16938) ([filimonov](https://github.com/filimonov)). -* 条件聚合函数 (例如:`avgIf`、`sumIf`、`maxIf`) 在缺少行并使用可为空参数时应返回 `NULL` . [#13964](https://github.com/ClickHouse/ClickHouse/pull/13964) ([Winter Zhang](https://github.com/zhang2014)). - -#### Bug 修复 - -* 修复非领导者 ReplicatedMergeTree 表的 `ON CLUSTER` 查询可能永远挂起的错误. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* 出现错误时解析 `format_avro_schema_registry_url` 的IP. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* 修复在 `ALTER TABLE ... MODIFY COLUMN ... NewType` 之后可能发生的服务器崩溃, 当 `SELECT` 在更改列上具有 `WHERE` 表达式并且更改尚未完成时. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* 安装脚本应始终在 config 文件夹中创建子目录. 这仅与使用自定义配置的 Docker 构建相关. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* 修复使用 `ORDER BY` 的查询可能出现的错误 `Illegal type of argument` . 修复 [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果没有数据写入 WriteBufferFromS3,则中止分段上传. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复在不带任何参数的情况下使用 `any` 时的崩溃. 这是针对 [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803) 的 . cc @azat. [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* 在启用了 `transform_null_in` 设置的情况下修复多个列和元组上的 `IN` 运算符. 修复 [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* 这将修复 optimize_read_in_order/optimize_aggregation_in_order 与 max_threads>0 和 ORDER BY 中的表达式. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* 修复 [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574), 修复 [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231), 修复使用时远程查询失败'if'后缀聚合函数. [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). -* 在异常的情况下查询完成得更快. 如果发生异常, 取消对远程副本的执行. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse 版本 v20.9.5.5-stable, 2020-11-13 - -#### Bug 修复 - -* 当查询分析器打开并且 ClickHouse 安装在操作系统上时, 修复罕见的无声崩溃, glibc 版本已经 (据说) 损坏了某些功能的异步展开表. 这修复了 [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). 这修复了 [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在, 当从输入解析 AVRO 时, LowCardinality 将从类型中删除. 修复 [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* 通过适当缩小 GTID 集,修复使用 MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine 和 MySQL Slave 上启用的 `slave_parallel_worker` 时元数据的快速增长. 这修复了 [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* 修复分布式的 DROP TABLE (使用 INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* 复复制队列中非常大的条目的处理. 如果表结构非常大 (接近 1 MB), 则非常大的条目可能会出现在 ALTER 查询中. 这修复了 [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了由于未创建过滤集而可能删除部分返回数据时的不一致行为. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复 MySQL 数据库的错误. 当用作数据库引擎的 MySQL 服务器关闭时, 某些查询会引发异常, 因为它们试图从禁用的服务器获取表, 而这是不必要的. 例如, 查询 `SELECT ... FROM system.parts` 应该只适用于 MergeTree 表并且根本不要接触 MySQL 数据库. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). - - -### ClickHouse 版本 v20.9.4.76-stable (2020-10-29) - -#### Bug 修复 - -* 在函数 `dictGet` 出现异常时修复double free. 如果字典加载错误, 可能会发生这种情况. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复由总数/汇总/多维数据集修饰符和最小/最大功能按键分组. 修复 [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). -* 修复异步分布式插入 w/prefer_localhost_replica=0 和 internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). -* 修复 TwoLevelStringHashTable 实现中的一个非常错误的代码, 这可能会导致内存泄漏. 我很惊讶这个 bug 怎么能潜伏这么久.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). -* 修复无论限制如何都可以过度分配内存的情况. 这将关闭 [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `ReplicatedVersionedCollapsingMergeTree` 的 `ALTER MODIFY ... ORDER BY` 查询挂起. 这修复了 [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). -* 修复整理名称和字符集名称解析器并支持字符串类型的 `length = 0`. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). -* 允许对具有复杂键的字典使用直接布局. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). -* 在一段时间不活动后发生复制错误时, 防止副本挂起 5-10 分钟. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). -* 在插入或从 MaterializedView 中选择并同时删除目标表时修复罕见的段错误 (适用于原子数据库引擎). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). -* 修复解析设置配置文件时的歧义: `CREATE USER ... SETTINGS profile readonly` 现在被视为使用名为 `readonly` 的配置文件, 而不是名为 `profile` 的具有只读约束的设置. 这修复了 [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复数据库创建失败时的崩溃. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了当表被并发重命名 (对于原子数据库引擎) 时, `DROP TABLE IF EXISTS` 失败并带有 `Table ... does not exist` 错误的问题. 修复了并发执行具有多个表的某些 DDL 查询 (如 `DROP DATABASE”和“RENAME TABLE` )时罕见的死锁 修复了并发执行 `DROP/DETACH TABLE` 时 `Table ... does not exist` 的 `DROP/DETACH DATABASE` 失败 `. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). -* 如果查询具有 `WHERE`、`PREWHERE` 和 `GLOBAL IN`, 则修复来自 `Distributed` 表的查询的错误空结果. 修复 [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 RBAC 中可能的死锁. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复在 `ALTER MODIFY COLUMN` 查询之后执行的 `SELECT ... ORDER BY DESC` 查询中的异常 `Block structure mismatch` . 修复 [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). -* 修复 MaterializeMySQL 的 `select count()` 不准确. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). -* 修复某些查询情况, 其中仅选择虚拟列. 以前可能会抛出 `Not found column _nothing in block` 异常. 修复 [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `max_replicated_logs_to_keep` 设置的默认值过低, 这可能导致副本过于频繁地丢失. 通过选择要克隆的最新副本来改进丢失的副本恢复过程. 也不要从丢失的复制品中取出旧零件, 而是将它们拆下. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). -* 修复错误 `Cannot add simple transform to empty Pipe` ,该错误在从与目标表结构不同的 `Buffer` 表中读取时发生. 如果目标表返回空的查询结果是可能的. 修复 [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 S3 表函数中的 globs 错误,URL 中的区域未应用于 S3 客户端配置. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 分离只读表时递减 `ReadonlyReplica` 指标. 这修复了 [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). -* 将单个参数传递给 ReplicatedMergeTree 而不是忽略它时抛出错误. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). - -#### 改进 - -* 现在, 无论集群配置中的 `` 设置如何,都可以执行 `ALTER ... ON CLUSTER` 查询. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* 在创建表时在 `ReplicatedMergeTree` 参数中展开 `{database}`、`{table}` 和 `{uuid}` 宏. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). - - -### ClickHouse 版本 v20.9.3.45-stable (2020-10-09) - -#### Bug 修复 - -* 修复错误 `Cannot find column` , 如果对 `MV` 的查询包含 `ARRAY JOIN` , 则在插入` MATERIALIZED VIEW` 时可能会发生该错误. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 AMQP-CPP 中的竞争条件. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). -* 修复查询计划的 ReadFromStorage 步骤中资源的销毁顺序. 在极少数情况下, 它可能会导致崩溃. 可能与[#15610](https://github.com/ClickHouse/ClickHouse/issues/15610)有关. [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了使用 `JSON*` 函数时会导致 `VALUES`、`LIMIT`或 `IN` 运算符右侧的 `Element ... is not a constant expression` 错误. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). -* 防止出现错误消息 `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call` 的可能性. 这修复了 [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 显著减少 AggregatingInOrderTransform/optimize_aggregation_in_order 中的内存使用. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). -* Mutation 可能会在 `MOVE` 或 `REPLACE PARTITION` 之后等待一些不存在的部分, 或者在极少数情况下, 在 `DETACH` 或 `DROP PARTITION` 之后挂起. 已修复. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* 修复如果执行具有相同模式的 `LIKE` , 则 `LIKE` 运算符不再区分大小写的错误. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). -* 在选择数据中不存在但依赖于数据中也不存在的其他列时修复 `Missing columns` 错误. 修复 [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). -* 修复了 DDLWorker 中事件订阅的错误, 该错误很少会导致查询在 `ON CLUSTER` 中挂起. 引入 [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). -* 当 `boundingRatio` 聚合函数的第二个参数类型错误时报告正确的错误. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). -* 修复诸如 `SELECT toStartOfDay(today())` 之类的查询失败抱怨空 time_zone 参数的错误. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复合并树表重命名和背景清理过程中的竞争条件. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). -* 修复启用 system.logs 时服务器启动时罕见的竞争条件. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). -* 修复 QueryLog 中的 MSan 报告. 未初始化的内存可用于 `memory_usage` 字段. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复将 joinGet 与 LowCardinality 类型一起使用时的实例崩溃. 这修复了 [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). -* 修复表引擎 `Buffer` 中的错误, 该错误不允许在 `ALTER` 查询后将新结构的数据插入到 `Buffer` 中. 修复 [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). -* 调整mysql列定义包中的decimals字段大小. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). -* 修复了在 Mac OS 上的 docker 中运行 clickhouse-server 时, 在原子数据库中执行 DDL 查询时出现的 `Cannot rename ... errno: 22, strerror: Invalid argument` 错误. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). -* 修复了当子查询包含 finalizeAggregation 函数时使谓词下推工作的问题.修复 [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* 如果必须从 ZK 获取配置文件 (使用 `from_zk` 包含选项), 则修复服务器在与 ZooKeeper 交谈时可能会在启动时卡住的问题. 这修复了 [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). - -#### 改进 - -* 现在可以使用 `ALTER` 查询更改 `VersionedCollapsingMergeTree` 的版本列的类型. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse 版本 v20.9.2.20, 2020-09-22 - -#### 向后不兼容变更 - -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重启, 旧版本在新版本存在的情况下启动, 可能会导致 到 `Part ... intersects previous part` 错误. 为防止出现此错误, 首先在所有集群节点上安装更新的 clickhouse-server 软件包, 然后重新启动 (因此, 当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* 添加了列转换器`EXCEPT`、`REPLACE`、`APPLY`, 可以应用于所选列的列表 (在`*` 或`COLUMNS(...)` 之后). 例如, 您可以编写 `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. 另一个例子: `select * apply(length) apply(max) from wide_string_table` 找出所有字符串列的最大长度. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). -* 添加了一个聚合函数 `rankCorr`, 用于计算秩相关系数. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 添加了表函数 `view`, 它将子查询转换为表对象. 这有助于传递查询. 例如, 它可以用于远程/集群表功能. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). - -#### Bug 修复 - -* 修复了在赋值表达式和常量值 (如 `UPDATE x = 42` )中使用 Nullable 列的 `ALTER UPDATE` 突变导致列或段错误中的值不正确时的错误. 修复[#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* 修复了错误的十进制乘法结果导致结果列小数位数错误. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了 `Nullable` 列不正确的排序顺序. 这修复了 [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了索引分析中与类型为 `FixedString` 的主键的比较不一致的问题, 如果它们与较小的字符串进行比较. 这修复了 [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). -* 修复了如果表具有单个部分的分区会导致错误的合并分配的错误. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). -* 如果使用专门设计的参数调用函数 `bar`, 则可能会发生缓冲区溢出. 这将关闭 [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 `system.asynchronous_metrics` 中发布每个逻辑核心的 CPU 频率. 这修复了 [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复了使用 `MaterializeMySQL` 数据库引擎时 `.metadata.tmp File exists` 的错误. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). -* 修复部分调用 `extractAllGroups` 函数可能触发 `Memory limit exceeded` 错误的问题. 这修复了 [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 SIGSEGV 以尝试插入 StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). -* 当被查询的列具有 `DEFAULT` 表达式时, 修复 `SELECT` 查询中的罕见错误, 该表达式依赖于另一列, 该列也具有 `DEFAULT` 并且不存在于选择查询中且不存在于磁盘上. 部分修复 [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* 修复了收缩的`Int -> Int` 签名类型转换的错误单调性检测. 可能会导致查询结果不正确. 此错误在 [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513) 中公布. [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). -* 修复了执行 `ALTER ... MODIFY QUERY` 时物化视图元数据中丢失的默认数据库名称. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). -* 修复当涉及 LowCardinality 和 Nullable 类型时函数 `has` 可能不正确的结果. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). -* 在使用 ReplicatedMergeTree Engine 对表进行 CREATE 查询期间, 在 Zookeeper 异常后清理数据目录. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复带有组合子 `-Resample` 的函数中罕见的段错误, 这可能会出现在参数非常大的溢出结果中. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). -* 检查 `topK` 聚合函数中的数组大小溢出. 如果没有此检查, 用户可能会发送带有精心设计的参数的查询, 这将导致服务器崩溃. 这将关闭 [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 代理重启/启动/停止/重新加载 SysVinit 到 systemd (如果使用). [#14460](https://github.com/ClickHouse/ClickHouse/pull/14460) ([Azat Khuzhin](https://github.com/azat)). -* 如果 PipelineExecutor 本身发生异常, 则停止查询执行. 这可以防止罕见的查询挂起. [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复在创建为 `AS table_function` 的表的 `ALTER` 查询期间崩溃. 修复 [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). -* 使用 REFRESH 命令修复 ALTER LIVE VIEW 查询期间的异常. LIVE VIEW 是一项实验性功能. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复带有嵌套解释器的查询的 QueryPlan 生命周期 (对于 EXPLAIN PIPELINE graph=1). [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). -* 更好地检查 SSD 缓存复杂键外部字典中的元组大小. 这修复了 [#13981](https://github.com/ClickHouse/ClickHouse/issues/13981). [#14313](https://github.com/ClickHouse/ClickHouse/pull/14313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 `ALIAS` 列类型上不允许使用 `CODEC` . 修复[#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). -* 在非全局级别执行时修复 GRANT ALL 语句. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复 lambda 中的 arrayJoin() 捕获 (抛出逻辑错误消息的异常). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). - -#### 实验功能 - -* 通过给定的 SELECT 查询添加了用于随机数据库生成的 `db-generator` 工具. 当用户只有不完整的错误报告时, 它可能有助于重现问题. [#14442](https://github.com/ClickHouse/ClickHouse/pull/14442) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [#10973](https://github.com/ClickHouse/ClickHouse/issues/10973) ([ZeDRoman](https://github.com/ZeDRoman)). - -#### 改进 - -* 允许在分布式存储中使用多卷存储配置. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). -* 禁止在 `toStartOf*` 类型的函数中使用空的 time_zone 参数. [#14509](https://github.com/ClickHouse/ClickHouse/pull/14509) ([Bharat Nallan](https://github.com/bharatnc)). -* MySQL 处理程序为诸如 `SET @@var = value` 之类的查询返回 `OK`. 此类声明被忽略. 这是必需的, 因为某些 MySQL 驱动程序会在握手后发送 `SET @@` 查询以进行设置 https://github.com/ClickHouse/ClickHouse/issues/9336#issuecomment-686222422 . [#14469](https://github.com/ClickHouse/ClickHouse/pull/14469) ([BohuTANG](https://github.com/BohuTANG)). -* 现在, 如果之前没有实现 TTL, 将在合并期间应用它们. [#14438](https://github.com/ClickHouse/ClickHouse/pull/14438) ([alesapin](https://github.com/alesapin)). -* 现在 `clickhouse-obfuscator` 支持 [#13163](https://github.com/ClickHouse/ClickHouse/issues/13163) 中提出的 UUID 类型. [#14409](https://github.com/ClickHouse/ClickHouse/pull/14409) ([dimarub2000](https://github.com/dimarub2000)). -* 添加了 [#11384](https://github.com/ClickHouse/ClickHouse/issues/11384) 中提出的新设置 `system_events_show_zero_values`. [#14404](https://github.com/ClickHouse/ClickHouse/pull/14404) ([dimarub2000](https://github.com/dimarub2000)). -* 在 `MaterializeMySQL` 中将主键隐式转换为非空 (与 `MySQL` 相同). 修复 [#14114](https://github.com/ClickHouse/ClickHouse/issues/14114). [#14397](https://github.com/ClickHouse/ClickHouse/pull/14397) ([Winter Zhang](https://github.com/zhang2014)). -* 用 https://github.com/cerevra/int 的实现替换 boost multiprecision 中的宽整数 (256 位). 256 位整数是实验性的. [#14229](https://github.com/ClickHouse/ClickHouse/pull/14229) ([Artem Zuikov](https://github.com/4ertus2)). -* 为 `system.part_log` 中的部分添加默认压缩编解码器, 名称为 `default_compression_codec`. [#14116](https://github.com/ClickHouse/ClickHouse/pull/14116) ([alesapin](https://github.com/alesapin)). -* 为`DateTime` 类型添加精度参数. 它允许使用 `DateTime` 名称代替 `DateTime64`. [#13761](https://github.com/ClickHouse/ClickHouse/pull/13761) ([Winter Zhang](https://github.com/zhang2014)). -* 为 Redis 外部字典添加了 requirepass 授权. [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). -* RabbitMQ 引擎的改进: 添加了连接和通道故障处理、正确提交、插入故障处理、更好的交换、队列持久性和队列恢复机会、新的队列设置. 固定测试. [#12761](https://github.com/ClickHouse/ClickHouse/pull/12761) ([Kseniia Sumarokova](https://github.com/kssenii)). -* 支持紧凑部分的自定义编解码器. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - -#### 性能改进 - -* 使用 LIMIT/LIMIT BY/ORDER BY 优化查询以使用 GROUP BY sharding_key (在 `optimize_skip_unused_shards` 和 `optimize_distributed_group_by_sharding_key` 下). [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)). -* 为多个 `JOIN` 和 `IN` 并行创建集合. 它可能会稍微提高具有几种不同 `IN subquery` 表达式的查询的性能. [#14412](https://github.com/ClickHouse/ClickHouse/pull/14412) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 通过为每个消费者提供独立线程来提高 Kafka 引擎性能. 流引擎的单独线程池 (如 Kafka). [#13939](https://github.com/ClickHouse/ClickHouse/pull/13939) ([fastio](https://github.com/fastio)). - -#### 构建/测试/打包改进 - -* 通过从 `Functions` 中删除调试信息来降低调试构建中的二进制大小. 只有 Yandex 中使用非常旧的链接器的一个内部项目才需要这样做. [#14549](https://github.com/ClickHouse/ClickHouse/pull/14549) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 准备使用 clang 11 进行构建. [#14455](https://github.com/ClickHouse/ClickHouse/pull/14455) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复反向移植脚本中的逻辑. 在以前的版本中, 任何 100% 红色的标签都会触发它. 这很奇怪. [#14433](https://github.com/ClickHouse/ClickHouse/pull/14433) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 集成测试使用默认的基本配置. 所有配置更改都是显式的, 例如 main_configs、user_configs 和字典参数. [#13647](https://github.com/ClickHouse/ClickHouse/pull/13647) ([Ilya Yatsishin](https://github.com/qoega)). - - - -## ClickHouse 版本 20.8 - -### ClickHouse 版本 v20.8.12.2-lts, 2021-01-16 - -#### Bug 修复 - -* 修复 *If 具有一元函数和 Nullable 类型的组合器. [#18806](https://github.com/ClickHouse/ClickHouse/pull/18806) ([Azat Khuzhin](https://github.com/azat)). -* 限制从宽部分合并到紧凑部分. 在垂直合并的情况下, 它会导致结果部分损坏. [#18381](https://github.com/ClickHouse/ClickHouse/pull/18381) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse 版本 v20.8.11.17-lts, 2020-12-25 - -#### Bug 修复 - -* 在合并期间禁用 AIO 写入, 因为它会导致合并期间主键列的数据损坏极为罕见. [#18481](https://github.com/ClickHouse/ClickHouse/pull/18481) ([alesapin](https://github.com/alesapin)). -* 修复了在使用类型为 Nullable(String) 的参数执行 `toType(...)` 函数 (`toDate`、`toUInt32` 等) 时出现的 `value is too short` 错误. 现在这些函数在解析错误时返回 `NULL` 而不是抛出异常. 修复 [#7673](https://github.com/ClickHouse/ClickHouse/issues/7673). [#18445](https://github.com/ClickHouse/ClickHouse/pull/18445) ([tavplubix](https://github.com/tavplubix)). -* 使用组合子 `Distinct` 修复聚合函数中可能的崩溃, 同时使用两级聚合. 修复 [#17682](https://github.com/ClickHouse/ClickHouse/issues/17682). [#18365](https://github.com/ClickHouse/ClickHouse/pull/18365) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse 版本 v20.8.10.13-lts, 2020-12-24 - -#### Bug 修复 - -* 当使用 `logger.size` 参数配置服务器日志轮换并且数值大于 2^32 时, 日志没有正确轮换. [#17905](https://github.com/ClickHouse/ClickHouse/pull/17905) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 使用 `min_compress_block_size` 修复了 MergeTreeWriterSettings 中 `max_compress_block_size` 的错误初始化. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). -* 修复了 ClickHouse 无法恢复与 MySQL 服务器的连接时的问题. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复了在不同副本上杀死相应突变时的“ALTER”查询挂起. 这修复了 [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* 修复了 ClickHouse 低估标记缓存大小时的错误. 当有很多带有标记的小文件时可能会发生. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* 修复了启用设置 `optimize_redundant_functions_in_order_by` 的 `ORDER BY` . [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了导致崩溃的 `ColumnConst` 比较. 这个修复了 [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088). [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* 修复了非领导者 ReplicatedMergeTreeTables 的`ON CLUSTER` 查询可能永远挂起的错误. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* 避免远程查询的不必要的网络错误, 这些错误在执行时可能会被取消, 例如使用 `LIMIT` 的查询. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* 出现错误时解析 `format_avro_schema_registry_url` 的IP. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* 修复了在 `ALTER TABLE ... MODIFY COLUMN ... NewType` 之后, 当 `SELECT` 在更改列上具有 `WHERE` 表达式并且更改尚未完成时可能出现的服务器崩溃. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* 安装脚本应始终在 config 文件夹中创建子目录. 这仅与使用自定义配置的 Docker 构建相关. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* 修复了使用 `ORDER BY` 的查询可能出现的错误 `Illegal type of argument` 。 修复 [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果没有数据写入 WriteBufferFromS3,则中止分段上传. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复了在不带任何参数的情况下使用 `any` 时的崩溃问题. 这修复了 [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* 修复了多个列和元组上的 `IN` 运算符, 启用了 `transform_null_in` 设置. 修复 [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `optimize_read_in_order/optimize_aggregation_in_order` 的不一致行为, max_threads > 0 和 ORDER BY 中的表达式. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* 修复了查询包含`ARRAY JOIN`时查询优化产生错误结果的问题. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* 在异常的情况下查询完成得更快. 如果发生异常, 取消对远程副本的执行. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse 版本 v20.8.6.6-lts, 2020-11-13 - -#### Bug 修复 - -* 当查询分析器打开并且 ClickHouse 安装在操作系统上时, 修复罕见的无声崩溃, glibc 版本已经 (据说) 损坏了某些功能的异步展开表. 这修复了 [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). 这修复了 [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在, 当从输入解析 AVRO 时, LowCardinality 将从类型中删除. 修复 [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* 通过适当缩小 GTID 集,修复使用 MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine 和 MySQL Slave 上启用的 `slave_parallel_worker` 时元数据的快速增长. 这修复了 [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* 修复分布式的 DROP TABLE (使用 INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* 修复了复制队列中非常大的条目的处理. 如果表结构非常大 (接近 1 MB), 则非常大的条目可能会出现在 ALTER 查询中. 这修复了 [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了由于未创建过滤集而可能删除部分返回数据时的不一致行为. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复 MySQL 数据库的错误. 当用作数据库引擎的 MySQL 服务器关闭时, 某些查询会引发异常, 因为它们试图从禁用的服务器获取表, 而这是不必要的. 例如, 查询 `SELECT ... FROM system.parts` 应该只适用于 MergeTree 表并且根本不要接触 MySQL 数据库. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). - - -### ClickHouse 版本 v20.8.5.45-lts, 2020-10-29 - -#### Bug 修复 - -* 在函数 `dictGet` 出现异常时修复double free. 如果字典加载错误, 可能会发生这种情况. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复由总数/汇总/多维数据集修饰符和最小/最大功能按键分组. 修复 [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). -* 修复异步分布式插入 w/prefer_localhost_replica=0 和 internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). -* 修复了在使用字符串键的 `GROUP BY` 期间可能出现的内存泄漏, 这是由 `TwoLevelStringHashTable` 实现中的错误引起的. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). -* 修复无论限制如何都可以过度分配内存的情况. 这将关闭 [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复` ReplicatedVersionedCollapsingMergeTree` 的 `ALTER MODIFY ... ORDER BY` 查询挂起。 这修复了 [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). -* 修复整理名称和字符集名称解析器并支持字符串类型的 `length = 0` . [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). -* 允许对具有复杂键的字典使用直接布局. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). -* 在一段时间不活动后发生复制错误时, 防止副本挂起 5-10 分钟. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). -* 在插入或从 MaterializedView 中选择并同时删除目标表时修复罕见的段错误 (适用于原子数据库引擎). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). -* 修复解析设置配置文件时的歧义: `CREATE USER ... SETTINGS profile readonly` 现在被视为使用名为 `readonly` 的配置文件, 而不是名为 `profile` 的具有只读约束的设置. 这修复了 [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复数据库创建失败时的崩溃. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了当表被并发重命名 (对于原子数据库引擎) 时, `DROP TABLE IF EXISTS` 失败并带有 `Table ... does not exist` 错误的问题. 修复了并发执行具有多个表的某些 DDL 查询 (如 `DROP DATABASE` 和 `RENAME TABLE` ) 时罕见的死锁, 修复了并发执行 `DROP/DETACH TABLE` 时 `Table ... does not exist` 的 `DROP/DETACH DATABASE` 失败 `. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). -* 如果查询具有`WHERE`、`PREWHERE` 和`GLOBAL IN`, 则修复来自`Distributed` 表的查询的错误空结果. 修复 [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 RBAC 中可能的死锁. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复在 `ALTER MODIFY COLUMN` 查询之后执行的 `SELECT ... ORDER BY DESC` 查询中的异常 `Block structure mismatch`. 修复 [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). -* 修复某些查询情况, 其中仅选择虚拟列. 以前可能会抛出 `Not found column _nothing in block` 异常. 修复 [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). -* 修复错误 `Cannot find column` , 如果对 `MV` 的查询包含 `ARRAY JOIN` , 则在插入 `MATERIALIZED VIEW` 时可能会发生该错误. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 `max_replicated_logs_to_keep` 设置的默认值过低, 这可能导致副本过于频繁地丢失. 通过选择要克隆的最新副本来改进丢失的副本恢复过程. 也不要从丢失的复制品中取出旧零件, 而是将它们拆下. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). -* 修复错误 `Cannot add simple transform to empty Pipe`, 该错误在从与目标表结构不同的 `Buffer` 表中读取时发生. 如果目标表返回空的查询结果是可能的. 修复 [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 S3 表函数中的 globs 错误, URL 中的区域未应用于 S3 客户端配置. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 分离只读表时递减 `ReadonlyReplica` 指标. 这修复了 [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). -* 将单个参数传递给 ReplicatedMergeTree 而不是忽略它时抛出错误. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). - -#### 改进 - -* 现在, 无论集群配置中的 `` 设置如何, 都可以执行 `ALTER ... ON CLUSTER` 查询. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* 在创建表时在 `ReplicatedMergeTree` 参数中展开 `{database}`、`{table}` 和 `{uuid}` 宏. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). - - -### ClickHouse 版本 v20.8.4.11-lts, 2020-10-09 - -#### Bug 修复 - -* 修复查询计划的 `ReadFromStorage` 步骤中资源的销毁顺序. 在极少数情况下, 它可能会导致崩溃. 可能与[#15610](https://github.com/ClickHouse/ClickHouse/issues/15610)有关. [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了使用 `JSON*` 函数时会导致 `VALUES`、 `LIMIT` 或 `IN` 运算符右侧的 `Element ... is not a constant expression` 错误. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). -* 防止出现错误消息 `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call` 的可能性. 这修复了 [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 显著减少 AggregatingInOrderTransform/optimize_aggregation_in_order 中的内存使用. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). -* Mutation 可能会在 `MOVE` 或 `REPLACE PARTITION` 之后等待一些不存在的部分, 或者在极少数情况下, 在 `DETACH` 或 `DROP PARTITION` 之后挂起. 已修复. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* 修复如果执行具有相同模式的 `ILIKE`, 则 `ILIKE` 运算符不再区分大小写的错误. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). -* 在选择数据中不存在但依赖于数据中也不存在的其他列时修复 `Missing columns` 错误. 修复 [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). -* 修复了 DDLWorker 中事件订阅的错误, 该错误很少会导致查询在“ON CLUSTER”中挂起. 引入 [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). -* 当 `boundingRatio` 聚合函数的第二个参数类型错误时报告正确的错误. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). -* 修复合并树表重命名和背景清理过程中的竞争条件. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). -* 修复启用 system.logs 时服务器启动时罕见的竞争条件. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). -* 修复 QueryLog 中的 MSan 报告. 未初始化的内存可用于字段 `memory_usage` . [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复将 joinGet 与 LowCardinality 类型一起使用时的实例崩溃. 这修复了 [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). -* 修复表引擎 `Buffer` 中的错误, 该错误不允许在 `ALTER` 查询后将新结构的数据插入到 `Buffer` 中. 修复 [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). -* 调整mysql列定义包中的decimals字段大小. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). -* 我们已经在 String 和 FixedString 之间使用了填充比较 (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333) . 此 PR 将相同的逻辑应用于字段比较, 以更正 FixedString 作为主键的使用. 这修复了 [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). -* 如果使用专门设计的参数调用函数 `bar`, 则可能会发生缓冲区溢出. 这将关闭 [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在 Mac OS 上的 docker 中运行 clickhouse-server 时, 在原子数据库中执行 DDL 查询时出现的 `Cannot rename ... errno: 22, strerror: Invalid argument` 错误. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). -* 现在设置 `number_of_free_entries_in_pool_to_execute_mutation` 和 `number_of_free_entries_in_pool_to_lower_max_size_of_merge` 可以等于 `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). -* 当子查询包含 finalizeAggregation 函数时, 修复使谓词下推工作. 修复 [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* 在 `system.asynchronous_metrics` 中发布每个逻辑核心的 CPU 频率. 这修复了 [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复了使用 `MaterializeMySQL` 数据库引擎时 `.metadata.tmp File exists` 的错误. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). -* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复了收缩的 `Int -> Int` 签名类型转换的错误单调性检测. 可能会导致查询结果不正确. 此错误在 [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513) 中公布. [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). -* 修复了 `Nullable` 列不正确的排序顺序. 这修复了 [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - -#### 改进 - -* 现在可以使用 `ALTER` 查询更改 `VersionedCollapsingMergeTree` 的版本列的类型. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse 版本 v20.8.3.18-stable, 2020-09-18 - -#### Bug 修复 - -* 修复部分调用 `extractAllGroups` 函数可能触发“超出内存限制”错误的问题. 这修复了 [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 SIGSEGV 以尝试插入 StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). -* 当被查询的列具有 `DEFAULT` 表达式时, 修复 `SELECT` 查询中的罕见错误, 该表达式依赖于另一列, 该列也具有“DEFAULT”并且不存在于选择查询中且不存在于磁盘上. 部分修复 [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* 修复了执行 `ALTER ... MODIFY QUERY` 时物化视图元数据中丢失的默认数据库名称. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). -* 修复了在赋值表达式和常量值 (如 `UPDATE x = 42`) 中使用 Nullable 列的 `ALTER UPDATE` 突变导致列或段错误中的值不正确时的错误. 修复 [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* 修复错误的十进制乘法结果导致结果列小数位数错误. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). -* 添加了检查器, 因为既不调用 `lc->isNullable()` 也不调用 `ls->getDictionaryPtr()->isNullable()` 将返回正确的结果. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([myrrc](https://github.com/myrrc)). -* 在为 StorageReplicatedMergeTree 引擎创建查询期间 Zookeeper 异常后清理数据目录. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). -* 使用组合器 -Resample 修复函数中罕见的段错误, 这可能会出现在参数非常大的溢出结果中. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). - -#### 改进 - -* 如果有正在进行的 S3 请求, 则加快服务器关闭过程. [#14858](https://github.com/ClickHouse/ClickHouse/pull/14858) ([Pavel Kovalenko](https://github.com/Jokser)). -* 允许在分布式存储中使用多卷存储配置. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). -* 如果有正在进行的 S3 请求, 则加快服务器关闭过程. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). -* 支持紧凑部分的自定义编解码器. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse 版本 v20.8.2.3-stable, 2020-09-08 - -#### 向后不兼容变更 - -* 现在 `OPTIMIZE FINAL` 查询不会重新计算在创建 TTL 之前添加的部件的 TTL. 使用 `ALTER TABLE ... MATERIALIZE TTL` 一次来计算它们, 之后 `OPTIMIZE FINAL` 将正确评估TTL. 此行为从未适用于复制表. [#14220](https://github.com/ClickHouse/ClickHouse/pull/14220) ([alesapin](https://github.com/alesapin)). -* 扩展 `parallel_distributed_insert_select` 设置, 添加一个选项以将 `INSERT` 运行到本地表中. 该设置将类型从 `Bool` 更改为 `UInt64`, 因此不再支持值 `false` 和 `true`. 如果服务器配置中有这些值, 服务器将不会启动. 请将它们分别替换为 `0` 和 `1`. [#14060](https://github.com/ClickHouse/ClickHouse/pull/14060) ([Azat Khuzhin](https://github.com/azat)). -* 移除对 `ODBCDriver` 输入/输出格式的支持. 这是曾经用于与 ClickHouse ODBC 驱动程序通信的弃用格式, 现在早已被 `ODBCDriver2` 格式取代. 解决 [#13629](https://github.com/ClickHouse/ClickHouse/issues/13629). [#13847](https://github.com/ClickHouse/ClickHouse/pull/13847) ([hexiaoting](https://github.com/hexiaoting)). -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重启, 旧版本在新版本存在的情况下启动, 可能会导致 `Part ... intersects previous part` 错误. 为防止出现此错误, 首先在所有集群节点上安装更新的 clickhouse-server 软件包, 然后重新启动 (因此,当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* 添加为与 `config.xml` 中指定的设置相对应的列指定 `Default` 压缩编解码器的功能. 实现: [#9074](https://github.com/ClickHouse/ClickHouse/issues/9074). [#14049](https://github.com/ClickHouse/ClickHouse/pull/14049) ([alesapin](https://github.com/alesapin)). -* 支持 Kafka 中的 Kerberos 身份验证,使用 `krb5` 和 `cyrus-sasl` 库. [#12771](https://github.com/ClickHouse/ClickHouse/pull/12771) ([Ilya Golshtein](https://github.com/ilejn)). -* 添加函数`normalizeQuery`, 用占位符替换文字、文字序列和复杂别名. 添加函数 `normalizedQueryHash` , 为类似的查询返回相同的 64 位哈希值. 它有助于分析查询日志. 这将关闭 [#11271](https://github.com/ClickHouse/ClickHouse/issues/11271). [#13816](https://github.com/ClickHouse/ClickHouse/pull/13816) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加 `time_zones` 表. [#13880](https://github.com/ClickHouse/ClickHouse/pull/13880) ([Bharat Nallan](https://github.com/bharatnc)). -* 添加函数 `defaultValueOfTypeName`, 返回给定类型的默认值. [#13877](https://github.com/ClickHouse/ClickHouse/pull/13877) ([hcz](https://github.com/hczhcz)). -* 添加 `countDigits(x)` 函数, 用于计算整数或小数列中的小数位数. 添加 `isDecimalOverflow(d, [p])` 函数, 用于检查 Decimal 列中的值是否超出其 (或指定的) 精度. [#14151](https://github.com/ClickHouse/ClickHouse/pull/14151) ([Artem Zuikov](https://github.com/4ertus2)). -* 添加 `quantileExactLow` 和 `quantileExactHigh` 实现, 并分别具有 `medianExactLow` 和 `medianExactHigh` 的别名. [#13818](https://github.com/ClickHouse/ClickHouse/pull/13818) ([Bharat Nallan](https://github.com/bharatnc)). -* 添加了将日期/时间值截断为指定日期/时间部分的 `date_trunc` 函数. [#13888](https://github.com/ClickHouse/ClickHouse/pull/13888) ([Vladimir Golovchenko](https://github.com/vladimir-golovchenko)). -* 将新的可选部分 `` 添加到主配置. [#13425](https://github.com/ClickHouse/ClickHouse/pull/13425) ([Vitaly Baranov](https://github.com/vitlibar)). -* 添加允许更改表示例子句的 `ALTER SAMPLE BY` 语句. [#13280](https://github.com/ClickHouse/ClickHouse/pull/13280) ([Amos Bird](https://github.com/amosbird)). -* 函数 `position` 现在支持可选的 `start_pos` 参数. [#13237](https://github.com/ClickHouse/ClickHouse/pull/13237) ([vdimir](https://github.com/vdimir)). - -#### Bug 修复 - -* 修复客户端交互模式下进度条可见数据乱码问题. 这修复了 [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) 和 [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) 和 [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) 并修复了 [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了按多列排序时如果 `LowCardinality` 列不正确的排序顺序. 这修复了 [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 检查 `topK` 聚合函数中的数组大小溢出. 如果没有此检查, 用户可能会发送带有精心设计的参数的查询, 这将导致服务器崩溃. 这将关闭 [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了如果表具有单个部分的分区可能导致错误合并分配的错误. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). -* 如果 PipelineExecutor 本身发生异常, 则停止查询执行. 这可以防止罕见的查询挂起. 续 [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复在创建为 `AS table_function` 的表的 `ALTER` 查询期间崩溃. 修复 [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). -* 使用 REFRESH 命令修复 ALTER LIVE VIEW 查询期间的异常. 实时取景是一项实验性功能. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). -* 修复带有嵌套解释器的查询的 QueryPlan 生命周期 (对于 EXPLAIN PIPELINE graph=1). [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). -* 在从某些外部来源获取架构期间修复 `lickhouse-odbc-bridge` 的段错误. PR 修复了. [#14267](https://github.com/ClickHouse/ClickHouse/pull/14267) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复 [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277) 中引入的标记包含搜索崩溃. [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). -* 使用命名元组修复表的创建. 这修复了 [#13027](https://github.com/ClickHouse/ClickHouse/issues/13027). [#14143](https://github.com/ClickHouse/ClickHouse/pull/14143) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复最小负十进制数的格式. 这修复了 [#14111](https://github.com/ClickHouse/ClickHouse/issues/14111). [#14119](https://github.com/ClickHouse/ClickHouse/pull/14119) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复 `DistributedFilesToInsert` 指标 (不应该归零). [#14095](https://github.com/ClickHouse/ClickHouse/pull/14095) ([Azat Khuzhin](https://github.com/azat)). -* 使用 const 2d 数组将 `pointInPolygon` 修复为多边形. [#14079](https://github.com/ClickHouse/ClickHouse/pull/14079) ([Alexey Ilyukhov](https://github.com/livace)). -* 修复了 `Poco::Exception: no space left on device` 额外信息中错误的挂载点. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). -* 在非全局级别执行时修复 GRANT ALL 语句. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复解析器以拒绝使用引擎创建表作为表函数. [#13940](https://github.com/ClickHouse/ClickHouse/pull/13940) ([hcz](https://github.com/hczhcz)). -* 如果启用了 `optimize_duplicate_order_by_and_distinct` 设置, 则使用 `DISTINCT` 关键字修复选择查询和使用 UNION ALL 的子查询中的错误结果. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了重命名 `Distributed` 表时潜在的死锁. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). -* 修复按多列排序时 `FixedString` 列的不正确排序. 修复 [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `topK`/`topKWeighted` 合并的潜在不精确结果 (使用非默认参数). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). -* 修复与 NULL 比较时, 从 MergeTree 表中读取 INDEX 类型为 SET 失败的问题. 这修复了[#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). -* 修复 lambda 中的 `arrayJoin` 捕获 (LOGICAL_ERROR). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). -* 在函数 `range` 中添加步骤溢出检查. [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). -* 修复了并发执行 `DROP DATABASE` 和 `CREATE TABLE` 时的 `Directory not empty` 错误. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 `h3KRing` 函数添加范围检查. 这修复了[#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 DETACH 和后台合并之间的竞争条件. 零件在分离后可能会复活. 这是 [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) 的延续, 它没有解决问题, 但引入了一个在极少数情况下开始失败的测试, 证明了这个问题. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当 log_queries_min_type > QUERY_START 时修复日志记录 Settings.Names/Values. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). -* 当verbose=1时修复 `/replicas_status` 端点响应状态代码. [#13722](https://github.com/ClickHouse/ClickHouse/pull/13722) ([javi santana](https://github.com/javisantana)). -* 在检查用户和组时修复 `clickhouse-server.init` 中的错误消息. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). -* 不要在 `optimize_move_functions_out_of_any` 设置下优化 any(arrayJoin()) -> arrayJoin(). [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). -* 使用 StorageMerge 和 `set enable_optimize_predicate_expression=1` 修复 JOIN 中的崩溃. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复关于 `number_of_free_entries_in_pool_to_lower_max_size_of_merge` 设置的值的错误消息中的错字. [#13678](https://github.com/ClickHouse/ClickHouse/pull/13678) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 并发 `ALTER ... REPLACE/MOVE PARTITION ...` 查询可能会导致死锁. 已修复. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). -* 修复了有时缓存字典从源返回默认值而不是当前值时的行为. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复紧凑部分中的二级索引损坏. 紧凑的部件是实验功能. [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). -* 修复必须在单个副本上执行的查询的过早 `ON CLUSTER` 超时. 修复 [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). -* 修复函数 `netloc` 中的错误代码. 这修复了[#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `StorageMemory` 中可能出现的竞争. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 HTTP 协议中 `TSV/CSVWithNames` 格式中缺失或过多的标头. 这修复了 [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). -* 当数据库或表的名称包含点时, 修复从 users.xml 解析行策略. 这修复了[#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). -* 在连接断开一次后修复对 `redis` 字典的访问. `cache` 和 `direct` 字典布局可能会发生这种情况. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). -* 删除了使用 ClickHouseDictionarySource 查询远程表时错误的身份验证访问检查. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). -* 正确区分某些情况下的子查询, 用于公共子表达式的消除. [#8333](https://github.com/ClickHouse/ClickHouse/issues/8333). [#8367](https://github.com/ClickHouse/ClickHouse/pull/8367) ([Amos Bird](https://github.com/amosbird)). - -#### 改进 - -* 在 `ALIAS` 列类型上不允许使用 `CODEC`. 修复 [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). -* 等待字典更新完成时, 使用由 `query_wait_timeout_milliseconds` 设置指定的超时而不是硬编码值. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 添加设置 `min_index_granularity_bytes` 以防止意外创建具有非常低的 `index_granularity_bytes` 设置的表. [#14139](https://github.com/ClickHouse/ClickHouse/pull/14139) ([Bharat Nallan](https://github.com/bharatnc)). -* 现在可以从使用不同 ZooKeeper 的集群中获取分区: `ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'zk-name:/path-in-zookeeper'`. 这对于将数据传送到新集群很有用. [#14155](https://github.com/ClickHouse/ClickHouse/pull/14155) ([Amos Bird](https://github.com/amosbird)). -* 如果 Memory 表是由大量非常小的块构建的, 那么它的性能会稍好一些 (这不太可能). 这个想法的作者: [Mark Papadakis](https://github.com/markpapadakis). 关闭 [#14043](https://github.com/ClickHouse/ClickHouse/issues/14043). [#14056](https://github.com/ClickHouse/ClickHouse/pull/14056) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 条件聚合函数 (例如: `avgIf`、`sumIf`、`maxIf`) 在缺少行并使用可为空参数时应返回 `NULL` . [#13964](https://github.com/ClickHouse/ClickHouse/pull/13964) ([Winter Zhang](https://github.com/zhang2014)). -* 将 -Resample 组合器中的限制增加到 1M. [#13947](https://github.com/ClickHouse/ClickHouse/pull/13947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* 更正了 AvroConfluent 格式的错误, 该错误导致 Kafka 表引擎在收到异常小的、格式错误的消息时停止处理消息. [#13941](https://github.com/ClickHouse/ClickHouse/pull/13941) ([Gervasio Varela](https://github.com/gervarela)). -* 修复长查询的错误错误. 为了正确查询, 可能会出现除 `Max query size exceeded` 之外的语法错误. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* `TabSeparated` 格式的空值的更好的错误消息. [#13906](https://github.com/ClickHouse/ClickHouse/pull/13906) ([jiang tao](https://github.com/tomjiang1987)). -* 如果数组元素的类型是 Float32/Float64, 函数 `arrayCompact` 将按位比较 NaN. 在以前的版本中, 如果数组元素的类型是 Float32/Float64,NaN 总是不相等, 如果类型更复杂, 例如 Nullable(Float64), NaN 总是相等. 这将关闭 [#13857](https://github.com/ClickHouse/ClickHouse/issues/13857). [#13868](https://github.com/ClickHouse/ClickHouse/pull/13868) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `lgamma` 函数中的数据竞争. 这场比赛只在 `tsan` 中被抓到, 没有真正发生过副作用. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当数组作为字段操作时, 避免太慢的查询. 改为抛出异常. [#13753](https://github.com/ClickHouse/ClickHouse/pull/13753) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 新增Redis requirepass授权 (针对redis字典源码). [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). -* 添加 MergeTree Write-Ahead-Log (WAL) 转储工具. WAL 是一个实验性功能. [#13640](https://github.com/ClickHouse/ClickHouse/pull/13640) ([BohuTANG](https://github.com/BohuTANG)). -* 在以前的版本中, 如果使用专门设计的参数调用, `lcm` 函数可能会在调试构建中产生断言冲突. 这修复了 [#13368](https://github.com/ClickHouse/ClickHouse/issues/13368). [#13510](https://github.com/ClickHouse/ClickHouse/pull/13510) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在更多情况下为 `toDate/toDateTime` 函数提供单调性. 单调性信息用于索引分析 (更复杂的查询将能够使用索引). 现在输入参数更自然地饱和并提供更好的单调性. [#13497](https://github.com/ClickHouse/ClickHouse/pull/13497) ([Amos Bird](https://github.com/amosbird)). -* 支持自定义设置的复合标识符. 自定义设置是 ClickHouse 代码库与其他代码库的集成点 (对 ClickHouse 本身没有好处) [#13496](https://github.com/ClickHouse/ClickHouse/pull/13496) ([Vitaly Baranov](https://github.com/vitlibar)). -* 将部分从 DiskLocal 并行移动到 DiskS3. `DiskS3` 是一个实验性功能. [#13459](https://github.com/ClickHouse/ClickHouse/pull/13459) ([Pavel Kovalenko](https://github.com/Jokser)). -* 默认启用混合粒度部分. [#13449](https://github.com/ClickHouse/ClickHouse/pull/13449) ([alesapin](https://github.com/alesapin)). -* S3 重定向中正确的远程主机检查 (与安全相关的事情). [#13404](https://github.com/ClickHouse/ClickHouse/pull/13404) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 将 `QueryTimeMicroseconds`、`SelectQueryTimeMicroseconds` 和 `InsertQueryTimeMicroseconds` 添加到 system.events. [#13336](https://github.com/ClickHouse/ClickHouse/pull/13336) ([ianton-ru](https://github.com/ianton-ru)). -* 修复 Decimal 具有太大负指数时的调试断言. 修复 [#13188](https://github.com/ClickHouse/ClickHouse/issues/13188). [#13228](https://github.com/ClickHouse/ClickHouse/pull/13228) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 DiskS3 添加了缓存层 (缓存到本地磁盘标记和索引文件). `DiskS3` 是一个实验性功能. [#13076](https://github.com/ClickHouse/ClickHouse/pull/13076) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复 readline 以便现在将历史记录转储到文件中. [#13600](https://github.com/ClickHouse/ClickHouse/pull/13600) ([Amos Bird](https://github.com/amosbird)). -* 默认使用 `Atomic` 引擎创建 `system` 数据库 (准备在任何地方默认启用 `Atomic` 数据库引擎). [#13680](https://github.com/ClickHouse/ClickHouse/pull/13680) ([tavplubix](https://github.com/tavplubix)). - -#### 性能改进 - -* 使用 `LowCardinality` 稍微优化非常短的查询. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). -* 当设置 `max_insert_threads` 时, 为表引擎 `Null`、`Memory`、`Distributed` 和 `Buffer` 启用并行插入. [#14120](https://github.com/ClickHouse/ClickHouse/pull/14120) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果在部件扫描时超过 `max_rows_to_read` 限制, 则快速失败. 此更改背后的动机是, 如果明显已超过 `max_rows_to_read`, 则跳过所有选定部分的范围扫描. 对于大量零件的查询, 这种变化非常明显. [#13677](https://github.com/ClickHouse/ClickHouse/pull/13677) ([Roman Khavronenko](https://github.com/hagen1778)). -* 通过 UInt8/UInt16 键略微提高聚合性能. [#13099](https://github.com/ClickHouse/ClickHouse/pull/13099) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 优化`has()`、`indexOf()`和 `countEqual()` 函数, 用于 `Array(LowCardinality(T))` 和常量右参数. [#12550](https://github.com/ClickHouse/ClickHouse/pull/12550) ([myrrc](https://github.com/myrrc)). -* 在执行简单的 `INSERT SELECT` 查询时, 自动将 `max_threads` 设置为1 或 `max_insert_threads`, 并将 `max_block_size` 设置为 `min_insert_block_size_rows`. 相关 [#5907](https://github.com/ClickHouse/ClickHouse/issues/5907). [#12195](https://github.com/ClickHouse/ClickHouse/pull/12195) ([flynn](https://github.com/ucasFL)). - -#### 实验功能 - -* ClickHouse 可以用作 MySQL 副本 - 它由 `MaterializeMySQL` 数据库引擎实现. 实现 [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). -* 为它们添加类型 `Int128`、`Int256`、`UInt256` 和相关函数. 使用 Decimal256 扩展小数 (精度高达 76 位). 新类型在设置 `allow_experimental_bigint_types` 下. 它的工作非常缓慢和糟糕. 实施不完整. 请不要使用此功能. [#13097](https://github.com/ClickHouse/ClickHouse/pull/13097) ([Artem Zuikov](https://github.com/4ertus2)). - -#### 构建/测试/打包改进 - -* 添加了 `clickhouse install` 脚本, 如果你只有一个二进制文件, 这很有用. [#13528](https://github.com/ClickHouse/ClickHouse/pull/13528) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许在没有配置的情况下运行 `clickhouse` 二进制文件. [#13515](https://github.com/ClickHouse/ClickHouse/pull/13515) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用 `codespell` 启用检查代码中的拼写错误. [#13513](https://github.com/ClickHouse/ClickHouse/pull/13513) [#13511](https://github.com/ClickHouse/ClickHouse/pull/13511) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 CI 中启用 Shellcheck 作为 .sh 测试的 linter. 这将关闭 [#13168](https://github.com/ClickHouse/ClickHouse/issues/13168). [#13530](https://github.com/ClickHouse/ClickHouse/pull/13530) [#13529](https://github.com/ClickHouse/ClickHouse/pull/13529) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加 CMake 选项以失败配置而不是自动重新配置, 默认启用. [#13687](https://github.com/ClickHouse/ClickHouse/pull/13687) ([Konstantin](https://github.com/podshumok)). -* 通过 system.build_options 中的 TZDATA_VERSION 公开嵌入式 tzdata 的版本. [#13648](https://github.com/ClickHouse/ClickHouse/pull/13648) ([filimonov](https://github.com/filimonov)). -* 在构建期间改进 system.time_zones 表的生成. 关闭 [#14209](https://github.com/ClickHouse/ClickHouse/issues/14209). [#14215](https://github.com/ClickHouse/ClickHouse/pull/14215) ([filimonov](https://github.com/filimonov)). -* 使用来自包存储库的最新 tzdata 构建 ClickHouse. [#13623](https://github.com/ClickHouse/ClickHouse/pull/13623) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在skip_list.json中添加编写js样式注释的能力. [#14159](https://github.com/ClickHouse/ClickHouse/pull/14159) ([alesapin](https://github.com/alesapin)). -* 确保没有复制粘贴的 GPL 代码. [#13514](https://github.com/ClickHouse/ClickHouse/pull/13514) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 切换测试 docker 图像以使用基于测试的父级. [#14167](https://github.com/ClickHouse/ClickHouse/pull/14167) ([Ilya Yatsishin](https://github.com/qoega)). -* 启动docker-compose集群时添加重试逻辑; 增加 COMPOSE_HTTP_TIMEOUT. [#14112](https://github.com/ClickHouse/ClickHouse/pull/14112) ([vzakaznikov](https://github.com/vzakaznikov)). -* 在压力测试中启用 `system.text_log` 以发现更多错误. [#13855](https://github.com/ClickHouse/ClickHouse/pull/13855) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Testflows LDAP 模块: 为 openldap4 添加缺少的证书和 dhparam.pem. [#13780](https://github.com/ClickHouse/ClickHouse/pull/13780) ([vzakaznikov](https://github.com/vzakaznikov)). -* ZooKeeper 在 CI 基础设施的单元测试中无法可靠地工作. 使用单元测试进行 ZooKeeper 与真正的 ZooKeeper 交互从一开始就是个坏主意 (单元测试不应该验证复杂的分布式系统). 我们已经为此目的使用集成测试, 它们更适合. [#13745](https://github.com/ClickHouse/ClickHouse/pull/13745) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了用于样式检查的 docker 图像. 添加样式检查所有 docker 和 docker compose 文件都位于 docker 目录中. [#13724](https://github.com/ClickHouse/ClickHouse/pull/13724) ([Ilya Yatsishin](https://github.com/qoega)). -* 修复 Mac OS 上的 cassandra 构建. [#13708](https://github.com/ClickHouse/ClickHouse/pull/13708) ([Ilya Yatsishin](https://github.com/qoega)). -* 修复共享构建中的链接错误. [#13700](https://github.com/ClickHouse/ClickHouse/pull/13700) ([Amos Bird](https://github.com/amosbird)). -* 更新 LDAP 用户身份验证套件以检查它是否适用于 RBAC. [#13656](https://github.com/ClickHouse/ClickHouse/pull/13656) ([vzakaznikov](https://github.com/vzakaznikov)). -* 删除了 `contrib/aws` 的 `-DENABLE_CURL_CLIENT`. [#13628](https://github.com/ClickHouse/ClickHouse/pull/13628) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 增加 ClickHouse 节点的运行状况检查超时, 并在发现不健康的容器时添加对转储 docker-compose 日志的支持. [#13612](https://github.com/ClickHouse/ClickHouse/pull/13612) ([vzakaznikov](https://github.com/vzakaznikov)). -* 确保 [#10977](https://github.com/ClickHouse/ClickHouse/issues/10977) 无效. [#13539](https://github.com/ClickHouse/ClickHouse/pull/13539) ([Amos Bird](https://github.com/amosbird)). -* 从robot-clickhouse 跳过PR. [#13489](https://github.com/ClickHouse/ClickHouse/pull/13489) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 将 Dockerfiles 从集成测试移动到 `docker/test` 目录. docker_compose 文件在 `runner` docker 容器中可用. Docker 镜像是在 CI 中构建的, 而不是在集成测试中. [#13448](https://github.com/ClickHouse/ClickHouse/pull/13448) ([Ilya Yatsishin](https://github.com/qoega)). - - -## ClickHouse 版本 20.7 - -### ClickHouse 版本 v20.7.2.30-stable, 2020-08-31 - -#### 向后不兼容变更 - -* 以至少一个浮点数作为参数的函数 `modulo` (运算符 `%` )将直接计算浮点数的除法余数, 而不将两个参数都转换为整数. 它使行为与大多数 DBMS 兼容. 这也适用于 Date 和 DateTime 数据类型. 添加别名 `mod` . 这将关闭 [#7323](https://github.com/ClickHouse/ClickHouse/issues/7323). [#12585](https://github.com/ClickHouse/ClickHouse/pull/12585) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 弃用零日期/日期时间值的特殊打印为 `0000-00-00` 和`0000-00-00 00:00:00`. [#12442](https://github.com/ClickHouse/ClickHouse/pull/12442) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 函数 `groupArrayMoving*` 不适用于分布式查询. 它的结果是在不正确的数据类型中计算的 (没有升级到最大的类型). `groupArrayMovingAvg` 函数返回的整数与 `avg` 函数不一致. 这修复了[#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 MergeTree 设置添加健全性检查. 如果设置不正确,服务器将拒绝启动或创建表, 并向用户打印详细说明. [#13153](https://github.com/ClickHouse/ClickHouse/pull/13153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 防止用户可能将 `background_pool_size` 设置为低于 `number_of_free_entries_in_pool_to_execute_mutation` 或 `number_of_free_entries_in_pool_to_lower_max_size_of_merge` 的值. 在这些情况下, ALTER 将不起作用或合并的最大大小将太有限. 它将抛出异常解释要做什么. 这将关闭 [#10897](https://github.com/ClickHouse/ClickHouse/issues/10897). [#12728](https://github.com/ClickHouse/ClickHouse/pull/12728) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重启, 旧版本在新版本存在的情况下启动, 可能会导致 `Part ... intersects previous part` 错误. 为防止出现此错误, 首先在所有集群节点上安装更新的 clickhouse-server 软件包, 然后重新启动 (因此, 当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* 提供高效 `reverse geocoding` 查找的多边形字典类型 - 在许多多边形的字典 (世界地图) 中通过坐标查找区域. 它使用精心优化的算法和递归网格来保持较低的 CPU 和内存使用率. [#9278](https://github.com/ClickHouse/ClickHouse/pull/9278) ([achulkov2](https://github.com/achulkov2)). -* 添加了对预配置用户的 LDAP 身份验证支持 ( `Simple Bind` 方法). [#11234](https://github.com/ClickHouse/ClickHouse/pull/11234) ([Denis Glazachev](https://github.com/traceon)). -* 引入设置 `alter_partition_verbose_result`, 它为某些类型的 `ALTER TABLE ... PARTITION ...` 查询 (当前是`ATTACH` 和`FREEZE`)输出有关触摸部分的信息. 关闭 [#8076](https://github.com/ClickHouse/ClickHouse/issues/8076). [#13017](https://github.com/ClickHouse/ClickHouse/pull/13017) ([alesapin](https://github.com/alesapin)). -* 为 bayesian-ab-testing 添加 `bayesAB` 函数. [#12327](https://github.com/ClickHouse/ClickHouse/pull/12327) ([achimbab](https://github.com/achimbab)). -* 添加了 `system.crash_log` 表, 用于收集致命错误的堆栈跟踪. 这个表应该是空的. [#12316](https://github.com/ClickHouse/ClickHouse/pull/12316) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了 http 标头 `X-ClickHouse-Database` 和 `X-ClickHouse-Format`, 可用于设置默认数据库和输出格式. [#12981](https://github.com/ClickHouse/ClickHouse/pull/12981) ([hcz](https://github.com/hczhcz)). -* 将 `minMap` 和 `maxMap` 函数支持添加到 `SimpleAggregateFunction`. [#12662](https://github.com/ClickHouse/ClickHouse/pull/12662) ([Ildus Kurbangaliev](https://github.com/ildus)). -* 添加设置 `allow_non_metadata_alters`, 它限制执行修改磁盘上数据的 `ALTER` 查询. 默认禁用. 关闭 [#11547](https://github.com/ClickHouse/ClickHouse/issues/11547). [#12635](https://github.com/ClickHouse/ClickHouse/pull/12635) ([alesapin](https://github.com/alesapin)). -* 添加了一个函数 `formatRow` 以支持通过给定格式将任意表达式转换为字符串. 它对于操作 SQL 输出很有用, 并且与 `columns` 函数结合使用非常通用. [#12574](https://github.com/ClickHouse/ClickHouse/pull/12574) ([Amos Bird](https://github.com/amosbird)). -* 添加 `FROM_UNIXTIME` 函数以兼容 MySQL, 与 [12149](https://github.com/ClickHouse/ClickHouse/issues/12149) 相关. [#12484](https://github.com/ClickHouse/ClickHouse/pull/12484) ([flynn](https://github.com/ucasFL)). -* 如果启用了 `allow_nullable_key` 表设置, 则允许 Nullable 类型作为 MergeTree 表中的键. 关闭 [#5319](https://github.com/ClickHouse/ClickHouse/issues/5319). [#12433](https://github.com/ClickHouse/ClickHouse/pull/12433) ([Amos Bird](https://github.com/amosbird)). -* 与 [COS](https://intl.cloud.tencent.com/product/cos) 集成. [#12386](https://github.com/ClickHouse/ClickHouse/pull/12386) ([fastio](https://github.com/fastio)). -* 添加 `mapAdd` 和 `mapSubtract` 函数以添加/减去键映射值. [#11735](https://github.com/ClickHouse/ClickHouse/pull/11735) ([Ildus Kurbangaliev](https://github.com/ildus)). - -#### Bug 修复 - -* 修复必须在单个副本上执行的查询的过早 `ON CLUSTER` 超时. 修复 [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). -* 修复 [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277) 中引入的标记包含搜索崩溃. [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). -* 使用缓存布局修复外部字典中的竞争条件, 这可能导致服务器崩溃. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* 修复客户端交互模式下进度条可见数据乱码问题. 这修复了 [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) 和 [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) 和 [#13584] (https://github.com/ClickHouse/ClickHouse/issues/13584) 并修复了 [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了当使用 ORDER BY 多列时 `LowCardinality` 列的不正确排序顺序. 这修复了 [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 删除了硬编码超时, 它错误地否决了缓存字典的 `query_wait_timeout_milliseconds` 设置. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了 `Poco::Exception: no space left on device` 额外信息中错误的挂载点. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). -* 在启用了 `optimize_duplicate_order_by_and_distinct` 设置的情况下, 当子查询也具有 `DISTINCT` 时, 使用 `DISTINCT` 关键字修复选择查询的错误查询优化. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了重命名 `Distributed` 表时潜在的死锁. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). -* 修复使用 ORDER BY 多列时 `FixedString` 列的不正确排序。 修复 [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `topK`/`topKWeighted` 聚合的潜在较低精度 (使用非默认参数). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). -* 修复从 MergeTree 表中读取 INDEX 类型为 SET 时与 NULL 相比失败的问题. 这修复了 [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). -* 修复函数`range()`中的步骤溢出. [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). -* 修复了并发执行 `DROP DATABASE` 和 `CREATE TABLE` 时的 `Directory not empty` 错误. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 `h3KRing` 函数添加范围检查. 这修复了 [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 DETACH 和后台合并之间的竞争条件. 零件在分离后可能会复活. 这是 [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) 的延续, 它没有解决问题, 但引入了一个在极少数情况下开始失败的测试, 证明了这个问题. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当 `log_queries_min_type` 大于 `QUERY_START` 时修复日志 Settings.Names/Values . [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). -* 在检查用户和组时修复 `clickhouse-server.init` 中的错误消息. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). -* 不要在 `optimize_move_functions_out_of_any` 下将 `any(arrayJoin())` 优化为 `arrayJoin()` . [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). -* 修复了并发 `ALTER ... REPLACE/MOVE PARTITION ...` 查询中可能出现的死锁. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). -* 修复了有时缓存字典从源返回默认值而不是当前值时的行为. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复紧凑部分中的二级索引损坏 (紧凑部分是一个实验性功能). [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). -* 修复函数 `netloc` 中的错误代码. 这修复了 [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当 unix 时间戳作为参数传递时, 修复 `parseDateTimeBestEffort` 函数中的错误. 这修复了 [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复无效的返回类型, 用于将元组与 `NULL` 元素进行比较. 修复 [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复错误优化导致 `aggregate function any(x) is found inside another aggregate function in query` 错误 `SET optimize_move_functions_out_of_any = 1` 和 `any()` 内的别名. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复 `storagmemory` 中可能存在的竞争问题. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果查询返回零行, 修复`Arrow` 和`Parquet` 格式的空输出. 这样做是因为空输出对于这种格式无效. [#13399](https://github.com/ClickHouse/ClickHouse/pull/13399) ([hcz](https://github.com/hczhcz)). -* 在 `ORDER BY` 子句中使用常量列和主键前缀修复选择查询. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 clickhouse-local 的 `PrettyCompactMonoBlock`. 使用 `PrettyCompactMonoBlock` 修复极端/总数. 修复 [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 system.text_log 中的死锁. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). 它是 [#12339](https://github.com/ClickHouse/ClickHouse/issues/12339) 的一部分. 这修复了 [#12325]. [#13386](https://github.com/ClickHouse/ClickHouse/pull/13386) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了`File(TSVWithNames*)` (标题被多次写入), 修复了 `clickhouse-local --format CSVWithNames*` (缺少标题, 在[#12197](https://github.com/ClickHouse/ClickHouse/ 问题/12197)), 修复了零行 (缺少标题) 的 `clickhouse-local --format CSVWithNames*`. [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). -* 当函数 `groupArrayMovingSum` 反序列化空状态时, 修复段错误. 修复 [#13339](https://github.com/ClickHouse/ClickHouse/issues/13339). [#13341](https://github.com/ClickHouse/ClickHouse/pull/13341) ([alesapin](https://github.com/alesapin)). -* 在 `JOIN ON` 部分中的 `arrayJoin()` 函数上抛出错误. [#13330](https://github.com/ClickHouse/ClickHouse/pull/13330) ([Artem Zuikov](https://github.com/4ertus2)). -* 使用 `join_use_nulls=1` 修复 `LEFT ASOF JOIN` 中的崩溃. [#13291](https://github.com/ClickHouse/ClickHouse/pull/13291) ([Artem Zuikov](https://github.com/4ertus2)). -* 在从延迟副本查询的情况下, 修复可能的错误 `Totals having transform was already added to pipeline` . [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果用户将特制的参数传递给函数 `h3ToChildren`, 服务器可能会崩溃. 这修复了 [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在具有 NaN 值的 Float 类型上调用的 `uniqExact`、`topK`、`sumDistinct` 和类似聚合函数的潜在低性能和稍微不正确的结果. 它还在调试版本中触发断言. 这修复了 [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当主键包含具有单调函数的表达式并且查询包含与类型不同的常量的比较时, 修复 KeyCondition 中的断言. 这修复了[#12465](https://github.com/ClickHouse/ClickHouse/issues/12465). [#13251](https://github.com/ClickHouse/ClickHouse/pull/13251) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 返回在函数 roundUpToPowerOfTwoOrZero() 中设置了 MSB 的数字的传递数字. 它可以防止数组大小溢出时的潜在错误. [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). -* 如果使用可为空的 constexpr 作为不是文字 NULL 的 cond, 则修复函数. 修复 [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在数组元素为 Nullable 且数组下标也可为 Null 的情况下, 修复 `arrayElement` 函数中的断言. 这修复了 [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用常量参数修复 DateTime64 转换函数. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). -* 当数据库或表的名称包含点时, 修复从 users.xml 解析行策略. 这修复了 [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). -* 在连接断开一次后修复对 `redis` 字典的访问. `cache` 和 `direct` 字典布局可能会发生这种情况. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). -* 使用函数修复错误的索引分析. 从 `MergeTree` 表中读取时, 可能会导致跳过某些数据部分. 修复 [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* 修复错误 `Cannot convert column because it is constant but values of constants are different in source and result` , 远程查询在查询范围内使用确定性函数, 但查询之间不具有确定性, 例如`now()`、`now64()`、`randConstant()`. 修复 [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了使用 `ORDER BY` 元组和小 `LIMIT` 查询时可能发生的崩溃. 修复 [#12623](https://github.com/ClickHouse/ClickHouse/issues/12623). [#13009](https://github.com/ClickHouse/ClickHouse/pull/13009) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复带有 `UNION` 和 `JOIN` 的查询的 `Block structure mismatch` 错误. 修复 [#12602](https://github.com/ClickHouse/ClickHouse/issues/12602). [#12989](https://github.com/ClickHouse/ClickHouse/pull/12989) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 更正了 `merge_with_ttl_timeout` 逻辑, 当过期影响一个时间间隔内的多个分区时, 该逻辑无法正常工作. (作者 @excitoon). [#12982](https://github.com/ClickHouse/ClickHouse/pull/12982) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复从 DDL 查询创建的范围哈希字典的列重复. 这修复了[#10605](https://github.com/ClickHouse/ClickHouse/issues/10605). [#12857](https://github.com/ClickHouse/ClickHouse/pull/12857) ([alesapin](https://github.com/alesapin)). -* 修复对从本地副本中选择的线程数的不必要限制. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了当 `ALTER DELETE` 和 `ALTER MODIFY COLUMN` 查询作为单个突变同时执行时的罕见错误. 错误导致 `count.txt` 中的行数不正确,从而导致部分数据不正确. 此外, 修复了同时使用 `ALTER RENAME COLUMN` 和 `ALTER ADD COLUMN` 的小错误. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). -* 使用 `clickhouse` 字典源查询远程表时使用了错误的凭据. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). -* 修复 `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* 修复大元组的性能,这些元组在`IN` 部分被解释为函数. 当用户出于某种晦涩的原因编写 `WHERE x IN tuple(1, 2, ...)` 而不是 `WHERE x IN (1, 2, ...)` 的情况. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 input_format_parallel_parsing 的内存跟踪 (通过将线程附加到组). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* 在 `any(func())` 的情况下修复错误的优化 `optimize_move_functions_out_of_any=1`. [#12664](https://github.com/ClickHouse/ClickHouse/pull/12664) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了 [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) 使用 const 表达式修复布隆过滤器索引. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* 当代理不可用时 (不仅如此) 修复 StorageKafka 中的 SIGSEGV. [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* 添加对带有 `Array(UUID)` 参数的函数 `if` 的支持. 这修复了 [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果用户存在, CREATE USER IF NOT EXISTS 现在不会抛出异常. 这修复了[#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). -* 在意外情况下 (例如从 UInt64 列中减去时),在 `ALTER ... UPDATE` 期间可以抛出异常 `There is no supertype...`. 这修复了 [#7306](https://github.com/ClickHouse/ClickHouse/issues/7306). This fixes [#4165](https://github.com/ClickHouse/ClickHouse/issues/4165). [#12633](https://github.com/ClickHouse/ClickHouse/pull/12633) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复使用外部排序的查询可能出现的 `Pipeline stuck` 错误. 修复 [#12617](https://github.com/ClickHouse/ClickHouse/issues/12617). [#12618](https://github.com/ClickHouse/ClickHouse/pull/12618) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `OPTIMIZE DEDUPLICATE` 的错误 `Output of TreeExecutor is not sorted` . 修复 [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572). [#12613](https://github.com/ClickHouse/ClickHouse/pull/12613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复查询优化过程中函数 `any` 结果的别名可能丢失的问题. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* 在 DROP TABLE 上删除分布式表 (来自异步插入的块) 的数据. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -现在 ClickHouse 将在文件 `checksums.txt` 不存在时重新计算部件的校验和. 自 [#9827](https://github.com/ClickHouse/ClickHouse/issues/9827) 以来已损坏. [#12545](https://github.com/ClickHouse/ClickHouse/pull/12545) ([alesapin](https://github.com/alesapin)). -* 修复当 `enable_mixed_granularity_parts=1` 时, 在 `ALTER DELETE` 查询后导致旧部件损坏的错误. 修复 [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* 修复实时视图表中可能导致数据重复的竞争条件. LIVE VIEW 是一项实验性功能. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修复 `AggregateFunction(avg, ...)`值二进制格式的向后兼容性. 这修复了 [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当我们加入字典键的表达时, 修复 JOIN 与字典中的崩溃: `t JOIN dict ON expr(dict.id) = t.id`. 在这种情况下禁用字典连接优化. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). -* 当指定非常大的 LIMIT 或 OFFSET 时修复溢出. 这修复了 [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). 这修复了 [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* kafka: 修复 SIGSEGV 如果在批处理中间有错误消息. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). - -#### 改进 - -* 在 ZooKeeper 中保留少量日志. 当有许多服务器/表/插入时, 避免在离线副本的情况下过度增长 ZooKeeper 节点. [#13100](https://github.com/ClickHouse/ClickHouse/pull/13100) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在, 如果在 ALTER 或突变期间发生错误, 则异常会转发给客户端. 关闭 [#11329](https://github.com/ClickHouse/ClickHouse/issues/11329). [#12666](https://github.com/ClickHouse/ClickHouse/pull/12666) ([alesapin](https://github.com/alesapin)). -* 将 `QueryTimeMicroseconds`、`SelectQueryTimeMicroseconds` 和 `InsertQueryTimeMicroseconds` 添加到 `system.events`, 以及 system.metrics、processes、query_log 等. [#13028](https://github.com/ClickHouse/ClickHouse/pull/13028) ([ianton-ru](https://github.com/ianton-ru)). -* 将 `SelectedRows` 和 `SelectedBytes` 添加到 `system.events`, 以及 system.metrics、processes、query_log 等. [#12638](https://github.com/ClickHouse/ClickHouse/pull/12638) ([ianton-ru](https://github.com/ianton-ru)). -* 将 `current_database` 信息添加到 `system.query_log`. [#12652](https://github.com/ClickHouse/ClickHouse/pull/12652) ([Amos Bird](https://github.com/amosbird)). -* 允许 `TabSeparatedRaw` 作为输入格式. [#12009](https://github.com/ClickHouse/ClickHouse/pull/12009) ([hcz](https://github.com/hczhcz)). -* 现在 `joinGet` 支持多键查找. [#12418](https://github.com/ClickHouse/ClickHouse/pull/12418) ([Amos Bird](https://github.com/amosbird)). -* 允许 `*Map` 聚合函数处理带有 NULL 的数组. 修复 [#13157](https://github.com/ClickHouse/ClickHouse/issues/13157). [#13225](https://github.com/ClickHouse/ClickHouse/pull/13225) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 避免解析 DateTime 值时溢出, 这将导致其时区中的 unix 时间戳为负 (例如, 莫斯科的 `1970-01-01 00:00:00` ). 相反, 饱和为零. 这修复了 [#3470](https://github.com/ClickHouse/ClickHouse/issues/3470). This fixes [#4172](https://github.com/ClickHouse/ClickHouse/issues/4172). [#12443](https://github.com/ClickHouse/ClickHouse/pull/12443) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* AvroConfluent: 跳过 Kafka 墓碑记录 - 支持跳过损坏的记录 [#13203](https://github.com/ClickHouse/ClickHouse/pull/13203) ([Andrew Onyshchuk](https://github.com/oandrew)). -* 修复长查询的错误错误. 为了正确查询, 可能会出现除 `Max query size exceeded` 之外的语法错误. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `lgamma` 函数中的数据竞争. 这场比赛只在 `tsan` 中被抓到,真的没有副作用. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 ATTACH/ALTER/CREATE QUOTA 语句的 `Week` 间隔格式. [#13417](https://github.com/ClickHouse/ClickHouse/pull/13417) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). -* 现在, 在紧凑零件加工中遇到损坏零件时也会报告. 紧凑型零件是一项实验性功能. [#13282](https://github.com/ClickHouse/ClickHouse/pull/13282) ([Amos Bird](https://github.com/amosbird)). -* 修复 `geohashesInBox` 中的断言. 这修复了 [#12554](https://github.com/ClickHouse/ClickHouse/issues/12554). [#13229](https://github.com/ClickHouse/ClickHouse/pull/13229) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `parseDateTimeBestEffort` 中的断言. 这修复了 [#12649](https://github.com/ClickHouse/ClickHouse/issues/12649). [#13227](https://github.com/ClickHouse/ClickHouse/pull/13227) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Processors/PipelineExecutor 中的小优化: 跳出循环, 因为这样做很有意义. [#13058](https://github.com/ClickHouse/ClickHouse/pull/13058) ([Mark Papadakis](https://github.com/markpapadakis)). -* 支持不带 TABLE 关键字的 TRUNCATE 表. [#12653](https://github.com/ClickHouse/ClickHouse/pull/12653) ([Winter Zhang](https://github.com/zhang2014)). -* 修复默认情况下解释查询格式覆盖. 这修复了 [#12541](https://github.com/ClickHouse/ClickHouse/issues/12432). [#12541](https://github.com/ClickHouse/ClickHouse/pull/12541) ([BohuTANG](https://github.com/BohuTANG)). -* 允许以更标准的方式设置 JOIN 种类和类型: `LEFT SEMI JOIN` 而不是 `SEMI LEFT JOIN`. 目前两者都是正确的. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). -* 将 `multiple_joins_rewriter_version` 的默认值更改为 2. 它启用知道列名的新的多连接重写器. [#12469](https://github.com/ClickHouse/ClickHouse/pull/12469) ([Artem Zuikov](https://github.com/4ertus2)). -* 为对 S3 存储的请求添加多个指标. [#12464](https://github.com/ClickHouse/ClickHouse/pull/12464) ([ianton-ru](https://github.com/ianton-ru)). -* 使用 `--secure` 参数为clickhouse-benchmark使用正确的默认安全端口. 这修复了 [#11044](https://github.com/ClickHouse/ClickHouse/issues/11044). [#12440](https://github.com/ClickHouse/ClickHouse/pull/12440) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `Log`、`TinyLog`、`StripeLog` 引擎中的回滚插入错误. 在以前的版本中, 插入错误会导致表状态不一致 (这按照文档工作,对于这些表引擎来说是正常的). 这修复了 [#12402](https://github.com/ClickHouse/ClickHouse/issues/12402). [#12426](https://github.com/ClickHouse/ClickHouse/pull/12426) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为`Atomic` 数据库引擎实现`RENAME DATABASE` 和`RENAME DICTIONARY` - 添加隐式`{uuid}` 宏, 可在`ReplicatedMergeTree` 的ZooKeeper 路径中使用. 它适用于 `CREATE ... ON CLUSTER ...` 查询. 将 `show_table_uuid_in_table_create_query_if_not_nil` 设置为 `true` 以使用它. - 将 `ReplicatedMergeTree` 引擎参数设为可选, 默认使用 `/clickhouse/tables/{uuid}/{shard}/` 和 `{replica}`. 关闭 [#12135](https://github.com/ClickHouse/ClickHouse/issues/12135). - 小修正. - 这些更改破坏了 `Atomic` 数据库引擎的向后兼容性. 以前创建的“原子”数据库必须手动转换为新格式. 原子数据库是一个实验性功能. [#12343](https://github.com/ClickHouse/ClickHouse/pull/12343) ([tavplubix](https://github.com/tavplubix)). -* 将 `AWSAuthV4Signer` 分离到不同的记录器中, 从日志消息中删除过多的 `AWSClient: AWSClient`. [#12320](https://github.com/ClickHouse/ClickHouse/pull/12320) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 磁盘访问存储中更好的异常消息. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). -* 参数数目无效的函数 `in` 的更好例外. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* 修复有关自适应粒度的错误消息. [#12624](https://github.com/ClickHouse/ClickHouse/pull/12624) ([alesapin](https://github.com/alesapin)). -* 修复格式化后的设置解析. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). -* 如果 MergeTree 表不包含 ORDER BY 或 PARTITION BY, 则可能会请求 ALTER 清除所有列, 而 ALTER 将卡住. 已修复 [#7941](https://github.com/ClickHouse/ClickHouse/issues/7941). [#12382](https://github.com/ClickHouse/ClickHouse/pull/12382) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 避免在每次查询后从历史文件重新加载完成 (以避免历史与其他客户端会话重叠). [#13086](https://github.com/ClickHouse/ClickHouse/pull/13086) ([Azat Khuzhin](https://github.com/azat)). - -#### 性能改进 - -* 某些操作的内存使用量降低高达 2 倍. [#12424](https://github.com/ClickHouse/ClickHouse/pull/12424) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 针对匹配精确 PK 范围的查询优化 PK 查找. [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277) ([Ivan Babrou](https://github.com/bobrik)). -* 使用 `LowCardinality` 稍微优化非常短的查询. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). -* 通过 UInt8/UInt16 键略微提高聚合性能. [#13091](https://github.com/ClickHouse/ClickHouse/pull/13091) and [#13055](https://github.com/ClickHouse/ClickHouse/pull/13055) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 下推`LIMIT`步骤查询计划 (在子查询内). [#13016](https://github.com/ClickHouse/ClickHouse/pull/13016) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 并行主键查找和跳过部分的索引阶段, 如 [#11564](https://github.com/ClickHouse/ClickHouse/issues/11564) 中所述. [#12589](https://github.com/ClickHouse/ClickHouse/pull/12589) ([Ivan Babrou](https://github.com/bobrik)). -* 如果 `set optimize_if_transform_strings_to_enum = 1`, 则将函数 `if` 和 `transform` 的字符串类型参数转换为枚举. [#12515](https://github.com/ClickHouse/ClickHouse/pull/12515) ([Artem Zuikov](https://github.com/4ertus2)). -* 如果 `set optimize_monotonous_functions_in_order_by=1`, 则用 `ORDER BY` 中的参数替换单调函数. [#12467](https://github.com/ClickHouse/ClickHouse/pull/12467) ([Artem Zuikov](https://github.com/4ertus2)). -* 通过优化添加顺序, 如果 `set optimize_redundant_functions_in_order_by = 1`, 则用 `ORDER by x` 重写 `ORDER BY x, f(x)` . [#12404](https://github.com/ClickHouse/ClickHouse/pull/12404) ([Artem Zuikov](https://github.com/4ertus2)). -* 当子查询包含`WITH` 子句时允许下推谓词. 这修复了 [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). -* 提高读取紧凑部件的性能. 紧凑型零件是一项实验性功能. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* 尝试在 `DiskS3` 中实现流优化. DiskS3 是一项实验性功能. [#12434](https://github.com/ClickHouse/ClickHouse/pull/12434) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### 构建/测试/打包改进 - -* 使用 `shellcheck` 进行 sh 测试 linting. [#13200](https://github.com/ClickHouse/ClickHouse/pull/13200) [#13207](https://github.com/ClickHouse/ClickHouse/pull/13207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 GitHub 挂钩中添加为拉取请求设置标签的脚本. [#13183](https://github.com/ClickHouse/ClickHouse/pull/13183) ([alesapin](https://github.com/alesapin)). -* 删除一些递归子模块. 见 [#13378](https://github.com/ClickHouse/ClickHouse/issues/13378). [#13379](https://github.com/ClickHouse/ClickHouse/pull/13379) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 确保所有子模块都来自正确的 URL. Continuation of [#13379](https://github.com/ClickHouse/ClickHouse/issues/13379). This fixes [#13378](https://github.com/ClickHouse/ClickHouse/issues/13378). [#13397](https://github.com/ClickHouse/ClickHouse/pull/13397) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了对用户声明的设置的支持, 可以从内部查询访问. 当 ClickHouse 引擎用作另一个系统的组件时需要这样做. [#13013](https://github.com/ClickHouse/ClickHouse/pull/13013) ([Vitaly Baranov](https://github.com/vitlibar)). -* 在 TestFlows 中添加了对 INSERT 权限的 RBAC 功能的测试. 正在测试 SELECT 的扩展表. 添加了匹配新表引擎测试的要求. [#13340](https://github.com/ClickHouse/ClickHouse/pull/13340) ([MyroTk](https://github.com/MyroTk)). -* 修复压力测试中服务器重启时超时错误. [#13321](https://github.com/ClickHouse/ClickHouse/pull/13321) ([alesapin](https://github.com/alesapin)). -* 现在快速测试将等待服务器重试. [#13284](https://github.com/ClickHouse/ClickHouse/pull/13284) ([alesapin](https://github.com/alesapin)). -* 函数`materialize()` (用于 ClickHouse 测试的函数) 将按预期为 NULL 工作 - 通过将其转换为非常量列. [#13212](https://github.com/ClickHouse/ClickHouse/pull/13212) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 AArch64 中的 libunwind 构建. 这修复了 [#13204](https://github.com/ClickHouse/ClickHouse/issues/13204). [#13208](https://github.com/ClickHouse/ClickHouse/pull/13208) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 zkutil gtest 中进行更多重试以防止测试不稳定. [#13165](https://github.com/ClickHouse/ClickHouse/pull/13165) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 对 RBAC 测试流的小修正. [#13152](https://github.com/ClickHouse/ClickHouse/pull/13152) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修复 `00960_live_view_watch_events_live.py` 测试. [#13108](https://github.com/ClickHouse/ClickHouse/pull/13108) ([vzakaznikov](https://github.com/vzakaznikov)). -* 改进文档部署脚本中的缓存清除. [#13107](https://github.com/ClickHouse/ClickHouse/pull/13107) ([alesapin](https://github.com/alesapin)). -* 重写了一些孤立的测试到gtest. 从测试中移除无用的内容. [#13073](https://github.com/ClickHouse/ClickHouse/pull/13073) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 在 TestFlows 中添加了对 `SELECT` 权限的 RBAC 功能的测试. [#13061](https://github.com/ClickHouse/ClickHouse/pull/13061) ([Ritaank Tiwari](https://github.com/ritaank)). -* 在快速测试检查中重新运行一些测试. [#12992](https://github.com/ClickHouse/ClickHouse/pull/12992) ([alesapin](https://github.com/alesapin)). -* 修复 `rdkafka` 库中的 MSan 错误. 这将关闭 [#12990](https://github.com/ClickHouse/ClickHouse/issues/12990). 将 `rdkafka` 更新到 1.5 版 (主). [#12991](https://github.com/ClickHouse/ClickHouse/pull/12991) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果在带有 AVX-512 的服务器上运行测试, 则修复 base64 中的 UBSan 报告. 这修复了 [#12318](https://github.com/ClickHouse/ClickHouse/issues/12318). 作者: @qoega. [#12441](https://github.com/ClickHouse/ClickHouse/pull/12441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 HDFS 库中的 UBSan 报告. 这将关闭 [#12330](https://github.com/ClickHouse/ClickHouse/issues/12330). [#12453](https://github.com/ClickHouse/ClickHouse/pull/12453) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 检查我们能够将备份从旧版本恢复到新版本的能力. 这将关闭 [#8979](https://github.com/ClickHouse/ClickHouse/issues/8979). [#12959](https://github.com/ClickHouse/ClickHouse/pull/12959) ([alesapin](https://github.com/alesapin)). -* 不要在集成测试中构建 helper_container 图像. 在 CI 中构建 docker 容器并在集成测试中使用预构建的 helper_container. [#12953](https://github.com/ClickHouse/ClickHouse/pull/12953) ([Ilya Yatsishin](https://github.com/qoega)). -* 为主键列添加对 `ALTER TABLE CLEAR COLUMN` 查询的测试. [#12951](https://github.com/ClickHouse/ClickHouse/pull/12951) ([alesapin](https://github.com/alesapin)). -* 测试流测试中的超时时间增加. [#12949](https://github.com/ClickHouse/ClickHouse/pull/12949) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修复 Mac OS X 下的测试构建. 这将关闭 [#12767](https://github.com/ClickHouse/ClickHouse/issues/12767). [#12772](https://github.com/ClickHouse/ClickHouse/pull/12772) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 连接器-ODBC 更新为 mysql-connector-odbc-8.0.21. [#12739](https://github.com/ClickHouse/ClickHouse/pull/12739) ([Ilya Yatsishin](https://github.com/qoega)). -* 在 TestFlows 中添加 RBAC 语法测试. [#12642](https://github.com/ClickHouse/ClickHouse/pull/12642) ([vzakaznikov](https://github.com/vzakaznikov)). -* 提高 TestKeeper 的性能. 这将加速大量使用复制表的测试. [#12505](https://github.com/ClickHouse/ClickHouse/pull/12505) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在我们检查服务器是否能够在压力测试运行后启动. 这修复了 [#12473](https://github.com/ClickHouse/ClickHouse/issues/12473). [#12496](https://github.com/ClickHouse/ClickHouse/pull/12496) ([alesapin](https://github.com/alesapin)). -* 将 fmtlib 更新到 master (7.0.1). [#12446](https://github.com/ClickHouse/ClickHouse/pull/12446) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加 docker 镜像以进行快速测试. [#12294](https://github.com/ClickHouse/ClickHouse/pull/12294) ([alesapin](https://github.com/alesapin)). -* 集成测试的返工配置路径. [#12285](https://github.com/ClickHouse/ClickHouse/pull/12285) ([Ilya Yatsishin](https://github.com/qoega)). -* 添加编译器选项以控制堆栈帧不会太大. 这将有助于在具有小堆栈大小的纤程中运行代码. [#11524](https://github.com/ClickHouse/ClickHouse/pull/11524) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 更新 gitignore 文件. [#13447](https://github.com/ClickHouse/ClickHouse/pull/13447) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). - - -## ClickHouse 版本 20.6 - -### ClickHouse 版本 v20.6.3.28-stable - -#### 向后不兼容变更 - -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重启, 旧版本在新版本存在的情况下启动,可能会导致 到 `Part ... intersects previous part` 错误. 为防止出现此错误,首先在所有集群节点上安装更新的 clickhouse-server 软件包,然后重新启动 (因此, 当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* 添加了 `EXPLAIN` 查询的初始实现. 语法: `EXPLAIN SELECT ...`. 这修复了 [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 添加了存储 `RabbitMQ` . [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). -* 为 [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710) 实现了类似 PostgreSQL 的 `ILIKE` 操作符. [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). -* 支持 RIGHT 和 FULL JOIN 与 `SET join_algorithm = 'partial_merge'`. 只允许 ALL 严格 (ANY、SEMI、ANTI、ASOF 不是). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). -* 添加了一个函数 `initializeAggregation` 以初始化基于单个值的聚合. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). -* 支持 `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). -* 添加了函数 `parseDateTimeBestEffortUS` . [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). -* 支持输出格式 `ORC` (仅支持输入). [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). - -#### Bug 修复 - -* 修复了 `aggregate function any(x) is found inside another aggregate function in query` 错误, 其中包含 `SET optimize_move_functions_out_of_any = 1` 和 `any()` 中的别名. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了 clickhouse-local 的 `PrettyCompactMonoBlock` . 使用 `PrettyCompactMonoBlock` 修复了极端/总数. 这修复了 [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). -* 修复了在从延迟副本查询的情况下可能出现的错误 `Totals having transform was already added to pipeline` . [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果用户将特制的参数传递给函数 `h3ToChildren`, 服务器可能会崩溃. 这修复了 [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在具有 NaN 值的 Float 类型上调用的 `uniqExact`、`topK`、`sumDistinct` 和类似聚合函数的潜在低性能和稍微不正确的结果. 它还在调试版本中触发断言. 这修复了 [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了如果以nullable constexpr为cond, 则不是字面量 NULL 问题. 修复 [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在数组元素为 Nullable 且数组下标也可为 Null 的情况下, 修复了 `arrayElement` 函数中的断言. 这修复了 [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了带有常量参数的 `DateTime64` 转换函数. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). -* 用函数修正了错误的索引分析. 在从 `MergeTree` 表中读取时, 它可能会导致修剪错误的部分. 修复 [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了错误 `Cannot convert column because it is constant but values of constants are different in source and result`, 远程查询在查询范围内使用确定性函数, 但查询之间不具有确定性, 例如 `now()`、`now64()`、`randConstant()`. 修复 [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了对从本地副本进行选择的线程数的不必要限制. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了当 `ALTER DELETE` 和 `ALTER MODIFY COLUMN` 查询作为单个突变同时执行时的罕见错误. 错误导致 `count.txt` 中的行数不正确, 从而导致部分数据不正确. 另外, 修复了一个同时使用`ALTER RENAME COLUMN`和`ALTER ADD COLUMN` 的小错误. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). -* 修复了 `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* 修复了大元组的性能, 这些元组在 `IN` 部分被解释为函数. 当用户出于某种晦涩的原因编写 `WHERE x IN tuple(1, 2, ...)` 而不是 `WHERE x IN (1, 2, ...)` 的情况. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `input_format_parallel_parsing` 的内存跟踪 (通过将线程附加到组). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* 使用 const 表达式修复了布隆过滤器索引. 这修复了 [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了当 Broker 不可用时 (不仅如此) 在 `StorageKafka` 中的 `SIGSEGV`. [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* 添加了对带有 `Array(UUID)` 参数的函数 `if` 的支持. 这修复了 [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果用户存在, `CREATE USER IF NOT EXISTS` 现在不会抛出异常. 这修复了 [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). -* 磁盘访问存储中更好的异常消息. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). -* 函数 `groupArrayMoving*` 不适用于分布式查询. 它的结果是在不正确的数据类型中计算的 (没有升级到最大的类型). `groupArrayMovingAvg` 函数返回的整数与 `avg` 函数不一致. 这修复了 [#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了函数 `any` 缺少别名的问题. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了带有缓存布局的外部字典中的竞争条件, 这可能导致服务器崩溃. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* 在 DROP TABLE 上删除分布式表 (来自异步插入的块) 的数据. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当 `enable_mixed_granularity_parts=1` 时, 在 `ALTER DELETE` 查询后导致旧部件损坏的错误. 修复 [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* 参数数目无效的函数 `in` 的更好例外. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* 修复实时视图表中可能导致数据重复的竞争条件. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修正了从合并部分读取时的性能问题. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `AggregateFunction(avg, ...)` 值的二进制格式的向后兼容性. 这修复了 [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 固定设置解析格式后. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). -* 修复了启用 `text_log` 时的死锁. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了指定非常大的 `LIMIT` 或 `OFFSET` 时的溢出. 这修复了 [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果 `StorageMerge` , 则修复了可能的段错误. 这修复了 [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* 恢复在 [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) 中引入的更改以解决 [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). -* 额外检查布隆过滤器索引的参数. 这修复了 [#11408](https://github.com/ClickHouse/ClickHouse/issues/11408). [#12388](https://github.com/ClickHouse/ClickHouse/pull/12388) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在索引表的 WHERE 条件中使用负数或浮点常量时避免异常. 这修复了 [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在索引表的 WHERE 条件中使用负数或浮点常量时避免异常. 这修复了 [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复带有 `-State` 和 `Nullable` 参数的聚合函数的 `TOTALS/ROLLUP/CUBE`. 这修复了 [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在不允许使用 `RENAME` 时, `ALTER RENAME COLUMN` 查询的错误消息和退出代码. 修复 [#12301](https://github.com/ClickHouse/ClickHouse/issues/12301) 和 [#12303](https://github.com/ClickHouse/ClickHouse/issues/12303). [#12335](https://github.com/ClickHouse/ClickHouse/pull/12335) ([alesapin](https://github.com/alesapin)). -* 修复了 `ReplicatedMergeTree Queue` 中非常罕见的竞争条件. [#12315](https://github.com/ClickHouse/ClickHouse/pull/12315) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当使用具有非固定宽度类型的编解码器 `Delta` 或 `DoubleDelta` 时, 返回代码为 `LOGICAL_ERROR` 的异常而不是代码为 `BAD_ARGUMENTS` 的异常 (我们确保代码逻辑错误的异常永远不会发生). 这修复了 [#12110](https://github.com/ClickHouse/ClickHouse/issues/12110). [#12308](https://github.com/ClickHouse/ClickHouse/pull/12308) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `WITH FILL` 修饰符中列的顺序. 以前不遵守 `ORDER BY` 语句的列顺序. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). -* 当存在按虚拟列 (如 `Merge` 表中的 `_table`) 或系统表中的 "index" 列过滤数据 (例如从 `system.tables` 查询时按数据库名称过滤) 时, 避免 `bad cast` 异常 `, 并且这个表达式返回 `Nullable` 类型. 这修复了 [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了重命名列后的 `TTL`, 它依赖于 TTL 表达式. [#12304](https://github.com/ClickHouse/ClickHouse/pull/12304) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 SIGSEGV 如果在 `Kafka` 引擎中批处理中间有错误消息. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `DNS` 缓存更新过程中某些线程可能会随机挂起几秒钟的情况. [#12296](https://github.com/ClickHouse/ClickHouse/pull/12296) ([tavplubix](https://github.com/tavplubix)). -* 修复了设置名称中的错字. [#12292](https://github.com/ClickHouse/ClickHouse/pull/12292) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `TrieDictionary` 加载失败后显示错误. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). -* 对于可能导致崩溃的空数组, 函数 `arrayFill` 工作不正确. 这修复了 [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 实现到 `LowCardinality` 类型的公共类型的转换. 这允许对具有 LowCardinality 列和其他列的表执行 UNION ALL. 这修复了 [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). 这修复了 [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在对 `S3` 存储的请求中达到重定向限制的行为. [#12256](https://github.com/ClickHouse/ClickHouse/pull/12256) ([ianton-ru](https://github.com/ianton-ru)). -* 修复了在某些特殊类型的 `StorageFile` 标头中多次连续插入期间多次写入时的行为. 这个修复了 [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了 UInt8 值不等于 0 或 1 时的逻辑函数. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). -* 上限 max_memory_usage* 限制为进程驻留内存. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). -* 在 `GROUP BY` 内射函数消除期间修复 dictGet 参数检查. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当 `SummingMergeTree` 引擎从分区键中汇总列时的行为. 在列的显式定义与分区键列相交的情况下添加了一个例外. 这修复了 [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 如果 ODBC 连接不支持架构, 则不要将字典源的表名拆分为架构和表名本身. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了 `ALTER DELETE` 中的错误逻辑, 当条件评估为 NULL 时会导致删除记录. 这修复了 [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在存在别名的情况下发送到外部 DBMS (例如 MySQL、ODBC) 的查询转换. 这修复了[#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了冗余 ORDER BY 优化中的错误代码. 该错误是在 [#10067](https://github.com/ClickHouse/ClickHouse/issues/10067) 中引入的. [#12148](https://github.com/ClickHouse/ClickHouse/pull/12148) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了整数除法中的潜在溢出. 这修复了 [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `greatCircleDistance`、`geoDistance` 中潜在的无限循环. 这修复了 [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 规范化 `pid` 文件处理. 在以前的版本中, 如果服务器在没有正确关闭的情况下被杀死, 并且有另一个进程与以前运行的服务器具有相同的 pid, 则服务器可能会拒绝启动. 即使有另一台服务器正在运行, pid 文件也可能在服务器启动失败时被删除. 这修复了 [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了导致 ZooKeepeer 中 ReplicatedVersionedCollapsingMergeTree 表的表元数据不正确的错误. 修复 [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). -* 对于具有连接或附加到系统日志 (system.query_log、metric_log 等) 或 engine=Buffer 基础表的子查询的物化视图, 避免 `There is no query` 异常. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). -* 修复了对字典的 ENGINE=Dictionary 表的处理依赖性. 这修复了 [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). 这修复了 [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). -* 格式 `Parquet` 现在可以正确地与 `LowCardinality` 和 `LowCardinality(Nullable)` 类型一起使用. 修复 [#12086](https://github.com/ClickHouse/ClickHouse/issues/12086), [#8406](https://github.com/ClickHouse/ClickHouse/issues/8406). [#12108](https://github.com/ClickHouse/ClickHouse/pull/12108) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了由于线程总数限制错误导致的带有 `UNION` 的选择的性能. 修复 [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `-StateResample` 组合器修复了段错误. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `system.quey_log` 中选择的空 `result_rows` 和 `result_bytes` 指标. 修复 [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了不必要的限制从 `VIEW` 中选择的线程数. 修复 [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 DROP TABLE 上 StorageKafka 中的 SIGSEGV. [#12075](https://github.com/ClickHouse/ClickHouse/pull/12075) ([Azat Khuzhin](https://github.com/azat)). -* 修复了对 `PREWHERE` 使用错误类型时可能发生的崩溃。 修复 [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了带有 `Tuple(LowCardinality)` 参数的高阶函数的错误 `Cannot capture column`. 修复 [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 固定约束检查约束是否为常量表达式. 这修复了 [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在调用具有不同大小的 `FixedString` 类型参数的函数 `if` 时出现错误结果和潜在崩溃的问题. 这修复了 [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### 改进 - -* 允许以更标准的方式设置 `JOIN` 种类和类型: `LEFT SEMI JOIN`而不是`SEMI LEFT JOIN`. 目前两者都是正确的. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). -* 缓冲区引擎的lifetime_rows/lifetime_bytes. [#12421](https://github.com/ClickHouse/ClickHouse/pull/12421) ([Azat Khuzhin](https://github.com/azat)). -* 将详细异常消息写入客户端而不是 `MySQL server has gone away` . [#12383](https://github.com/ClickHouse/ClickHouse/pull/12383) ([BohuTANG](https://github.com/BohuTANG)). -* 允许更改用于打印网格边框的字符集. 可用的字符集如下: UTF-8、ASCII. 设置 `output_format_pretty_grid_charset` 启用此功能. [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). -* 1.支持 MySQL 'SELECT DATABASE()' [#9336](https://github.com/ClickHouse/ClickHouse/issues/9336). 2. 添加 MySQL 替换查询集成测试. [#12314](https://github.com/ClickHouse/ClickHouse/pull/12314) ([BohuTANG](https://github.com/BohuTANG)). -* 为 MySQL 客户端/驱动程序添加 `KILL QUERY [connection_id]` 以取消长查询, 问题 [#12038](https://github.com/ClickHouse/ClickHouse/issues/12038). [#12152](https://github.com/ClickHouse/ClickHouse/pull/12152) ([BohuTANG](https://github.com/BohuTANG)). -* 在 `formatDateTime` 函数中添加了对 `%g` (两位 ISO 年份) 和 `%G` (四位 ISO 年份) 替换的支持. [#12136](https://github.com/ClickHouse/ClickHouse/pull/12136) ([vivarum](https://github.com/vivarum)). -* 在 system.disks 中添加了 `type` 列. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). -* 改进的`REVOKE` 命令: 现在它只需要授予/管理选项才能被撤销. 例如, 现在要执行 `REVOKE ALL ON *.* FROM user1`, 不需要通过授予选项授予完全访问权限. 添加命令 `REVOKE ALL FROM user1` - 它从 `user1` 撤销所有授予的角色. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). -* 为 load_balancing 添加了副本优先级 (用于手动确定负载平衡的优先级). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). -* 将 S3 元数据中的路径切换为相对路径, 以便更轻松地处理 S3 blob. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### 性能改进 - -* 通过排序键的前缀改进了 `ORDER BY` 和 `GROUP BY` 的性能 (使用 `optimize_aggregation_in_order` 设置启用, 默认禁用). [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). -* 如果 `set optimize_injective_functions_inside_uniq=1`, 则删除 `uniq*()` 内的单射函数. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Ruslan Kamalov](https://github.com/kamalov-ruslan)). -* 索引不用于带文字的 IN 运算符, 在 v19.3 前后引入了性能回归. 这修复了[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). -* 为 DiskS3 (实验功能) 实现单部分上传. [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### 实验功能 -* 在 `MergeTree` -family 表中添加了新的部件内存格式, 将数据存储在内存中. 第一次合并时将部分写入磁盘. 如果零件的行或字节大小低于阈值 `min_rows_for_compact_part` 和 `min_bytes_for_compact_part`, 将以内存格式创建. 还提供了对 Write-Ahead-Log 的可选支持, 默认情况下启用并通过设置 `in_memory_parts_enable_wal` 来控制. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). - -#### 构建/测试/打包改进 - -* 为 clickhouse-client 实现基于 AST 的查询模糊测试模式. 请参阅 [this label](https://github.com/ClickHouse/ClickHouse/issues?q=label%3Afuzz+is%3Aissue) 了解我们最近通过模糊测试发现的问题列表. 其中大部分是由这个工具发现的, 还有一些是由 SQLancer 和 `00746_sql_fuzzy.pl` 发现的. [#12111](https://github.com/ClickHouse/ClickHouse/pull/12111) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 添加基于 Testflows 框架的新型测试. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). -* 添加了 S3 HTTPS 集成测试. [#12412](https://github.com/ClickHouse/ClickHouse/pull/12412) ([Pavel Kovalenko](https://github.com/Jokser)). -* 从单独的线程记录清除陷阱消息. 这将防止线程消毒剂下可能出现的死锁. [#12313](https://github.com/ClickHouse/ClickHouse/pull/12313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在功能和压力测试将能够使用旧版本的 `clickhouse-test` 脚本运行. [#12287](https://github.com/ClickHouse/ClickHouse/pull/12287) ([alesapin](https://github.com/alesapin)). -* 在 `orc` 构建期间删除奇怪的文件创建. [#12258](https://github.com/ClickHouse/ClickHouse/pull/12258) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 将常见的 docker compose 文件放置到集成 docker 容器中. [#12168](https://github.com/ClickHouse/ClickHouse/pull/12168) ([Ilya Yatsishin](https://github.com/qoega)). -* 修复来自 CodeQL 的警告. `CodeQL` 是另一个静态分析器, 我们将与我们已经使用的 `clang-tidy` 和 `PVS-Studio` 一起使用. [#12138](https://github.com/ClickHouse/ClickHouse/pull/12138) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* UNBUNDLED 构建的小 CMake 修复. [#12131](https://github.com/ClickHouse/ClickHouse/pull/12131) ([Matwey V. Kornilov](https://github.com/matwey)). -* 添加了一个不使用任何 Linux 发行版的最小 Docker 镜像的展示. [#12126](https://github.com/ClickHouse/ClickHouse/pull/12126) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 `clickhouse-server` docker 镜像中执行系统包的升级. [#12124](https://github.com/ClickHouse/ClickHouse/pull/12124) ([Ivan Blinkov](https://github.com/blinkov)). -* 将 `UNBUNDLED` 标志添加到 `system.build_options` 表. 将 `clickhouse-test` 的跳过列表移动到 clickhouse repo. [#12107](https://github.com/ClickHouse/ClickHouse/pull/12107) ([alesapin](https://github.com/alesapin)). -* 通过 [Anchore Container Analysis](https://docs.anchore.com) 安全分析工具定期检查, 该工具会在 `clickhouse-server` Docker 镜像中查找 [CVE](https://cve.mitre.org/). 还确认`Dockerfile` 是可构建的. 每天在 `master` 和 `Dockerfile` 的拉取请求上运行. [#12102](https://github.com/ClickHouse/ClickHouse/pull/12102) ([Ivan Blinkov](https://github.com/blinkov)). -* 每日检查 [GitHub CodeQL](https://securitylab.github.com/tools/codeql) 安全分析工具, 寻找 [CWE](https://cwe.mitre.org/). [#12101](https://github.com/ClickHouse/ClickHouse/pull/12101) ([Ivan Blinkov](https://github.com/blinkov)). -* 在 Dockerfile 中的第一个 `apt-get update` 之前安装 `ca-certificates`. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). - -## ClickHouse 版本 20.5 - -### ClickHouse 版本 v20.5.4.40-stable 2020-08-10 - -#### Bug 修复 - -* 用函数修正了错误的索引分析. 在从 `MergeTree` 表中读取时, 它可能会导致修剪错误的部分. 修复 [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了对从本地副本进行选择的线程数的不必要限制. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了大元组的性能, 这些元组在 `IN` 部分被解释为函数. 当用户出于某种晦涩的原因编写 `WHERE x IN tuple(1, 2, ...)` 而不是 `WHERE x IN (1, 2, ...)` 的情况. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 input_format_parallel_parsing 的内存跟踪 (通过将线程附加到组). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* 使用 const 表达式修复了布隆过滤器索引. 这修复了 [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了当 Broker 不可用时 (不仅如此) 在 `StorageKafka` 中的 `SIGSEGV`. [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* 添加了对带有 `Array(UUID)` 参数的函数 `if` 的支持. 这修复了 [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了函数 `any` 缺少别名的问题. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了带有缓存布局的外部字典中的竞争条件, 这可能导致服务器崩溃. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* 在 DROP TABLE 上删除分布式表 (来自异步插入的块) 的数据. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当 `enable_mixed_granularity_parts=1` 时, 在 `ALTER DELETE` 查询后导致旧部件损坏的错误. 修复 [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* 参数数目无效的函数 `in` 的更好例外. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了实时视图表中可能导致数据重复的竞争条件. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修正了从合并部分读取时的性能问题. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `AggregateFunction(avg, ...)` 值的二进制格式的向后兼容性. 这修复了 [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了启用 `text_log` 时的死锁. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了当指定非常大的限制或偏移量时的溢出问题. 这修复了 [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 StorageMerge 时可能出现的段错误. 关闭 [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* 还原 [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) 中引入的更改以解决 [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). -* 在索引表的 WHERE 条件中使用负数或浮点常量时避免异常. 这修复了 [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 即使存在依赖的 DEFAULT 表达式,也允许 CLEAR 列. 这修复了 [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了带有 `-State` 和 `Nullable` 参数的聚合函数的 TOTALS/ROLLUP/CUBE. 这修复了 [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 SIGSEGV 如果在 `Kafka` 引擎中批处理中间有错误消息. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当 `SummingMergeTree` 引擎从分区键中汇总列时的行为. 在列的显式定义与分区键列相交的情况下添加了一个例外. 这修复了 [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了在存在别名的情况下发送到外部 DBMS (例如 MySQL、ODBC) 的查询转换. 这修复了 [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了导致 ZooKeepeer 中 ReplicatedVersionedCollapsingMergeTree 表的表元数据不正确的错误. 修复 [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). -* 修复了从 `VIEW` 中选择的不必要的线程数限制. 修复 [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `join_algorithm=partial_merge` 修复了具有 LowCardinality 类型的 JOIN 崩溃. [#12035](https://github.com/ClickHouse/ClickHouse/pull/12035) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了条件为 NULL 的 `if()` 的错误结果. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). - -#### 性能改进 - -* 索引不用于带文字的 IN 运算符, 在 v19.3 前后引入了性能回归. 这修复了 [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). - -#### 构建/测试/打包改进 - -* 在 Dockerfile 中的第一个 `apt-get update` 之前安装 `ca-certificates`. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). - - -### ClickHouse 版本 v20.5.2.7-stable 2020-07-02 - -#### 向后不兼容变更 - -* 从 COUNT(DISTINCT) 和 `uniq` 聚合函数系列返回不可为空的结果. 如果所有传递的值都是 NULL, 则返回零. 这提高了 SQL 兼容性. [#11661](https://github.com/ClickHouse/ClickHouse/pull/11661) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了对在错误位置指定用户级设置的情况的检查. 用户级设置应该在 `` 部分内的 `users.xml` 中为特定用户配置文件指定 (或在 `` 中指定默认设置). 服务器不会以日志中的异常消息启动. 这修复了 [#9051](https://github.com/ClickHouse/ClickHouse/issues/9051). 如果您想跳过检查, 您可以将设置移动到适当的位置或将 `1` 添加到 config.xml. [#11449](https://github.com/ClickHouse/ClickHouse/pull/11449) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 默认情况下启用设置 `input_format_with_names_use_header`. 它会影响对输入格式 `-WithNames` 和 `-WithNamesAndTypes` 的解析. [#10937](https://github.com/ClickHouse/ClickHouse/pull/10937) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 删除 `experimental_use_processors` 设置. 它默认启用. [#10924](https://github.com/ClickHouse/ClickHouse/pull/10924) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 将 `zstd` 更新到 1.4.4. 它在性能和压缩比方面有一些小的改进. 如果您使用不同版本的 ClickHouse 运行副本, 您可能会看到合理的错误消息 `Data after merge is not byte-identical to data on another replicas.` 并附有说明. 这些消息没问题, 您不必担心. 此更改向后兼容, 但我们将其列在更改日志中, 以防您对这些消息感到疑惑. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了对无意义编解码器的检查和设置 `allow_suspicious_codecs` 来控制此检查. 这将关闭 [#4966](https://github.com/ClickHouse/ClickHouse/issues/4966). [#10645](https://github.com/ClickHouse/ClickHouse/pull/10645) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 一些 Kafka 设置更改了它们的默认值. 见 [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388). -* 从20.5以上的版本升级时, 如果进行滚动更新, 集群同时包含20.5以上和20.5以下的版本, 如果旧版本的ClickHouse节点重新启动, 旧版本在新版本存在的情况下启动, 可能会导致 `Part ... intersects previous part` 错误. 为防止出现此错误, 首先在所有集群节点上安装更新的 clickhouse-server 软件包, 然后重新启动 (因此, 当 clickhouse-server 重新启动时, 它将以新版本启动). - -#### 新特征 - -* `TTL DELETE WHERE` 和 `TTL GROUP BY` 用于表中的自动数据粗化和汇总. [#10537](https://github.com/ClickHouse/ClickHouse/pull/10537) ([expl0si0nn](https://github.com/expl0si0nn)). -* PostgreSQL 有线协议的实现. [#10242](https://github.com/ClickHouse/ClickHouse/pull/10242) ([Movses](https://github.com/MovElb)). -* 为 users, roles, grants, settings profiles, quotas, row policies 添加了系统表; 添加了命令 SHOW USER, SHOW [CURRENT|ENABLED] ROLES, SHOW SETTINGS PROFILES. [#10387](https://github.com/ClickHouse/ClickHouse/pull/10387) ([Vitaly Baranov](https://github.com/vitlibar)). -* 支持写入 ODBC 表函数 [#10554](https://github.com/ClickHouse/ClickHouse/pull/10554) ([ageraab](https://github.com/ageraab)). [#10901](https://github.com/ClickHouse/ClickHouse/pull/10901) ([tavplubix](https://github.com/tavplubix)). -* 添加基于 Linux `perf_events` 的查询性能指标 (这些指标是使用硬件 CPU 计数器和 OS 计数器计算的). 它是可选的, 需要在 clickhouse 二进制文件上设置 `CAP_SYS_ADMIN`. [#9545](https://github.com/ClickHouse/ClickHouse/pull/9545) [Andrey Skobtsov](https://github.com/And42). [#11226](https://github.com/ClickHouse/ClickHouse/pull/11226) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 现在支持 `CREATE` 查询中数据类型的 `NULL` 和 `NOT NULL` 修饰符. [#11057](https://github.com/ClickHouse/ClickHouse/pull/11057) ([Павел Потемкин](https://github.com/Potya)). -* 添加 `ArrowStream` 输入输出格式. [#11088](https://github.com/ClickHouse/ClickHouse/pull/11088) ([hcz](https://github.com/hczhcz)). -* 支持 Cassandra 作为外部字典源. [#4978](https://github.com/ClickHouse/ClickHouse/pull/4978) ([favstovol](https://github.com/favstovol)). -* 添加了一个新的布局 `direct`, 它直接从每个查询的源加载所有数据, 而不存储或缓存数据. [#10622](https://github.com/ClickHouse/ClickHouse/pull/10622) ([Artem Streltsov](https://github.com/kekekekule)). -* 向字典添加了新的 `complex_key_direct` 布局, 在查询执行期间不会在本地存储任何内容. [#10850](https://github.com/ClickHouse/ClickHouse/pull/10850) ([Artem Streltsov](https://github.com/kekekekule)). -* 添加了对 MySQL 样式全局变量语法 (存根) 的支持. 这是 MySQL 协议兼容性所必需的. [#11832](https://github.com/ClickHouse/ClickHouse/pull/11832) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用 `repl` 为 `clickhouse-client` 添加了语法高亮. [#11422](https://github.com/ClickHouse/ClickHouse/pull/11422) ([Tagir Kuskarov](https://github.com/kuskarov)). -* 添加了 `mind Map` 和 `maxMap` 函数. [#11603](https://github.com/ClickHouse/ClickHouse/pull/11603) ([Ildus Kurbangaliev](https://github.com/ildus)). -* 添加记录来自 `system.asynchronous_metrics` 的历史指标的 `system.asynchronous_metric_log` 表. [#11588](https://github.com/ClickHouse/ClickHouse/pull/11588) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 添加函数 `extractAllGroupsHorizontal(haystack, re)` 和 `extractAllGroupsVertical(haystack, re)`. [#11554](https://github.com/ClickHouse/ClickHouse/pull/11554) ([Vasily Nemkov](https://github.com/Enmk)). -* 添加 SHOW CLUSTER(S) 查询. [#11467](https://github.com/ClickHouse/ClickHouse/pull/11467) ([hexiaoting](https://github.com/hexiaoting)). -* 添加`netloc`函数用于提取网络位置, 类似于python中的`urlparse(url)`、`netloc`. [#11356](https://github.com/ClickHouse/ClickHouse/pull/11356) ([Guillaume Tassery](https://github.com/YiuRULE)). -* 为 engine=Kafka 添加 2 个虚拟列以访问消息头. [#11283](https://github.com/ClickHouse/ClickHouse/pull/11283) ([filimonov](https://github.com/filimonov)). -* 为 Kafka 引擎添加 `_timestamp_ms` 虚拟列 (类型为 `Nullable(DateTime64(3))`) . [#11260](https://github.com/ClickHouse/ClickHouse/pull/11260) ([filimonov](https://github.com/filimonov)). -* 添加函数 `randomFixedString`. [#10866](https://github.com/ClickHouse/ClickHouse/pull/10866) ([Andrei Nekrashevich](https://github.com/xolm)). -* 添加函数 `fuzzBits`, 以给定的概率随机翻转字符串中的位. [#11237](https://github.com/ClickHouse/ClickHouse/pull/11237) ([Andrei Nekrashevich](https://github.com/xolm)). -* 允许在比较运算符、IN 和 VALUES 部分中将数字与常量字符串进行比较. [#11647](https://github.com/ClickHouse/ClickHouse/pull/11647) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加 `round_robin` 负载平衡模式. [#11645](https://github.com/ClickHouse/ClickHouse/pull/11645) ([Azat Khuzhin](https://github.com/azat)). -* 添加 `cast_keep_nullable` 设置. 如果设置 `CAST(something_nullable AS Type)` 返回 `Nullable(Type)` . [#11733](https://github.com/ClickHouse/ClickHouse/pull/11733) ([Artem Zuikov](https://github.com/4ertus2)). -* 将列 `position` 添加到 `system.columns` 表, 将 `column_position` 添加到 `system.parts_columns` 表. 它包含从 1 开始的表中列的顺序位置.这将关闭 [#7744](https://github.com/ClickHouse/ClickHouse/issues/7744). [#11655](https://github.com/ClickHouse/ClickHouse/pull/11655) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* ON CLUSTER 支持系统 {FLUSH DISTRIBUTED,STOP/START DISTRIBUTED SEND}. [#11415](https://github.com/ClickHouse/ClickHouse/pull/11415) ([Azat Khuzhin](https://github.com/azat)). -* 添加 system.distribution_queue 表. [#11394](https://github.com/ClickHouse/ClickHouse/pull/11394) ([Azat Khuzhin](https://github.com/azat)). -* 支持Kafka中的所有格式设置, 在表级别公开一些设置, 调整默认值以获得更好的性能. [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388) ([filimonov](https://github.com/filimonov)). -* 添加 `port` 函数 (从 URL 中提取端口). [#11120](https://github.com/ClickHouse/ClickHouse/pull/11120) ([Azat Khuzhin](https://github.com/azat)). -* 现在 `dictGet*` 函数接受表名. [#11050](https://github.com/ClickHouse/ClickHouse/pull/11050) ([Vitaly Baranov](https://github.com/vitlibar)). -* 当使用 `-n` 参数时, `clickhouse-format` 工具现在能够格式化多个查询. [#10852](https://github.com/ClickHouse/ClickHouse/pull/10852) ([Darío](https://github.com/dgrr)). -* 可以为 DiskS3 配置代理解析器. [#10744](https://github.com/ClickHouse/ClickHouse/pull/10744) ([Pavel Kovalenko](https://github.com/Jokser)). -* 使 `pointInPolygon` 与非常量多边形一起工作. PointInPolygon 现在可以将 Array(Array(Tuple(..., ...))) 作为第二个参数, 多边形和孔的数组. [#10623](https://github.com/ClickHouse/ClickHouse/pull/10623) ([Alexey Ilyukhov](https://github.com/livace)) [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* 将 `move_ttl_info` 添加到 `system.parts` 以提供对移动 TTL 功能的内省. [#10591](https://github.com/ClickHouse/ClickHouse/pull/10591) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 可以通过代理使用 S3. [#10576](https://github.com/ClickHouse/ClickHouse/pull/10576) ([Pavel Kovalenko](https://github.com/Jokser)). -* 为数据类型添加 `NCHAR` 和 `NVARCHAR` 同义词. [#11025](https://github.com/ClickHouse/ClickHouse/pull/11025) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 已解决 [#7224](https://github.com/ClickHouse/ClickHouse/issues/7224): 将 `FailedQuery`、`FailedSelectQuery` 和 `FailedInsertQuery` 指标添加到 `system.events` 表. [#11151](https://github.com/ClickHouse/ClickHouse/pull/11151) ([Nikita Orlov](https://github.com/naorlov)). -* 向 `system.asynchronous_metrics` 添加更多 `jemalloc` 统计信息, 并确保我们看到它们的最新值. [#11748](https://github.com/ClickHouse/ClickHouse/pull/11748) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 允许指定默认 S3 凭据和自定义身份验证标头. [#11134](https://github.com/ClickHouse/ClickHouse/pull/11134) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). -* 添加了以各种精度将 DateTime64 导入/导出为 Int64 的新函数: `to-/fromUnixTimestamp64Milli/-Micro/-Nano`. [#10923](https://github.com/ClickHouse/ClickHouse/pull/10923) ([Vasily Nemkov](https://github.com/Enmk)). -* 允许为 MongoDB 字典指定 `mongodb://` URI. [#10915](https://github.com/ClickHouse/ClickHouse/pull/10915) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 现在可以在没有附属 LIMIT 子句的情况下使用 OFFSET 关键字. [#10802](https://github.com/ClickHouse/ClickHouse/pull/10802) ([Guillaume Tassery](https://github.com/YiuRULE)). -* 添加了 `system.licenses` 表. 此表包含位于 `contrib` 目录中的第三方库的许可证. 这将关闭 [#2890](https://github.com/ClickHouse/ClickHouse/issues/2890). [#10795](https://github.com/ClickHouse/ClickHouse/pull/10795) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 新函数 toStartOfSecond(DateTime64) -> DateTime64 使 DateTime64 值的亚秒部分无效. [#10722](https://github.com/ClickHouse/ClickHouse/pull/10722) ([Vasily Nemkov](https://github.com/Enmk)). -* 添加新的输入格式 `JSONAsString` , 它接受由换行符、空格和/或逗号分隔的一系列 JSON 对象. [#10607](https://github.com/ClickHouse/ClickHouse/pull/10607) ([Kruglov Pavel](https://github.com/Avogar)). -* 允许以比 4 MiB 更细的粒度步长分析内存. 添加了采样内存分析器以捕获随机分配/解除分配. [#10598](https://github.com/ClickHouse/ClickHouse/pull/10598) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `SimpleAggregateFunction` 现在也支持 `sumMap`. [#10000](https://github.com/ClickHouse/ClickHouse/pull/10000) ([Ildus Kurbangaliev](https://github.com/ildus)). -* 分布式表引擎支持 `ALTER RENAME COLUMN` . 继续 [#10727](https://github.com/ClickHouse/ClickHouse/issues/10727). 修复 [#10747](https://github.com/ClickHouse/ClickHouse/issues/10747). [#10887](https://github.com/ClickHouse/ClickHouse/pull/10887) ([alesapin](https://github.com/alesapin)). - -#### Bug 修复 - -* 修复十进制解析中的 UBSan 报告. 这修复了 [#7540](https://github.com/ClickHouse/ClickHouse/issues/7540). [#10512](https://github.com/ClickHouse/ClickHouse/pull/10512) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 解析 DateTime64 时修复潜在的浮点异常. 这修复了 [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. [#11895](https://github.com/ClickHouse/ClickHouse/pull/11895) [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608) [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 不允许在高阶函数中使用 arrayJoin. 它导致协议同步中断. 这将关闭 [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 FixedString 与常量 String 比较的错误结果. 这修复了 [#11393](https://github.com/ClickHouse/ClickHouse/issues/11393). 这个bug出现在20.4版本. [#11828](https://github.com/ClickHouse/ClickHouse/pull/11828) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复条件为 NULL 的 `if` 的错误结果. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复使用过多线程进行查询. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在 `SELECT ... FROM merge_tree_table ...` 中使用 `WITH ...` 时的 `Scalar does not exist` 异常 [#11621](https://github.com/ClickHouse/ClickHouse/issues/11621). [#11767](https://github.com/ClickHouse/ClickHouse/pull/11767) ([Amos Bird](https://github.com/amosbird)). -* 修复像 `SELECT *, xyz.*` 这样的查询的意外行为, 这些查询在预期错误时成功. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* 现在, 在修改元数据期间, 复制的取数据将被取消. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). -* 在检查相等性之前解析存储在 zookeeper 中的元数据. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). -* 修复了由 Values 输入格式中复杂文字的错误类型推导导致的 LOGICAL_ERROR. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* 修复 `ORDER BY ... WITH FILL` 在 const 列上. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* 复 SYSTEM SYNC REPLICA 中非常罕见的竞争条件. 如果创建了复制表, 同时另一个客户端从单独的连接中对该表发出 `SYSTEM SYNC REPLICA` 命令 (这不太可能, 因为另一个客户端应该知道该表已创建), 则有可能获得 nullptr 取消引用. [#11691](https://github.com/ClickHouse/ClickHouse/pull/11691) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 与 XDBC 桥通信时传递适当的超时. 最近在检查桥接活性和接收元信息时没有遵守超时. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 将 `LIMIT n WITH TIES` 与包含别名的 `ORDER BY` 语句一起使用. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). -* 修复带有并行 `FINAL` 的选择的可能的 `Pipeline stuck` . 修复 [#11636](https://github.com/ClickHouse/ClickHouse/issues/11636). [#11682](https://github.com/ClickHouse/ClickHouse/pull/11682) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复导致 `system.mutations` 状态不正确的错误. 它可能表明整个突变已经完成, 但服务器在复制队列中仍有 `MUTATE_PART` 任务并尝试执行它们. 这修复了 [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). -* 修复 CREATE USER 查询中的语法 hilite. [#11664](https://github.com/ClickHouse/ClickHouse/pull/11664) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加对带有不区分大小写标志的正则表达式的支持. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) 并修复 [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果设置了行级安全性, 则删除琐碎的计数查询优化. 在以前的版本中, 用户获取表中记录的总数而不是过滤。 这修复了 [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复字符串的布隆过滤器 (数据跳过索引). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* 如果没有 `-q` 选项, 数据库不会在启动时创建. [#11604](https://github.com/ClickHouse/ClickHouse/pull/11604) ([giordyb](https://github.com/giordyb)). -* 修复错误 `Block structure mismatch`, 用于从“缓冲区”表中采样读取的查询. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 clickhouse-client 错误的退出代码, 当 `exception.code() % 256 == 0`. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* 修复 ReplicatedMergeTree 的不同副本的 CREATE/DROP 中的竞争条件. 如果表未从 ZooKeeper 中完全删除或未成功创建, 则继续工作. 这修复了 [#11432](https://github.com/ClickHouse/ClickHouse/issues/11432). [#11592](https://github.com/ClickHouse/ClickHouse/pull/11592) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复服务器启动时有关 `Mark cache size was lowered` 的日志消息中的小错误. 这将关闭 [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复错误 `Size of offsets does not match size of column` , 用于查询中的 `PREWHERE column in (subquery)` 和 `ARRAY JOIN` . [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 `SHOW CREATE TABLE` 中罕见的段错误. 修复 [#11490](https://github.com/ClickHouse/ClickHouse/issues/11490). [#11579](https://github.com/ClickHouse/ClickHouse/pull/11579) ([tavplubix](https://github.com/tavplubix)). -* HTTP 会话中的所有查询都具有相同的 query_id. 它是固定的. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). -* 现在 clickhouse-server docker 容器将更喜欢 IPv6 检查服务器活动性. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* 修复了在启用 `min_bytes_to_use_direct_io` 且 PREWHERE 处于活动状态并使用 SAMPLE 或大量线程时可能发生的 `Data compressed with different methods` 错误. 这修复了 [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `` 的 shard_num/replica_num (破坏 use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). -* 将异步 INSERT 修复到分布式中, 以用于 prefer_localhost_replica=0 和 w/o internal_replication. [#11527](https://github.com/ClickHouse/ClickHouse/pull/11527) ([Azat Khuzhin](https://github.com/azat)). -* 使用 `-State` 函数在聚合过程中抛出异常时修复内存泄漏. 这修复了 [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `INSERT SELECT FINAL` 的 `Pipeline stuck` 异常, 其中 `SELECT` (`max_threads`> 1)有多个流, 而` INSERT` 只有一个 (`max_insert_threads`==0). [#11455](https://github.com/ClickHouse/ClickHouse/pull/11455) ([Azat Khuzhin](https://github.com/azat)). -* 修复查询中的错误结果, 例如 `select count() from t, u`. [#11454](https://github.com/ClickHouse/ClickHouse/pull/11454) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复编解码器的返回压缩大小. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当列具有带有非文字参数的压缩编解码器时修复服务器崩溃. 修复 [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* 修复了当表没有成功创建时, 在合并树关闭时可能出现的未初始化内存读取问题. [#11420](https://github.com/ClickHouse/ClickHouse/pull/11420) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 JOIN 在 `LowCarinality(T)` 和 `Nullable(T)` 上的崩溃. [#11380](https://github.com/ClickHouse/ClickHouse/issues/11380). [#11414](https://github.com/ClickHouse/ClickHouse/pull/11414) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复错误 `USING` 键的错误代码. [#11373](https://github.com/ClickHouse/ClickHouse/issues/11373). [#11404](https://github.com/ClickHouse/ClickHouse/pull/11404) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了参数超出纬度/经度范围的 `geohashesInBox`. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* `joinGet()` 函数更好的错误. [#11389](https://github.com/ClickHouse/ClickHouse/pull/11389) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复具有外部排序和限制的查询可能出现的 `Pipeline stuck` 错误. 修复 [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 在 ReplicatedMergeTree 中发送零件期间删除冗余锁. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). -* 修复了多行模式下 clickhouse-client 对 `\G`(垂直输出)的支持. 这将关闭 [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复使用 `Lazy` 数据库时潜在的段错误. [#11348](https://github.com/ClickHouse/ClickHouse/pull/11348) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复从 `Join` 表引擎 (没有 JOIN) 直接选择的崩溃和错误的可空性. [#11340](https://github.com/ClickHouse/ClickHouse/pull/11340) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复 `quantilesExactWeightedArray` 中的崩溃问题. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 现在合并在 `ALTER` 查询中更改元数据之前停止. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). -* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当提取的 JSON 具有不平衡的 { 或 [ 字符串时,修复 `visitParamExtractRaw`. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* 修复 ThreadPool 中非常罕见的竞争条件. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `clickhouse-copier` 中无关紧要的数据竞争. 通过集成测试发现. [#11313](https://github.com/ClickHouse/ClickHouse/pull/11313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复转换中潜在的未初始化内存. 示例: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了如果表的主键中有 Array 列并且查询正在使用 `empty` 或 `notEmpty` 函数按此列过滤时索引分析无法工作的问题. 这修复了 [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复当查询速度估计可能不正确时的错误, 并且如果查询受到 `max_network_bandwidth`、`max_execution_speed` 或 `priority` 设置的限制, `min_execution_speed` 的限制可能无法工作或无法正常工作。 将 `timeout_before_checking_execution_speed` 的默认值更改为非零值, 否则设置 `min_execution_speed` 和 `max_execution_speed` 无效/ 这修复了 [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). 这修复了 [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). 这修复了 [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). 可用性改进: 避免在 `clickhouse-client` 中将异常消息与进度条串联. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了使用错误参数调用 `SET DEFAULT ROLE` 时的崩溃. 这修复了 [#10586](https://github.com/ClickHouse/ClickHouse/issues/10586). [#11278](https://github.com/ClickHouse/ClickHouse/pull/11278) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复在以 `Protobuf` 格式读取格式错误的数据时崩溃的问题. 这修复了 [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了 `cache` 字典可以返回默认值而不是正常值 (只有过期键时) 的错误. 这仅影响字符串字段. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复错误 `Block structure mismatch in QueryPipeline` , 同时从内部查询中的常量读取 `VIEW`. 修复 [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复可能的异常 `Invalid status for associated output` . [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 现在将检查 `primary.idx` 是否在 `CREATE` 查询中定义. [#11199](https://github.com/ClickHouse/ClickHouse/pull/11199) ([alesapin](https://github.com/alesapin)). -* 使用 `Array(Array(LowCardinality))` 捕获参数修复高阶函数的可能错误 `Cannot capture column` . [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在超过 1000 个键和一些后端的情况下可能会失败的 `S3` 通配符. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 如果数据跳过索引依赖于将在后台合并期间修改的列 (对于 SummingMergeTree、AggregatingMergeTree 以及 TTL GROUP BY), 则计算错误. 此问题已通过合并后移动索引计算得到修复, 因此索引是在合并数据上计算的. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* 修复了在表引擎 = Kafka 的 DROP 期间 (或在服务器重新启动期间) 有时发生的挂起. [#11145](https://github.com/ClickHouse/ClickHouse/pull/11145) ([filimonov](https://github.com/filimonov)). -* 修复简单查询过多的线程预留 (优化减少线程数量, 在管道更改后部分中断). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). -* 如果没有最终确定, 则从突变终止任务中删除日志记录. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* 修复了更新后服务器启动过程中的死锁, 系统日志表的结构发生了变化. [#11106](https://github.com/ClickHouse/ClickHouse/pull/11106) ([alesapin](https://github.com/alesapin)). -* 修复了 registerDiskS3 中的内存泄漏. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). -* 当 JOIN 出现在 PREWHERE 或 `optimize_move_to_prewhere` 使 PREWHERE from WHERE 出现时, 修复错误 `No such name in Block::erase()`. [#11051](https://github.com/ClickHouse/ClickHouse/pull/11051) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复 Kafka 引擎表终止期间可能丢失的数据. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). -* 修复了 parseDateTime64BestEffort 参数解析错误. [#10925](https://github.com/ClickHouse/ClickHouse/issues/10925). [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* 现在可以在单个 `ALTER` 查询中对同一列进行 `ADD/DROP` 和 `RENAME`. 同时 `MODIFY` 和 `RENAME` 的异常信息变得更加清晰. 部分修复 [#10669](https://github.com/ClickHouse/ClickHouse/issues/10669). [#11037](https://github.com/ClickHouse/ClickHouse/pull/11037) ([alesapin](https://github.com/alesapin)). -* 修复了 S3 URL 的解析. [#11036](https://github.com/ClickHouse/ClickHouse/pull/11036) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复存在 `LIMIT` 时两级 `GROUP BY` 的内存跟踪. [#11022](https://github.com/ClickHouse/ClickHouse/pull/11022) ([Azat Khuzhin](https://github.com/azat)). -* 如果表未成功创建,则修复 MergeTree 中非常罕见的潜在释放后使用错误. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复原子数据库的元数据 (重命名的相对路径) 和数据 (符号链接的相对路径) 处理. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). -* 使用 `Atomic` 数据库引擎修复并发 `ALTER` 和 `DROP DATABASE` 查询时的服务器崩溃. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). -* 修复方法 getRawData() 中不正确的原始数据大小. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* 修复了 20.1 及更早版本之间两级聚合的不兼容性. 当在发起节点和远程节点上使用不同版本的 ClickHouse 并且 GROUP BY 结果的大小很大并且由单个 String 字段执行聚合时, 就会发生这种不兼容. 结果导致单个键的多个未合并的行. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 避免通过 DistributedBlockOutputStream 发送部分写入的文件. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). -* 复 `SELECT count(notNullIn(NULL, []))` 中的崩溃. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在表引擎 = Kafka 的 DROP 期间 (或在服务器重新启动期间) 有时发生的挂起. [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). -* 现在可以像 `a TO b, c TO a` 那样执行多个 `ALTER RENAME`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). -* 修复当您从同一列的多个线程的聚合函数状态获得结果时可能发生的竞争. 唯一可能发生的方法 (我发现) 是当您使用 `finalizeAggregation` 函数同时使用 `Memory` 引擎从表中读取时, 该引擎为 `quanite*` 函数存储 `AggregateFunction` 状态. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复与分布式表中元组的向后兼容性. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 StringHashTable 中的 SIGSEGV (如果这样的键不存在). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* 修复了在使用 `Atomic` 引擎从数据库中删除 `LiveView` 表后, `WATCH` 挂起的问题. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `ReplicatedMergeTree` 中的错误, 该错误可能导致 `OPTIMIZE` 查询上的某些 `ALTER` 在它变为非活动状态后挂起等待某些副本. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* 现在,如果参与 `CONSTRAINT` 表达式的列被重命名, 则约束会更新. 修复 [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). -* 修复缓存字典中未初始化内存的潜在读取. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 Block::sortColumns() 之后修复列顺序 (还添加了一个测试, 表明它会影响一些实际用例 - 缓冲区引擎). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* 修复 ODBC 桥在不要求引用标识符时的问题. 这修复了 [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 DateLUT 中的 UBSan 和 MSan 报告. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在关键条件下使用 `src_type` 进行正确的类型转换. 修复 [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* 摆脱旧的 libunwind 补丁. https://github.com/ClickHouse-Extras/libunwind/commit/500aa227911bd185a94bfc071d68f4d3b03cb3b1#r39048012 这允许在 `clang` 中禁用 `-fno-omit-frame-pointer` 至少 1% 的平均构建性能. [#10761](https://github.com/ClickHouse/ClickHouse/pull/10761) ([Amos Bird](https://github.com/amosbird)). -* 修复了在多个分片上使用浮点权重时的avgWeighted问题. [#10758](https://github.com/ClickHouse/ClickHouse/pull/10758) ([Baudouin Giard](https://github.com/bgiard)). -* 修复 `parallel_view_processing` 行为. 现在, 如果发生异常, 所有对 `MATERIALIZED VIEW` 的插入都应该无一例外地完成. 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 与 -State 结合时修复组合器 -OrNull 和 -OrDefault. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* 使用嵌套类型修复 `generateRandom` 中的崩溃. 修复 [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复合并后可能发生的 `SummingMergeTree` 中 `LowCardinality(FixedString)` 键列的数据损坏. 修复 [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复使用 `FINAL` 修饰符和 `ORDER BY` 优化包装成函数的主键的使用. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). -* 修复函数 `h3EdgeAngle` 中可能的缓冲区溢出. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复消失的总数. 如果查询具有带有外部 where 条件的连接或子查询, 则总计可能已被过滤. 修复 [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 HTTP 插入的原子性. 这修复了 [#9666](https://github.com/ClickHouse/ClickHouse/issues/9666). [#10687](https://github.com/ClickHouse/ClickHouse/pull/10687) ([Andrew Onyshchuk](https://github.com/oandrew)). -* 在一个查询中使用相同的集合修复 `IN` 运算符的多次使用. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了当 `readonly=2` 和 `cancel_http_readonly_queries_on_client_close=1` 时导致 http 请求卡在客户端关闭的错误. 修复 [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* 修复 AggregateTransform 构造函数中参数的顺序. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* 修复启用了 `distributed_aggregation_memory_efficient` 的远程查询缺乏并行执行的问题。 修复 [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `LIMIT` 修复查询的可能不正确的行数. 修复[#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复当表有很多部分时锁定并发更改的错误. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* 如果服务器在表启动之前关闭,则修复 StorageBuffer 中的 nullptr 取消引用. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复分布式查询的谓词优化 (`enable_optimize_predicate_expression=1`) 对于带有 `HAVING` 部分的查询 (即需要在服务器启动器上进行过滤时), 通过保留表达式的顺序 (这足以修复), 并且还强制 聚合器在索引上使用列名. 修复: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). -* 使用 LowCardinality 修复 optimize_skip_unused_shards. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). -* 修复服务器启动异常时 StorageBuffer 中的段错误. 修复 [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550). [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)). -* 在 `SYSTEM DROP DNS CACHE` 查询中还会删除缓存, 用于检查是否允许用户从某些 IP 地址进行连接. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `MATERIALIZED VIEW` 内部查询中不正确的标量结果, 以防该查询包含相关表. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了同步突变的处理条件变量. 在某些情况下, 该条件变量的信号可能会丢失. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复了在 `loadStoredObject()` 完成之前调用 `createDictionary()` 可能导致的崩溃. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复错误 `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). -* 修复 SELECT 列 ALIAS 的默认表达式类型与列类型不同. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* 实现了 DateTime64 和 String 值之间的比较 (就像 DateTime 一样). [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* 修复索引损坏,在某些情况下将合并部分合并到另一个合并部分后可能会发生. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). -* 默认禁用GROUP BY sharding_key优化 (`optimize_distributed_group_by_sharding_key` 已被引入并默认关闭, 由于sharding_key分析的技巧, 简单的例子是分片键中的 `if` ) 并修复 WITH ROLLUP/CUBE/TOTALS. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). -* 修复: [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263) (之后通过 INSERT 发送的 PR dist 在每个 INSERT 上被推迟) 修复: [#8756](https://github.com/ClickHouse/ClickHouse/issues/8756) (that PR breaks distributed sends with all of the following conditions met (unlikely setup for now I guess): `internal_replication == false`, multiple local shards (activates the hardlinking code) and `distributed_storage_policy` (makes `link(2)` fails on `EXDEV`)). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `max_rows_to_sort` 限制的错误. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 每次调用读取外部字典的任何函数时, 仅获取一次字典并检查访问权限. [#10928](https://github.com/ClickHouse/ClickHouse/pull/10928) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### 改进 - -* 在 `ALTER MODIFY TTL` 查询之后, 对旧数据应用 `TTL` . 这个行为是通过设置 `materialize_ttl_after_modify` 来控制的, 默认是启用的. [#11042](https://github.com/ClickHouse/ClickHouse/pull/11042) ([Anton Popov](https://github.com/CurtizJ)). -* 在解析字符串文字、VALUES 和各种文本格式中的 C 样式反斜杠转义时 (这是对 ClickHouse 和 MySQL 特有的 SQL 标准的扩展), 如果发现未知的转义序列 (例如`\%` 或 `\ w`), 这将使 `LIKE` 和`match` 正则表达式的使用更加方便 (写` name LIKE 'used\_cars'` 而不是 `name LIKE 'used\\_cars'` 就足够了) 同时更兼容. 这修复了 [#10922](https://github.com/ClickHouse/ClickHouse/issues/10922). [#11208](https://github.com/ClickHouse/ClickHouse/pull/11208) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 读取 Decimal 值时, 在点后切掉多余的数字. 此行为与 MySQL 和 PostgreSQL 更兼容. 这修复了 [#10202](https://github.com/ClickHouse/ClickHouse/issues/10202). [#11831](https://github.com/ClickHouse/ClickHouse/pull/11831) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果 ZooKeeper 中的元数据已经被删除并且不存在, 则允许 DROP 复制表 (这也是使用 TestKeeper 进行测试并重新启动服务器时的情况). 即使与 ZooKeeper 通信出现错误, 也允许重命名复制表. 这修复了 [#10720](https://github.com/ClickHouse/ClickHouse/issues/10720). [#11652](https://github.com/ClickHouse/ClickHouse/pull/11652) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 稍微改进从字符串中读取十进制的诊断. 这将关闭 [#10202](https://github.com/ClickHouse/ClickHouse/issues/10202). [#11829](https://github.com/ClickHouse/ClickHouse/pull/11829) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复信号处理程序中的睡眠调用. 它的睡眠时间比预期的要少. [#11825](https://github.com/ClickHouse/ClickHouse/pull/11825) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* (仅限 Linux) 操作系统相关的性能指标 (针对 CPU 和 I/O) 即使没有 `CAP_NET_ADMIN` 功能也能正常工作. [#10544](https://github.com/ClickHouse/ClickHouse/pull/10544) ([Alexander Kazakov](https://github.com/Akazz)). -* 添加了 `hostname` 作为函数 `hostName` 的别名. 此功能由 Yandex.Metrica 的 Victor Tarnavskiy 提出. [#11821](https://github.com/ClickHouse/ClickHouse/pull/11821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了对跨复制集群上分布式 `DDL` (更新/删除/删除分区) 的支持. [#11703](https://github.com/ClickHouse/ClickHouse/pull/11703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 如果我们无法侦听侦听地址之一 (例如,在 Docker 中 IPv6 不可用), 则在启动时在服务器日志中发出警告而不是错误. 请注意,如果服务器未能侦听所有列出的地址, 它将像以前一样拒绝启动. 这修复了 [#4406](https://github.com/ClickHouse/ClickHouse/issues/4406). [#11687](https://github.com/ClickHouse/ClickHouse/pull/11687) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 docker 映像启动时创建默认用户和数据库. [#10637](https://github.com/ClickHouse/ClickHouse/pull/10637) ([Paramtamtam](https://github.com/tarampampam)). -* 当多行查询打印到服务器日志时, 这些行被连接起来. 使其在多行字符串文字、标识符和单行注释的情况下正常工作. 这修复了 [#3853](https://github.com/ClickHouse/ClickHouse/issues/3853). [#11686](https://github.com/ClickHouse/ClickHouse/pull/11686) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在允许在命令中使用多个名称: CREATE USER、CREATE ROLE、ALTER USER、SHOW CREATE USER、SHOW GRANTS 等. [#11670](https://github.com/ClickHouse/ClickHouse/pull/11670) ([Vitaly Baranov](https://github.com/vitlibar)). -* 在跨复制集群上添加对分布式 DDL (`UPDATE/DELETE/DROP PARTITION`) 的支持. [#11508](https://github.com/ClickHouse/ClickHouse/pull/11508) ([frank lee](https://github.com/etah000)). -* 如果用户已使用显式值指定密码, 则从命令行中的 `clickhouse-client` 和 `clickhouse-benchmark` 清除密码. 这可以防止 `ps` 和类似工具暴露密码. [#11665](https://github.com/ClickHouse/ClickHouse/pull/11665) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果 ELF 文件与正在运行的二进制文件不对应, 请不要使用调试信息. 需要避免在堆栈跟踪中打印错误的函数名称和源位置. 这修复了 [#7514](https://github.com/ClickHouse/ClickHouse/issues/7514). [#11657](https://github.com/ClickHouse/ClickHouse/pull/11657) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当在 parseDateTimeBestEffortOrNull/Zero 函数中没有完全解析值时, 返回 NULL/零. 这修复了 [#7876](https://github.com/ClickHouse/ClickHouse/issues/7876). [#11653](https://github.com/ClickHouse/ClickHouse/pull/11653) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 跳过请求的 URL 中的空参数. 当你写 `http://localhost:8123/?&a=b` 或 `http://localhost:8123/?a=b&&c=d` 时, 它们可能会出现. 这将关闭 [#10749](https://github.com/ClickHouse/ClickHouse/issues/10749). [#11651](https://github.com/ClickHouse/ClickHouse/pull/11651) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许使用 `groupArrayArray` 和 `groupUniqArrayArray` 作为 `SimpleAggregateFunction`. [#11650](https://github.com/ClickHouse/ClickHouse/pull/11650) ([Volodymyr Kuznetsov](https://github.com/ksvladimir)). -* 在分析其他类型的索引条件时, 允许通过隐式转换与常量字符串进行比较. 这可能会关闭 [#11630](https://github.com/ClickHouse/ClickHouse/issues/11630). [#11648](https://github.com/ClickHouse/ClickHouse/pull/11648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* https://github.com/ClickHouse/ClickHouse/pull/7572#issuecomment-642815377 支持配置默认 HTTPHandlers. [#11628](https://github.com/ClickHouse/ClickHouse/pull/11628) ([Winter Zhang](https://github.com/zhang2014)). -* 制作更多输入格式以与 Kafka 引擎配合使用. 解决过早冲洗的问题. 修复 `kafka_num_consumers` 大于主题分区数时的性能问题. [#11599](https://github.com/ClickHouse/ClickHouse/pull/11599) ([filimonov](https://github.com/filimonov)). -* 改进 `multiple_joins_rewriter_version=2` 逻辑. 修复 lambda 别名的未知列错误. [#11587](https://github.com/ClickHouse/ClickHouse/pull/11587) ([Artem Zuikov](https://github.com/4ertus2)). -* 无法解析列声明列表时更好的异常消息. 这将关闭 [#10403](https://github.com/ClickHouse/ClickHouse/issues/10403). [#11537](https://github.com/ClickHouse/ClickHouse/pull/11537) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 改进 VIEW 的 `enable_optimize_predicate_expression=1` 逻辑. [#11513](https://github.com/ClickHouse/ClickHouse/pull/11513) ([Artem Zuikov](https://github.com/4ertus2)). -* 在实时视图表中添加对 PREWHERE 的支持. [#11495](https://github.com/ClickHouse/ClickHouse/pull/11495) ([vzakaznikov](https://github.com/vzakaznikov)). -* 自动更新 DNS 缓存, 用于检查是否允许用户从某个地址连接. [#11487](https://github.com/ClickHouse/ClickHouse/pull/11487) ([tavplubix](https://github.com/tavplubix)). -* 即使执行并发合并, OPTIMIZE FINAL 也会强制合并. 这将关闭 [#11309](https://github.com/ClickHouse/ClickHouse/issues/11309) 并关闭 [#11322](https://github.com/ClickHouse/ClickHouse/issues/11322). [#11346](https://github.com/ClickHouse/ClickHouse/pull/11346) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 抑制 clickhouse-client 中取消查询的输出. 在以前的版本中, 即使按 Ctrl+C 取消查询, 结果也可能会继续在终端中打印. 这将关闭 [#9473](https://github.com/ClickHouse/ClickHouse/issues/9473). [#11342](https://github.com/ClickHouse/ClickHouse/pull/11342) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在历史文件会在每次查询后更新, 如果多个客户端使用一个历史文件, 则不会出现竞争条件. 这修复了 [#9897](https://github.com/ClickHouse/ClickHouse/issues/9897). [#11453](https://github.com/ClickHouse/ClickHouse/pull/11453) ([Tagir Kuskarov](https://github.com/kuskarov)). -* 重新加载配置时更好地记录消息. [#11341](https://github.com/ClickHouse/ClickHouse/pull/11341) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在某些情况下,从 `clickhouse-client` 或 `clickhouse-format` 中的格式化查询中删除尾随空格. [#11325](https://github.com/ClickHouse/ClickHouse/pull/11325) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加设置 `output_format_pretty_max_value_width` . 如果 value 较长, 则将其剪切以避免在终端中输出过大的 value. 这将关闭. [#11324](https://github.com/ClickHouse/ClickHouse/pull/11324) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在内存映射不足的情况下更好的异常消息. 这将关闭 [#11027](https://github.com/ClickHouse/ClickHouse/issues/11027). [#11316](https://github.com/ClickHouse/ClickHouse/pull/11316) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 支持 (U)Int8、(U)Int16、ASOF JOIN 中的日期. [#11301](https://github.com/ClickHouse/ClickHouse/pull/11301) ([Artem Zuikov](https://github.com/4ertus2)). -* 支持 Kafka 表的 kafka_client_id 参数. 它还将 ClickHouse 在与 Kafka 通信时使用的默认 `client.id` 更改为更加详细和可用. [#11252](https://github.com/ClickHouse/ClickHouse/pull/11252) ([filimonov](https://github.com/filimonov)). -* 将 `DistributedFilesToInsert` 指标的值保留在异常上. 在以前的版本中, 该值是在我们要发送一些文件时设置的, 但如果出现异常并且一些文件仍处于待处理状态, 则该值为零. 现在它对应于文件系统中待处理文件的数量. [#11220](https://github.com/ClickHouse/ClickHouse/pull/11220) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加对多字数据类型名称的支持 (例如 `DOUBLE PRECISION` and `CHAR VARYING` ) 以获得更好的 SQL 兼容性. [#11214](https://github.com/ClickHouse/ClickHouse/pull/11214) ([Павел Потемкин](https://github.com/Potya)). -* 为某些数据类型提供同义词. [#10856](https://github.com/ClickHouse/ClickHouse/pull/10856) ([Павел Потемкин](https://github.com/Potya)). -* 现在默认启用查询日志. [#11184](https://github.com/ClickHouse/ClickHouse/pull/11184) ([Ivan Blinkov](https://github.com/blinkov)). -* 在表 system.users 和执行 SHOW CREATE USER 查询时显示身份验证类型. [#11080](https://github.com/ClickHouse/ClickHouse/pull/11080) ([Vitaly Baranov](https://github.com/vitlibar)). -* 删除 `Memory` 数据库引擎的显式 `DROP DATABASE` 上的数据. 修复 [#10557](https://github.com/ClickHouse/ClickHouse/issues/10557). [#11021](https://github.com/ClickHouse/ClickHouse/pull/11021) ([tavplubix](https://github.com/tavplubix)). -* 为 rdkafka 库的内部线程设置线程名称. 使来自 rdkafka 的日志在服务器日志中可用. [#10983](https://github.com/ClickHouse/ClickHouse/pull/10983) ([Azat Khuzhin](https://github.com/azat)). -* 支持查询中的 unicode 空格. 这有助于从 Word 或网页复制粘贴查询. 这修复了 [#10896](https://github.com/ClickHouse/ClickHouse/issues/10896). [#10903](https://github.com/ClickHouse/ClickHouse/pull/10903) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 允许大型 UInt 类型作为函数 `tupleElement` 中的索引. [#10874](https://github.com/ClickHouse/ClickHouse/pull/10874) ([hcz](https://github.com/hczhcz)). -* 尊重在 INSERT 到分布式上的prefer_localhost_replica/load_balancing. [#10867](https://github.com/ClickHouse/ClickHouse/pull/10867) ([Azat Khuzhin](https://github.com/azat)). -* 介绍 `min_insert_block_size_rows_for_materialized_views`、`min_insert_block_size_bytes_for_materialized_views` 设置. 此设置类似于 `min_insert_block_size_rows` 和 `min_insert_block_size_bytes`, 但仅适用于插入到 `MATERIALIZED VIEW` 中的块. 它有助于在推送到 MV 时控制块挤压并避免过多的内存使用. [#10858](https://github.com/ClickHouse/ClickHouse/pull/10858) ([Azat Khuzhin](https://github.com/azat)). -* 在服务器关闭期间摆脱复制队列中的异常. 修复 [#10819](https://github.com/ClickHouse/ClickHouse/issues/10819). [#10841](https://github.com/ClickHouse/ClickHouse/pull/10841) ([alesapin](https://github.com/alesapin)). -* 确保 `varSamp`、`varPop` 不能因数值错误返回负结果, 并且不能从负方差计算 `stddevSamp`、`stddevPop`. 这修复了 [#10532](https://github.com/ClickHouse/ClickHouse/issues/10532). [#10829](https://github.com/ClickHouse/ClickHouse/pull/10829) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 更好的 DNS 异常消息. 这修复了 [#10813](https://github.com/ClickHouse/ClickHouse/issues/10813). [#10828](https://github.com/ClickHouse/ClickHouse/pull/10828) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 将某些解析错误的 HTTP 响应代码更改为 400 Bad Request. 这个修复 [#10636](https://github.com/ClickHouse/ClickHouse/issues/10636). [#10640](https://github.com/ClickHouse/ClickHouse/pull/10640) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果 clickhouse-client 比 clickhouse-server 新, 则打印一条消息. [#10627](https://github.com/ClickHouse/ClickHouse/pull/10627) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加对 `INSERT INTO [db.]table WATCH` 查询的支持. [#10498](https://github.com/ClickHouse/ClickHouse/pull/10498) ([vzakaznikov](https://github.com/vzakaznikov)). -* 允许在 clickhouse-client 中传递 quota_key. 这将关闭 [#10227](https://github.com/ClickHouse/ClickHouse/issues/10227). [#10270](https://github.com/ClickHouse/ClickHouse/pull/10270) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### 性能改进 - -* 允许多个副本同时分配合并、突变、分区删除、移动和替换. 这将关闭 [#10367](https://github.com/ClickHouse/ClickHouse/issues/10367). [#11639](https://github.com/ClickHouse/ClickHouse/pull/11639) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#11795](https://github.com/ClickHouse/ClickHouse/pull/11795) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 关于表排序键的 GROUP BY 优化, 通过 `optimize_aggregation_in_order` 设置启用. [#9113](https://github.com/ClickHouse/ClickHouse/pull/9113) ([dimarub2000](https://github.com/dimarub2000)). -* 带有 final 的选择是并行执行的. 添加设置 `max_final_threads` 以限制使用的线程数. [#10463](https://github.com/ClickHouse/ClickHouse/pull/10463) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当生成小块时, 通过 `INSERT SELECT` 或 INSERT with clickhouse-client 提高 INSERT 查询的性能 (并行解析的典型情况). 这修复了 [#11275](https://github.com/ClickHouse/ClickHouse/issues/11275). 修复 CONSTRAINT 不适用于 DEFAULT 字段的问题. 这修复了 [#11273](https://github.com/ClickHouse/ClickHouse/issues/11273). 修复 TEMPORARY 表忽略 CONSTRAINTS 的问题. 这修复了 [#11274](https://github.com/ClickHouse/ClickHouse/issues/11274). [#11276](https://github.com/ClickHouse/ClickHouse/pull/11276) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 消除 SELECT 部分中 GROUP BY 键的最小/最大/任何聚合器的优化, 通过 `optimize_aggregators_of_group_by_keys` 设置启用. [#11667](https://github.com/ClickHouse/ClickHouse/pull/11667) ([xPoSx](https://github.com/xPoSx)). [#11806](https://github.com/ClickHouse/ClickHouse/pull/11806) ([Azat Khuzhin](https://github.com/azat)). -* 使用 `optimize_move_functions_out_of_any` 启用的新优化, 可从 `any` 函数中删除所有操作 [#11529](https://github.com/ClickHouse/ClickHouse/pull/11529) ([Ruslan](https://github.com/kamalov-ruslan)). -* 当使用 Pretty 格式时, 提高交互模式下 `clickhouse-client` 的性能. 在以前的版本中, 计算 UTF-8 字符串的可见宽度可能会花费大量时间. 这将关闭 [#11323](https://github.com/ClickHouse/ClickHouse/issues/11323). [#11323](https://github.com/ClickHouse/ClickHouse/pull/11323) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用 `ORDER BY` 和小的 `LIMIT` 提高查询的性能 (小于, 然后是 `max_block_size`). [#11171](https://github.com/ClickHouse/ClickHouse/pull/11171) ([Albert Kidrachev](https://github.com/Provet)). -* 添加运行时 CPU 检测以选择和调度最佳功能实现. 添加对多个目标的代码生成的支持. 这将关闭 [#1017](https://github.com/ClickHouse/ClickHouse/issues/1017). [#10058](https://github.com/ClickHouse/ClickHouse/pull/10058) ([DimasKovas](https://github.com/DimasKovas)). -* 默认启用 clickhouse 二进制文件的 `mlock`. 它将防止 clickhouse 可执行文件在高 IO 负载下被分页. [#11139](https://github.com/ClickHouse/ClickHouse/pull/11139) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用 `sum` 聚合函数进行查询而不使用 GROUP BY 键以更快地运行多次. [#10992](https://github.com/ClickHouse/ClickHouse/pull/10992) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过删除一些冗余数据移动来改进基数排序 (在带有简单键的 `ORDER BY` 中使用). [#10981](https://github.com/ClickHouse/ClickHouse/pull/10981) ([Arslan Gumerov](https://github.com/g-arslan)). -* 在 MergeJoin 中对左表的较大部分进行排序. 缓冲内存中的剩余块. 添加 `partial_merge_join_left_table_buffer_bytes` 设置来管理左侧块缓冲区大小. [#10601](https://github.com/ClickHouse/ClickHouse/pull/10601) ([Artem Zuikov](https://github.com/4ertus2)). -* 从子查询中删除重复的 ORDER BY 和 DISTINCT, 此优化通过 `optimize_duplicate_order_by_and_distinct` 启用 [#10067](https://github.com/ClickHouse/ClickHouse/pull/10067) ([Mikhail Malafeev](https://github.com/demo-99)). -* 此功能消除了 GROUP BY 部分中其他键的功能, 通过 `optimize_group_by_function_keys` 启用 [#10051](https://github.com/ClickHouse/ClickHouse/pull/10051) ([xPoSx](https://github.com/xPoSx)). -* 使用 `optimize_arithmetic_operations_in_aggregate_functions` 启用从聚合函数中提取算术运算的新优化 [#10047](https://github.com/ClickHouse/ClickHouse/pull/10047) ([Ruslan](https://github.com/kamalov-ruslan)). -* 使用基于 Poco 而不是 curl 的 S3 的 HTTP 客户端. 这将提高性能并降低 s3 存储和表函数的内存使用量. [#11230](https://github.com/ClickHouse/ClickHouse/pull/11230) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复与基于限制重新安排相关的 Kafka 性能问题, 这些限制总是被应用. [#11149](https://github.com/ClickHouse/ClickHouse/pull/11149) ([filimonov](https://github.com/filimonov)). -* 为 jemalloc 启用 percpu_arena:percpu (这将减少由于线程池造成的内存碎片). [#11084](https://github.com/ClickHouse/ClickHouse/pull/11084) ([Azat Khuzhin](https://github.com/azat)). -* 从 S3 HTTP 客户端读取响应时优化内存使用. [#11561](https://github.com/ClickHouse/ClickHouse/pull/11561) ([Pavel Kovalenko](https://github.com/Jokser)). -* 调整默认的 Kafka 设置以获得更好的性能. [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388) ([filimonov](https://github.com/filimonov)). - -#### 实验功能 - -* 添加数据类型 `Point` (Tuple(Float64, Float64)) 和 `Polygon` (Array(Array(Tuple(Float64, Float64))). [#10678](https://github.com/ClickHouse/ClickHouse/pull/10678) ([Alexey Ilyukhov](https://github.com/livace)). -* 添加一个 `hasSubstr` 函数, 允许在数组中查找子序列. 注: 此功能可能会更名, 恕不另行通知. [#11071](https://github.com/ClickHouse/ClickHouse/pull/11071) ([Ryad Zenine](https://github.com/r-zenine)). -* 添加了 OpenCL 支持和双音排序算法, 可用于对单列中的整数类型数据进行排序. 需要使用标志 `-DENABLE_OPENCL=1` 构建. 要使用双音排序算法而不是其他算法, 您需要为设置选项 `special_sort` 设置 `bitonic_sort`, 并确保 OpenCL 可用. 此功能不会提高性能或其他任何东西, 仅作为示例和演示目的提供. 如果在这个方向上没有进一步的发展, 它可能会在不久的将来被移除. [#10232](https://github.com/ClickHouse/ClickHouse/pull/10232) ([Ri](https://github.com/margaritiko)). - -#### 构建/测试/打包改进 - -* 为程序和实用程序启用 clang-tidy. [#10991](https://github.com/ClickHouse/ClickHouse/pull/10991) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 删除对 `tzdata` 的依赖: 如果 `/usr/share/zoneinfo` 目录不存在, 则不会失败. 请注意, 即使系统中没有安装 tzdata,所有时区都可以在 ClickHouse 中使用. [#11827](https://github.com/ClickHouse/ClickHouse/pull/11827) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了 MSan 和 UBSan 压力测试. 请注意, 我们已经有了用于功能测试的 MSan、UBSan, 而 "stress" 测试是另一种测试. [#10871](https://github.com/ClickHouse/ClickHouse/pull/10871) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在崩溃消息中打印编译器构建 ID. 这将使我们更加确定哪个二进制文件崩溃了. 添加了新函数 `buildId`. [#11824](https://github.com/ClickHouse/ClickHouse/pull/11824) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了一个测试以确保在 FREEZE 查询后突变继续工作. [#11820](https://github.com/ClickHouse/ClickHouse/pull/11820) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 不允许在名称中包含 "fail" 子字符串的测试, 因为当您键入 Ctrl+F 并搜索 "fail" 时, 这会使在浏览器中查看测试结果变得不那么方便. [#11817](https://github.com/ClickHouse/ClickHouse/pull/11817) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 从 HTTPHandlerFactory 中删除未使用的导入. [#11660](https://github.com/ClickHouse/ClickHouse/pull/11660) ([Bharat Nallan](https://github.com/bharatnc)). -* 添加了对执行复制程序的实例的随机抽样. 需要避免 `Too many simultaneous queries` 错误. 还增加了超时时间并降低了故障概率. [#11573](https://github.com/ClickHouse/ClickHouse/pull/11573) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复遗漏的 include. [#11525](https://github.com/ClickHouse/ClickHouse/pull/11525) ([Matwey V. Kornilov](https://github.com/matwey)). -* 通过删除旧的示例程序来加速构建。 还发现了一些孤儿功能测试. [#11486](https://github.com/ClickHouse/ClickHouse/pull/11486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 增加 CI 中构建的 ccache 大小. [#11450](https://github.com/ClickHouse/ClickHouse/pull/11450) ([alesapin](https://github.com/alesapin)). -* 在 deb 构建中只保留 unit_tests_dbms. [#11429](https://github.com/ClickHouse/ClickHouse/pull/11429) ([Ilya Yatsishin](https://github.com/qoega)). -* 将 librdkafka 更新到版本 [1.4.2](https://github.com/edenhill/librdkafka/releases/tag/v1.4.2). [#11256](https://github.com/ClickHouse/ClickHouse/pull/11256) ([filimonov](https://github.com/filimonov)). -* 重构 CMake 构建文件. [#11390](https://github.com/ClickHouse/ClickHouse/pull/11390) ([Ivan](https://github.com/abyss7)). -* 修复几个不稳定的集成测试. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). -* 添加对使用 UBSan 运行的单元测试的支持. [#11345](https://github.com/ClickHouse/ClickHouse/pull/11345) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 从集成测试 `test_insertion_sync_fails_with_timeout` 中删除冗余超时. [#11343](https://github.com/ClickHouse/ClickHouse/pull/11343) ([alesapin](https://github.com/alesapin)). -* 更好地检查 clickhouse-test 中的挂起查询. [#11321](https://github.com/ClickHouse/ClickHouse/pull/11321) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果服务器是在调试中构建或使用消毒剂构建的, 则发出警告. [#11304](https://github.com/ClickHouse/ClickHouse/pull/11304) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在 clickhouse-test 在测试运行之前检查服务器的活动性. [#11285](https://github.com/ClickHouse/ClickHouse/pull/11285) ([alesapin](https://github.com/alesapin)). -* 修复可能不稳定的测试 `00731_long_merge_tree_select_opened_files.sh`. 它不会经常失败, 但我们在试验 ThreadFuzzer 时发现了这个测试中潜在的竞争条件: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) 见 [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果 `curl` 调用超时, 则在 CI 中重复测试. 由于系统挂断超过 10 秒, 这在我们的 CI 基础设施中很常见. 这修复了 [#11267](https://github.com/ClickHouse/ClickHouse/issues/11267). [#11268](https://github.com/ClickHouse/ClickHouse/pull/11268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 从@donmikel 添加对连接表引擎的测试. 这将关闭 [#9158](https://github.com/ClickHouse/ClickHouse/issues/9158). [#11265](https://github.com/ClickHouse/ClickHouse/pull/11265) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复单元测试中的几个非重大错误. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). -* 现在, `cctz` 库的部分链接器命令将不会与其他库混在一起. [#11213](https://github.com/ClickHouse/ClickHouse/pull/11213) ([alesapin](https://github.com/alesapin)). -* 将 /programs/server 拆分为实际的程序和库. [#11186](https://github.com/ClickHouse/ClickHouse/pull/11186) ([Ivan](https://github.com/abyss7)). -* 改进 protobuf 和 gRPC 的构建脚本. [#11172](https://github.com/ClickHouse/ClickHouse/pull/11172) ([Vitaly Baranov](https://github.com/vitlibar)). -* 启用不起作用的性能测试. [#11158](https://github.com/ClickHouse/ClickHouse/pull/11158) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在启动任何 CH 实例之前为测试创建根 S3 存储桶. [#11142](https://github.com/ClickHouse/ClickHouse/pull/11142) ([Pavel Kovalenko](https://github.com/Jokser)). -* 添加非恒定多边形的性能测试. [#11141](https://github.com/ClickHouse/ClickHouse/pull/11141) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `00979_live_view_watch_continuous_aggregates` 测试. [#11024](https://github.com/ClickHouse/ClickHouse/pull/11024) ([vzakaznikov](https://github.com/vzakaznikov)). -* 添加在 tmpfs 上的集成测试中运行 zookeeper 的功能. [#11002](https://github.com/ClickHouse/ClickHouse/pull/11002) ([alesapin](https://github.com/alesapin)). -* 等待具有指数退避的 odbc-bridge. 在我们的 CI 环境中, 之前 200 毫秒的等待时间是不够的. [#10990](https://github.com/ClickHouse/ClickHouse/pull/10990) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复非确定性测试. [#10989](https://github.com/ClickHouse/ClickHouse/pull/10989) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了对空外部数据的测试. [#10926](https://github.com/ClickHouse/ClickHouse/pull/10926) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 每次测试都会重新创建数据库. 这改进了测试的分离. [#10902](https://github.com/ClickHouse/ClickHouse/pull/10902) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在列代码中添加了更多断言. [#10833](https://github.com/ClickHouse/ClickHouse/pull/10833) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 与清除系统更好的合作. 在清除失败的消息中打印有关 query_id 的信息. [#10832](https://github.com/ClickHouse/ClickHouse/pull/10832) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 "Split build smoke test" 检查中修复明显的竞争条件. [#10820](https://github.com/ClickHouse/ClickHouse/pull/10820) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 MergeTreeIndexFullText 中修复 (false) MSan 报告. 该问题首次出现在 [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968) 中. [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 为 MariaDB 客户端库添加 MSan 抑制. [#10800](https://github.com/ClickHouse/ClickHouse/pull/10800) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* GRPC make 找不到 protobuf 文件, 通过添加正确的链接来更改 make 文件. [#10794](https://github.com/ClickHouse/ClickHouse/pull/10794) ([mnkonkova](https://github.com/mnkonkova)). -* 为 base、utils、programs 启用额外警告 (`-Weverything`). 请注意, 我们已经为大部分代码提供了它. [#10779](https://github.com/ClickHouse/ClickHouse/pull/10779) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在 [#10396](https://github.com/ClickHouse/ClickHouse/issues/10396) 中, 对图书馆警告的抑制被错误地声明为公开. [#10776](https://github.com/ClickHouse/ClickHouse/pull/10776) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 恢复在 [#10396](https://github.com/ClickHouse/ClickHouse/issues/10396) 中意外删除的补丁. [#10774](https://github.com/ClickHouse/ClickHouse/pull/10774) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复性能测试错误, 第 2 部分. [#10773](https://github.com/ClickHouse/ClickHouse/pull/10773) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复性能测试错误. [#10766](https://github.com/ClickHouse/ClickHouse/pull/10766) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 更新交叉构建以使用 clang-10 编译器. [#10724](https://github.com/ClickHouse/ClickHouse/pull/10724) ([Ivan](https://github.com/abyss7)). -* 更新安装 RPM 包的说明. 这是由 Denis (TG login @ldviolet) 建议并由 Arkady Shejn 实施的. [#10707](https://github.com/ClickHouse/ClickHouse/pull/10707) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 试图修复 `tests/queries/0_stateless/01246_insert_into_watch_live_view.py` 测试. [#10670](https://github.com/ClickHouse/ClickHouse/pull/10670) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修复和重新启用 00979_live_view_watch_continuous_aggregates.py 测试. [#10658](https://github.com/ClickHouse/ClickHouse/pull/10658) ([vzakaznikov](https://github.com/vzakaznikov)). -* 修复 ASan 压力测试中的 OOM. [#10646](https://github.com/ClickHouse/ClickHouse/pull/10646) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复迁移到 clang-10 后出现的 HashTable 中的 UBSan 报告 (向 nullptr 添加零). [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在编译时的 tzdata 处理期间删除对 `ld` (bfd) 链接器的外部调用. [#10634](https://github.com/ClickHouse/ClickHouse/pull/10634) ([alesapin](https://github.com/alesapin)). -* 允许使用 `lld` 来链接 blob (资源). [#10632](https://github.com/ClickHouse/ClickHouse/pull/10632) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `LZ4` 库中的UBSan报告. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). See also [https://github.com/lz4/lz4/issues/857](https://github.com/lz4/lz4/issues/857) -* 将 LZ4 更新到最新的 dev 分支. [#10630](https://github.com/ClickHouse/ClickHouse/pull/10630) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了自动生成的机器可读文件与稳定版本的列表. [#10628](https://github.com/ClickHouse/ClickHouse/pull/10628) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `capnp::UnalignedFlatArrayMessageReader` 的 `capnproto` 版本检查. [#10618](https://github.com/ClickHouse/ClickHouse/pull/10618) ([Matwey V. Kornilov](https://github.com/matwey)). -* 测试中的内存使用率较低. [#10617](https://github.com/ClickHouse/ClickHouse/pull/10617) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在新的实时视图测试中修复硬编码超时. [#10604](https://github.com/ClickHouse/ClickHouse/pull/10604) ([vzakaznikov](https://github.com/vzakaznikov)). -* 在 tests/queries/0_stateless/helpers/client.py 中打开客户端时增加超时. [#10599](https://github.com/ClickHouse/ClickHouse/pull/10599) ([vzakaznikov](https://github.com/vzakaznikov)). -* 为 clang 构建启用 ThinLTO, 继续 [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435). [#10585](https://github.com/ClickHouse/ClickHouse/pull/10585) ([Amos Bird](https://github.com/amosbird)). -* 添加模糊器并准备 oss-fuzz 集成. [#10546](https://github.com/ClickHouse/ClickHouse/pull/10546) ([kyprizel](https://github.com/kyprizel)). -* 修复 FreeBSD 构建. [#10150](https://github.com/ClickHouse/ClickHouse/pull/10150) ([Ivan](https://github.com/abyss7)). -* 使用 pytest 框架为查询测试添加新版本. [#10039](https://github.com/ClickHouse/ClickHouse/pull/10039) ([Ivan](https://github.com/abyss7)). - - -## ClickHouse 版本 v20.4 - -### ClickHouse 版本 v20.4.8.99-stable 2020-08-10 - -#### Bug 修复 - -* 修复了当 unix 时间戳作为参数传递时 `parseDateTimeBestEffort` 函数中的错误. 这修复了 [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在具有 NaN 值的 Float 类型上调用的 `uniqExact`、`topK`、`sumDistinct` 和类似聚合函数的潜在低性能和稍微不正确的结果. 它还在调试版本中触发断言. 这修复了 [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果将可空 constexpr 作为不是文字 NULL 的 cond 的固定函数. 修复 [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在数组元素为 Nullable 且数组下标也可为 Null 的情况下, 修复了 `arrayElement` 函数中的断言. 这修复了 [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 用函数修正了错误的索引分析. 在从 `MergeTree` 表中读取时, 它可能会导致修剪错误的部分. 修复 [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了对从本地副本进行选择的线程数的不必要限制. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了查询 `WITH TOTALS` 时可能出现的数据中可能出现的额外溢出行. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了大元组的性能, 这些元组在 `IN` 部分被解释为函数. 当用户出于某种晦涩的原因编写 `WHERE x IN tuple(1, 2, ...)` 而不是 `WHERE x IN (1, 2, ...)` 时的情况. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了`input_format_parallel_parsing` 的内存跟踪(通过将线程附加到组). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) 当子查询包含 with 子句时允许推送谓词. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) 使用 const 表达式修复布隆过滤器索引. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了当 Broker 不可用时 (不仅如此) 在 `StorageKafka` 中的 `SIGSEGV`. [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* 添加了对带有 `Array(UUID)` 参数的函数 `if` 的支持. 这修复了 [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了带有缓存布局的外部字典中的竞争条件, 这可能导致服务器崩溃. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* 删除了 DROP TABLE 上分布式表 (来自异步插入的块) 的数据. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当 `enable_mixed_granularity_parts=1` 时, 在 `ALTER DELETE` 查询后导致旧部件损坏的错误。 修复 [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* 参数数目无效的函数 `in` 的更好例外. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了性能问题, 同时读取紧凑部件. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了当我们通过字典键的表达式加入时, JOIN 与字典中的崩溃: `t JOIN dict ON expr(dict.id) = t.id`. 在这种情况下禁用字典连接优化. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了 StorageMerge 时可能出现的段错误. 关闭 [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `WITH FILL` 修饰符中列的顺序. 以前不遵守 `ORDER BY` 语句的列顺序. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). -* 当存在按虚拟列 (如 `Merge` 表中的`_table`) 或系统表中的 "index" 列过滤数据 (例如从 `system.tables` 查询时按数据库名称过滤) 时, 避免 "bad cast" 异常, 并且这个表达式返回 `Nullable` 类型. 这修复了 [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* TrieDictionary 加载失败后显示错误. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). -* 对于可能导致崩溃的空数组, 函数 `arrayFill` 工作不正确. 这修复了 [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 实现了到 `LowCardinality` 类型的通用类型的转换. 这允许对具有 LowCardinality 列和其他列的表执行 UNION ALL. 这修复了[#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). 这修复了 [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在某些特殊类型的 `StorageFile` 标头中多次连续插入期间多次写入时的行为. 这个修复了 [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了 UInt8 值不等于 0 或 1 时的逻辑函数. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). -* 上限 max_memory_usage* 对进程驻留内存的限制. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). -* 在 GROUP BY 内射函数消除期间修复了 `dictGet` 参数检查. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). -* 如果 ODBC 连接不支持架构,则不要将字典源的表名拆分为架构和表名本身. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了 `ALTER DELETE` 中的错误逻辑, 当条件评估为 NULL 时会导致删除记录. 这修复了 [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在存在别名的情况下发送到外部 DBMS (例如 MySQL、ODBC) 的查询转换. 这修复了[#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了整数除法中的潜在溢出. 这修复了 [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `greatCircleDistance`、`geoDistance` 中潜在的无限循环. 这修复了 [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 规范化 `pid` 文件处理. 在以前的版本中, 如果服务器在没有正确关闭的情况下被杀死, 并且有另一个进程与以前运行的服务器具有相同的 pid, 则服务器可能会拒绝启动. 即使有另一台服务器正在运行, pid 文件也可能在服务器启动失败时被删除. 这修复了 [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了对字典的 ENGINE=Dictionary 表的处理依赖性. 这修复了[#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了由于线程总数限制错误导致的带有 `UNION` 的选择的性能. 修复 [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `-StateResample` 组合器修复了段错误. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 `system.quey_log` 中选择的空 `result_rows` 和 `result_bytes` 指标. 修复 [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了从 `VIEW` 中选择的不必要的线程数限制. 修复 [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了使用错误类型 `PREWHERE` 时可能发生的崩溃. 修复 [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了具有 `LowCardinality` 类型的函数 `defaultValueOfArgumentType` 的 `Expected single dictionary argument for function` 错误. 修复 [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了带有 `Tuple(LowCardinality)` 参数的高阶函数的错误 `Cannot capture column`. 修复 [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 加载数据库时并行解析表元数据. 这修复了有大量表时服务器启动缓慢的问题. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). -* 使 `topK` 聚合函数为 Enum 类型返回 Enum. 这修复了 [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了是否为常量表达式的约束检查. 这修复了 [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了元组与 `Nullable` 列的不正确比较. 修复 [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 allow_introspection_functions=0 时访问权限的计算. [#12031](https://github.com/ClickHouse/ClickHouse/pull/12031) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了在调用具有不同大小的 `FixedString` 类型参数的函数 `if` 时出现错误结果和潜在崩溃的问题. 这修复了 [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果使用偏移量 `-9223372036854775808` 调用函数, 则以函数 `neighbor` 作为唯一返回表达式的查询可能会返回空结果. 这修复了 [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 allow_ddl=0 时访问权限的计算. [#12015](https://github.com/ClickHouse/ClickHouse/pull/12015) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了 generateRandom 中可能导致崩溃的潜在数组大小溢出. 这修复了 [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了潜在的浮点异常. 这将关闭 [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了服务器启动时日志消息中错误的设置名称. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `Values` 格式中的 `Query parameter was not set` 问题. 修复 [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). -* 在查询 (参数化查询) 中保留替换的别名. 这修复了 [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了从默认存储策略更改存储策略时没有移动的错误. [#11893](https://github.com/ClickHouse/ClickHouse/pull/11893) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复了解析 `DateTime64` 时潜在的浮点异常. 这修复了 [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过 HTTP 接口固定内存记帐 (使用 `wait_end_of_query=1` 可能很重要). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). -* 在检查相等性之前解析存储在 zookeeper 中的元数据. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). - -#### 性能改进 - -* 索引不用于带文字的 IN 运算符, 在 v19.3 前后引入了性能回归. 这修复了 [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). - -#### 构建/测试/打包改进 - -* 在 Dockerfile 中的第一个 `apt-get update` 之前安装 `ca-certificates`. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). - - -### ClickHouse 版本 v20.4.6.53-stable 2020-06-25 - -#### Bug 修复 - -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. 续 [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 不允许在高阶函数中使用 arrayJoin. 它导致协议同步中断. 这将关闭 [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 FixedString 与常量 String 比较的错误结果. 这修复了. [#11828](https://github.com/ClickHouse/ClickHouse/pull/11828) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复条件为 NULL 的 `if()` 的错误结果. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复使用过多线程进行查询. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复像 `SELECT *, xyz.*` 这样的查询的意外行为, 这些查询在预期错误时成功. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* 现在复制的提取将在元数据更改期间取消. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). -* 修复了由 Values 输入格式中复杂文字的错误类型推导导致的 LOGICAL_ERROR. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* 修复 在 const 列上的 `ORDER BY ... WITH FILL` . [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* 与 XDBC 桥通信时传递适当的超时. 最近在检查桥接活性和接收元信息时没有遵守超时. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 将 `LIMIT n WITH TIES` 与包含别名的 `ORDER BY` 语句一起使用. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). -* 修复导致 `system.mutations` 状态不正确的错误. 它可能表明整个突变已经完成, 但服务器在复制队列中仍有 `MUTATE_PART` 任务并尝试执行它们. 这修复了 [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). -* 添加对带有不区分大小写标志的正则表达式的支持. 这修复了 [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果设置了行级安全性, 则删除琐碎的计数查询优化. 在以前的版本中, 用户获取表中记录的总数而不是过滤. 这修复了[#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复字符串的布隆过滤器 (数据跳过索引). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. (可能它以某种方式与 [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) 相关联). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复错误 `Block structure mismatch` , 用于从 `Buffer` 表中采样读取的查询. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 clickhouse-client 的错误退出代码, 当 exception.code() % 256 = 0 时. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* 修复服务器启动时有关 `Mark cache size was lowered` 的日志消息中的小错误. 这将关闭 [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复错误 `Size of offsets does not match size of column` , 用于查询中的 `PREWHERE column in (subquery)` 和 `ARRAY JOIN` . [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 `SHOW CREATE TABLE` 中罕见的段错误. 修复 [#11490](https://github.com/ClickHouse/ClickHouse/issues/11490). [#11579](https://github.com/ClickHouse/ClickHouse/pull/11579) ([tavplubix](https://github.com/tavplubix)). -* HTTP 会话中的所有查询都具有相同的 query_id. 它是固定的. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). -* 现在 clickhouse-server docker 容器将更喜欢 IPv6 检查服务器活动性. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* 修复 `` 的 shard_num/replica_num (破坏 use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). -* 修复可能导致表删除期间异常的竞争条件. 这有点棘手, 一点也不危险. 如果你想要解释, 请在电报中通知我. [#11523](https://github.com/ClickHouse/ClickHouse/pull/11523) ([alesapin](https://github.com/alesapin)). -* 使用 -State 函数在聚合过程中抛出异常时修复内存泄漏. 这修复了 [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果数据跳过索引依赖于将在后台合并期间修改的列 (对于 SummingMergeTree、AggregatingMergeTree 以及 TTL GROUP BY) , 则计算错误. 此问题已通过合并后移动索引计算得到修复, 因此索引是在合并数据上计算的. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* 摆脱旧的 libunwind 补丁. https://github.com/ClickHouse-Extras/libunwind/commit/500aa227911bd185a94bfc071d68f4d3b03cb3b1#r39048012 这允许在 `clang` 中禁用 `-fno-omit-frame-pointer` 至少 1% 的平均构建性能. [#10761](https://github.com/ClickHouse/ClickHouse/pull/10761) ([Amos Bird](https://github.com/amosbird)). -* 使用 `FINAL` 修饰符和 `ORDER BY` 优化来修复包装到函数中的主键的使用. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). - -#### 构建/测试/打包改进 - -* 修复单元测试中的几个非重大错误. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). -* 在 MergeTreeIndexFullText 中修复 (false) MSan 报告. 该问题首先出现在 [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse 版本 v20.4.5.36-stable 2020-06-10 - -#### Bug 修复 - -* 修复了在启用 `min_bytes_to_use_direct_io` 且 PREWHERE 处于活动状态并使用 SAMPLE 或大量线程时可能发生的 `Data compressed with different methods` 错误. 这修复了 [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复编解码器的返回压缩大小. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当列具有带有非文字参数的压缩编解码器时修复服务器崩溃. 修复 [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* 以 nan 为点修复 pointInPolygon. 修复 [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* 如果表未成功创建, 则修复 MergeTree 关闭时潜在的未初始化内存读取. [#11420](https://github.com/ClickHouse/ClickHouse/pull/11420) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了带有纬度/经度范围之外的参数的 geohashesInBox. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* 修复具有外部排序和限制的查询可能出现的 `Pipeline stuck` 错误. 修复 [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 在 ReplicatedMergeTree 中发送零件期间删除冗余锁. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). -* 修复了多行模式下 clickhouse-client 对 `\G`(垂直输出)的支持. 这将关闭 [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复使用 `Lazy` 数据库时潜在的段错误. [#11348](https://github.com/ClickHouse/ClickHouse/pull/11348) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `quantilesExactWeightedArray` 中的崩溃问题. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 现在合并在 `ALTER` 查询中更改元数据之前停止. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). -* 通过再次设置 `parallel_view_processing = 1` 并行写入 `MATERIALIZED VIEW` . 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当提取的 JSON 具有不平衡的 { 或 [ 字符串时, 修复visitParamExtractRaw. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* 修复 ThreadPool 中非常罕见的竞争条件. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 clickhouse-copier 中无关紧要的数据竞争. 通过集成测试发现. [#11313](https://github.com/ClickHouse/ClickHouse/pull/11313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复转换中潜在的未初始化内存. 示例: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了如果表的主键中有 Array 列并且查询正在使用 `empty` 或 `notEmpty` 函数按此列过滤时索引分析无法工作的问题. 这修复了 [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复当查询速度估计可能不正确时的错误, 并且如果查询受到 `max_network_bandwidth`、`max_execution_speed` 或 `priority` 设置的限制, `min_execution_speed` 的限制可能无法工作或无法正常工作. 将 `timeout_before_checking_execution_speed` 的默认值更改为非零值, 否则设置 `min_execution_speed` 和 `max_execution_speed` 无效. 这修复了 [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). 这修复了 [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732)。 这修复了 [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). 可用性改进:避免在“clickhouse-client”中将异常消息与进度条串联. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复使用错误参数调用 SET DEFAULT ROLE 时的崩溃. 这修复了[#10586](https://github.com/ClickHouse/ClickHouse/issues/10586). [#11278](https://github.com/ClickHouse/ClickHouse/pull/11278) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复读取 Protobuf 格式错误数据时崩溃的问题. 这修复了 [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了缓存字典可以返回默认值而不是正常值 (只有过期键时) 的错误. 这仅影响字符串字段. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复错误 `Block structure mismatch in QueryPipeline`, 同时从内部查询中的常量读取 `VIEW` . 修复 [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复可能的异常 `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `Array(Array(LowCardinality))` 捕获参数修复高阶函数的可能错误 `Cannot capture column` . [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 S3 globbing 在超过 1000 个密钥和一些后端的情况下可能会失败的问题. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 如果数据跳过索引依赖于将在后台合并期间修改的列 (对于 SummingMergeTree、AggregatingMergeTree 以及 TTL GROUP BY), 则计算错误. 此问题已通过合并后移动索引计算得到修复, 因此索引是在合并数据上计算的. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* 修复与基于限制重新安排相关的 Kafka 性能问题, 这些限制总是被应用. [#11149](https://github.com/ClickHouse/ClickHouse/pull/11149) ([filimonov](https://github.com/filimonov)). -* 修复了在表引擎 = Kafka 的 DROP 期间 (或在服务器重新启动期间) 有时发生的挂起. [#11145](https://github.com/ClickHouse/ClickHouse/pull/11145) ([filimonov](https://github.com/filimonov)). -* 修复简单查询过多的线程预留 (优化减少线程数量, 在管道更改后部分中断). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). -* 修复分布式查询的谓词优化 (`enable_optimize_predicate_expression=1`) 对于带有 `HAVING` 部分的查询 (即需要在服务器启动器上进行过滤时), 通过保留表达式的顺序 (这足以修复), 并且还强制 聚合器在索引上使用列名. 修复: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). - -#### 构建/测试/打包改进 - -* 修复几个不稳定的集成测试. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). - -### ClickHouse 版本 v20.4.4.18-stable 2020-05-26 - -与 v20.4.3.16 稳定版相比没有变化. - -### ClickHouse 版本 v20.4.3.16-stable 2020-05-23 - -#### Bug 修复 - -* 如果没有最终确定, 则从突变终止任务中删除日志记录. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* 修复了 registerDiskS3 中的内存泄漏. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). -* 修复 Kafka 引擎表终止时可能丢失的数据. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). -* 复了 `parseDateTime64BestEffort` 参数解析错误. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* 如果表未成功创建, 则修复了 `MergeTree` 中非常罕见的潜在释放后使用错误. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986), [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了原子数据库的元数据 (重命名的相对路径) 和数据 (符号链接的相对路径) 处理. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). -* 修复了使用 `Atomic` 数据库引擎并发 `ALTER` 和 `DROP DATABASE` 查询时服务器崩溃的问题. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `getRawData()` 方法中不正确的原始数据大小. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* 修复了 20.1 及更早版本之间两级聚合的不兼容问题. 当在发起节点和远程节点上使用不同版本的 ClickHouse 并且 GROUP BY 结果的大小很大并且由单个 String 字段执行聚合时, 就会发生这种不兼容. 结果导致单个键的多个未合并的行. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了通过 `DistributedBlockOutputStream` 发送部分写入的文件. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `SELECT count(notNullIn(NULL, []))` 中的崩溃. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在 `Kafka` 表引擎的 `DROP` 期间有时发生的挂起. (或在服务器重新启动期间). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). -* 修复了无法像 `a TO b, c TO a` 那样执行多个 `ALTER RENAME` 的问题. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). -* 修复了当您从同一列的多个线程的聚合函数状态获得结果时可能发生的竞争. 它可能发生的唯一方法是当您使用 `finalizeAggregation` 函数同时使用 `Memory` 引擎从表中读取时, 该引擎为 `quantile*` 函数存储 `AggregateFunction` 状态. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了与分布式表中元组的向后兼容性. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* 如果这样的键不存在, 则修复了 `StringHashTable` 中的 `SIGSEGV`. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* 修复了在使用 `Atomic` 引擎从数据库中删除 `LiveView` 表后, `WATCH` 挂起的问题. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `ReplicatedMergeTree` 中的错误, 该错误可能导致 `OPTIMIZE` 查询上的某些 `ALTER` 在它变为非活动状态后挂起等待某些副本. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* 现在, 如果参与 `CONSTRAINT` 表达式的列被重命名, 则约束会更新. 修复 [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). -* 修复了缓存字典中未初始化内存的潜在读取. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `Block::sortColumns()` 之后的列顺序. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当不要求引用标识符时 `ODBC` 桥的问题. 修复 [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `DateLUT` 中的 `UBSan` 和 `MSan` 报告. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了关键条件下错误的类型转换. 修复 [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* 修复了 `parallel_view_processing` 行为. 现在, 如果发生异常, 所有对 `MATERIALIZED VIEW` 的插入都应该无一例外地完成. 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 固定组合器 `-OrNull` 和 `-OrDefault` 与 `-State` 结合使用. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* 修复了函数 `h3EdgeAngle` 中可能出现的缓冲区溢出. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了当表有很多部分时锁定并发更改的错误. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* 如果服务器在表启动前关闭, 则修复了 `StorageBuffer` 中的 `nullptr` 取消引用. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 使用 `LowCardinality` 修复了 `optimize_skip_unused_shards` . [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). -* 修复了同步突变的处理条件变量. 在某些情况下, 该条件变量的信号可能会丢失. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 修复了在 `loadStoredObject()` 完成之前调用 `createDictionary()` 时可能发生的崩溃. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了列 `ALIAS` 的 `SELECT`, 它的默认表达式类型与列类型不同. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* 实现了 DateTime64 和 String 值之间的比较. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* 默认禁用 `GROUP BY` sharding_key优化 ( `optimize_distributed_group_by_sharding_key` 已经被引入并默认关闭, 由于 sharding_key 分析的技巧, 简单的例子是 sharding key 中的 `if`) 并修复它为 `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). -* 已修复 [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). -* 添加了关于 `max_rows_to_sort` 设置的测试. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了创建布隆过滤器索引的向后兼容性. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). - -### ClickHouse 版本 v20.4.2.9, 2020-05-12 - -#### 向后不兼容变更 -* 系统表 (例如 system.query_log、system.trace_log、system.metric_log) 对小于 10 MiB 的部分使用合并数据部分格式. 自 20.3 版起支持压缩数据部分格式. 如果你打算降级到20.3以下的版本, 你应该手动删除 `/var/lib/clickhouse/data/system/` 中系统日志的表数据. -* 当字符串比较涉及 FixedString 并且比较的参数大小不同时, 将较小的字符串填充到较大的字符串的长度进行比较. 如果我们想象 FixedString 数据类型对应于 SQL CHAR,这是为了 SQL 兼容性. 这将关闭 [#9272](https://github.com/ClickHouse/ClickHouse/issues/9272). [#10363](https://github.com/ClickHouse/ClickHouse/pull/10363) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使 SHOW CREATE TABLE 多行. 现在它更具可读性, 更像 MySQL. [#10049](https://github.com/ClickHouse/ClickHouse/pull/10049) ([Azat Khuzhin](https://github.com/azat)) -* 添加了在 `pointInPolygon` 函数中使用并默认启用的设置 `validate_polygons`. [#9857](https://github.com/ClickHouse/ClickHouse/pull/9857) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 新特征 -* 添加对从 ClickHouse 到 Zookeeper 的安全连接的支持 [#10184](https://github.com/ClickHouse/ClickHouse/pull/10184) ([Konstantin Lebedev](https://github.com/xzkostyan)) -* 支持自定义 HTTP 处理程序。 参见 [#5436](https://github.com/ClickHouse/ClickHouse/issues/5436) 的描述. [#7572](https://github.com/ClickHouse/ClickHouse/pull/7572) ([Winter Zhang](https://github.com/zhang2014)) -* 添加 MessagePack 输入/输出格式. [#9889](https://github.com/ClickHouse/ClickHouse/pull/9889) ([Kruglov Pavel](https://github.com/Avogar)) -* 添加正则表达式输入格式. [#9196](https://github.com/ClickHouse/ClickHouse/pull/9196) ([Kruglov Pavel](https://github.com/Avogar)) -* 添加了输出格式 `Markdown`, 用于在 Markdown 文档中嵌入表格. [#10317](https://github.com/ClickHouse/ClickHouse/pull/10317) ([Kruglov Pavel](https://github.com/Avogar)) -* 添加了对字典中自定义设置部分的支持. 还修复了问题 [#2829](https://github.com/ClickHouse/ClickHouse/issues/2829). [#10137](https://github.com/ClickHouse/ClickHouse/pull/10137) ([Artem Streltsov](https://github.com/kekekekule)) -* 在 DDL 查询中为 `CREATE DICTIONARY` 添加了自定义设置支持. [#10465](https://github.com/ClickHouse/ClickHouse/pull/10465) ([Artem Streltsov](https://github.com/kekekekule)) -* 添加简单的服务器范围内存分析器, 当服务器内存使用率高于下一个分配阈值时, 它将收集分配上下文. [#10444](https://github.com/ClickHouse/ClickHouse/pull/10444) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加设置 `always_fetch_merged_part` 限制副本自己合并部分, 并且总是喜欢从其他副本下载. [#10379](https://github.com/ClickHouse/ClickHouse/pull/10379) ([alesapin](https://github.com/alesapin)) -* 添加函数 `JSONExtractKeysAndValuesRaw`, 它从JSON对象中提取原始数据. [#10378](https://github.com/ClickHouse/ClickHouse/pull/10378) ([hcz](https://github.com/hczhcz)) -* 将操作系统的内存使用添加到 `system.asynchronous_metrics`. [#10361](https://github.com/ClickHouse/ClickHouse/pull/10361) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了函数 `least` 和 `greatest` 的通用变体. 现在它们可以处理任意数量的任意类型的参数. 这修复了 [#4767](https://github.com/ClickHouse/ClickHouse/issues/4767) [#10318](https://github.com/ClickHouse/ClickHouse/pull/10318) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 现在, ClickHouse 在其一侧控制字典源的超时. 缓存字典配置中添加了两个新设置: `strict_max_lifetime_seconds`,默认为 `max_lifetime`,以及 `query_wait_timeout_milliseconds`, 默认为一分钟. 第一个设置对于 `allow_read_expired_keys` 设置也很有用 (禁止读取非常过期的键). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 添加 log_queries_min_type 以过滤哪些条目将写入 query_log. [#10053](https://github.com/ClickHouse/ClickHouse/pull/10053) ([Azat Khuzhin](https://github.com/azat)) -* 添加函数 `isConstant`. 此函数检查其参数是否为常量表达式并返回 1 或 0. 用于开发、调试和演示目的. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加 joinGetOrNull 以在 key 丢失时返回 NULL 而不是返回默认值. [#10094](https://github.com/ClickHouse/ClickHouse/pull/10094) ([Amos Bird](https://github.com/amosbird)) -* 如果设置了选项 `transform_null_in`, 则认为 `NULL` 等于 `IN` 运算符中的 `NULL`. [#10085](https://github.com/ClickHouse/ClickHouse/pull/10085) ([achimbab](https://github.com/achimbab)) -* 为 MergeTree 表引擎系列添加 `ALTER TABLE ... RENAME COLUMN`. [#9948](https://github.com/ClickHouse/ClickHouse/pull/9948) ([alesapin](https://github.com/alesapin)) -* 支持并行分布式 INSERT SELECT. [#9759](https://github.com/ClickHouse/ClickHouse/pull/9759) ([vxider](https://github.com/Vxider)) -* 添加在分布式上查询分布式的能力 (不带 `distributed_group_by_no_merge`) ... [#9923](https://github.com/ClickHouse/ClickHouse/pull/9923) ([Azat Khuzhin](https://github.com/azat)) -* 添加函数 `arrayReduceInRanges` , 它聚合给定范围内的数组元素. [#9598](https://github.com/ClickHouse/ClickHouse/pull/9598) ([hcz](https://github.com/hczhcz)) -* 在 prometheus 导出器上添加字典状态. [#9622](https://github.com/ClickHouse/ClickHouse/pull/9622) ([Guillaume Tassery](https://github.com/YiuRULE)) -* 添加函数 `arrayAUC`. [#8698](https://github.com/ClickHouse/ClickHouse/pull/8698) ([taiyang-li](https://github.com/taiyang-li)) -* 支持 `DROP VIEW` 语句以获得更好的TPC-H兼容性. [#9831](https://github.com/ClickHouse/ClickHouse/pull/9831) ([Amos Bird](https://github.com/amosbird)) -* 将 `strict_order` 选项添加到 windowFunnel(). [#9773](https://github.com/ClickHouse/ClickHouse/pull/9773) ([achimbab](https://github.com/achimbab)) -* 支持 `DATE` 和 `TIMESTAMP` SQL 操作符, 例如 `选择日期'2001-01-01'`. [#9691](https://github.com/ClickHouse/ClickHouse/pull/9691) ([Artem Zuikov](https://github.com/4ertus2)) - -#### 实验功能 -* 添加了实验性数据库引擎 Atomic. 它支持非阻塞 `DROP` 和 `RENAME TABLE` 查询以及原子 `EXCHANGE TABLES t1 AND t2` 查询. [#7512](https://github.com/ClickHouse/ClickHouse/pull/7512) ([tavplubix](https://github.com/tavplubix)) -* 对 S3 上的 ReplicatedMergeTree 的初始支持 (它以次优方式工作). [#10126](https://github.com/ClickHouse/ClickHouse/pull/10126) ([Pavel Kovalenko](https://github.com/Jokser)) - -#### Bug 修复 -* 修复了 `MATERIALIZED VIEW` 内部查询中不正确的标量结果, 以防该查询包含相关表. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复了当 `readonly=2` 和 `cancel_http_readonly_queries_on_client_close=1` 时导致 HTTP 请求卡在客户端关闭连接上的错误. [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)) -* 在服务器启动时抛出异常时修复 StorageBuffer 中的段错误. 修复 [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550) [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)) -* 查询 `SYSTEM DROP DNS CACHE` 现在还删除用于检查是否允许用户从某些 IP 地址连接的缓存. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)) -* 修复在一个查询中使用多个具有相同集合的 `IN` 运算符. 修复 [#10539](https://github.com/ClickHouse/ClickHouse/issues/10539) [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)) -* 使用嵌套类型修复 `generateRandom` 中的崩溃. 修复 [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复合并后可能发生的 `SummingMergeTree` 中 `LowCardinality(FixedString)` 键列的数据损坏. 修复 [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复 aggregation_memory_efficient_merge_threads 设置的逻辑. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)) -* 修复消失的总数. 如果查询具有 `JOIN` 或带有外部 `WHERE` 条件的子查询, 则总计可能被过滤。 修复[#10674](https://github.com/ClickHouse/ClickHouse/issues/10674) [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复启用了 `distributed_aggregation_memory_efficient` 的远程查询缺乏并行执行的问题. 修复 [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655) [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 使用 `LIMIT` 修复查询的可能不正确的行数. 修复 [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709) [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复索引损坏, 在某些情况下将合并部分合并到另一个合并部分后可能会发生这种情况. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)) -* 修复当mutation完成所有部分, 但在 `is_done=0` 中挂掉的情况. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)) -* 修复 unix 纪元开始时与 UTC 有小数偏移的时区的溢出. 修复 [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更好的输入格式诊断. 修复 [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204) [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)) -* 修复 `simpleLinearRegression()` 中大整数上的数字溢出. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)) -* 修复分布式关闭中的use-after-free, 避免等待发送所有批次 [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)) -* 将 CA 证书添加到 clickhouse-server docker 镜像. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)) -* 修复使用 `addressToLine` 函数或 AggregateFunctionState 列时可能发生的罕见无限循环. [#10466](https://github.com/ClickHouse/ClickHouse/pull/10466) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 在分布式查询期间处理zookeeper `no node error`. [#10050](https://github.com/ClickHouse/ClickHouse/pull/10050) ([Daniel Chen](https://github.com/Phantomape)) -* 修复更改列的默认值后服务器无法附加表的错误. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) -* 将默认表达式类型隐式转换为 ALIAS 列的列类型. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)) -* 如果 `ATTACH DATABASE` 失败, 不要删除元数据目录. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) -* 避免依赖系统 tzdata. 修复了在 CentOS 8 上加载 `Africa/Casablanca` 时区的问题. [#10211](https://github.com/ClickHouse/ClickHouse/issues/10211) [#10425](https://github.com/ClickHouse/ClickHouse/pull/10425) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 如果数据以仲裁插入然后被删除 (DROP PARTITION、TTL 等), 则修复一些问题. 它导致在 SELECT 中卡住 INSERT 或误报异常. 修复 [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 创建 BloomFilter 索引时检查参数的数量和类型 [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623) [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)) -* 更喜欢 `fallback_to_stale_replicas` 而不是 `skip_unavailable_shards` , 否则当两个设置都指定并且没有最新的副本时, 查询将失败 (来自@alex-zaitsev的补丁 ). [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) -* 修复带有 ARRAY JOIN、ORDER BY 和 LIMIT 的查询可能返回不完整结果的问题. 修复 [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([Vadim Plakhtinskiy](https://github.com/VadimPlh)) -* 在 DETACH/ATTACH 之后将数据库名称添加到字典名称. 修复 system.dictionaries 表和 `SYSTEM RELOAD` 查询. [#10415](https://github.com/ClickHouse/ClickHouse/pull/10415) ([Azat Khuzhin](https://github.com/azat)) -* 修复处理器管道中极端情况可能导致的错误结果. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复启用设置 `distributed_group_by_no_merge` 时可能出现的段错误 (在 20.3.7.46 中由 [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131) 引入). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复 `Array(Tuple(...))` 数据类型的错误扁平化. 修复 [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 JOIN 内的常量列名可能与 JOIN 外的常量名冲突. [#9950](https://github.com/ClickHouse/ClickHouse/pull/9950) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 在 Block::sortColumns() 之后修复列的顺序. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)) -* 修复 `ConcatProcessor` 中可能发生的 `Pipeline stuck` 错误,该错误可能在远程查询中发生. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 不要为聚合预留磁盘. 修复 [#9241](https://github.com/ClickHouse/ClickHouse/issues/9241) [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) -* 修复了时区的日期时间函数的错误行为, 这些时区在 UTC 的正负偏移量之间发生了变化 (例如 Pacific/Kiritimati). 修复 [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 避免在 `dictIsIn` 函数中出现无限循环. 修复 #515. [#10365](https://github.com/ClickHouse/ClickHouse/pull/10365) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 默认禁用 GROUP BY sharding_key 优化并修复 WITH ROLLUP/CUBE/TOTALS. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)) -* 检查零件时检查错误代码, 如果错误类似于 `not enough memory`, 请不要将零件标记为损坏. 修复 [#6269](https://github.com/ClickHouse/ClickHouse/issues/6269) [#10364](https://github.com/ClickHouse/ClickHouse/pull/10364) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在系统表中显示有关未加载词典的信息. [#10234](https://github.com/ClickHouse/ClickHouse/pull/10234) ([Vitaly Baranov](https://github.com/vitlibar)) -* 如果服务器在表启动前关闭, 则修复 StorageBuffer 中的 nullptr 取消引用. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了 `ReplicatedMergeTree` 中的 `DROP` 与 `OPTIMIZE` 竞赛. 如果存在并发的 OPTIMIZE 查询, 则 DROP 可能会在 ZooKeeper 的副本路径中留下一些垃圾. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) -* 修复 `Logical error: CROSS JOIN has expressions` 错误, 用于使用逗号和名称连接混合的查询. 修复 [#9910](https://github.com/ClickHouse/ClickHouse/issues/9910) [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)) -* 使用 `max_bytes_before_external_group_by` 修复查询. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复在某些情况下限制解析器中最大递归深度的问题. 此修复 [#10283](https://github.com/ClickHouse/ClickHouse/issues/10283) 此修复可能会引入轻微的不兼容性: 通过 clickhouse-client 进行长时间和深度查询可能会拒绝工作, 您应该调整设置 `max_query_size ` 和 `max_parser_depth` 相应. [#10295](https://github.com/ClickHouse/ClickHouse/pull/10295) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 允许对多个 JOIN 使用 `count(*)`. 修复 [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) -* 使用 `max_rows_to_group_by` 和 `group_by_overflow_mode = 'break'` 修复错误 `Pipeline stuck`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复使用 DDL 查询创建 `range_hashed` 字典时出现的 `Cannot add column` 错误. 修复 [#10093](https://github.com/ClickHouse/ClickHouse/issues/10093). [#10235](https://github.com/ClickHouse/ClickHouse/pull/10235) ([alesapin](https://github.com/alesapin)) -* 修复罕见的可能异常 `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复了 ClickHouse 会抛出 `Unknown function lambda` 的错误. 当用户尝试在 ENGINE = Replicated* 的表上运行 ALTER UPDATE/DELETE 时出现错误消息. 检查非确定性函数现在可以正确处理 lambda 表达式. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)) -* 修复了当 SELECT ... FROM system.tables 在具有惰性引擎的数据库上运行时发生的 StorageSystemTables 中相当罕见的段错误. [#10209](https://github.com/ClickHouse/ClickHouse/pull/10209) ([Alexander Kazakov](https://github.com/Akazz)) -* 当查询实际上应该在 LIMIT 停止时, 修复可能的无限查询执行, 同时从无限源 (如 `system.numbers` 或 `system.zeros` )读取. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复了 Date 类型的 `generateRandom` 函数. 这修复了 [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). 修复了将 2106 年的日期插入到使用旧式分区的 MergeTree 表但分区以 1970 年命名的边缘情况. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 如果 View 的表定义与 SELECT 查询不对应, 则转换类型. 这修复了 [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022) [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 RFC-2822 中字符串的 `parseDateTimeBestEffort`, 当一周中的某一天是星期二或星期四时. 这修复了 [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082) [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 JOIN 内的常量列名可能与 JOIN 外的常量名冲突. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在存在数组连接函数的情况下修复移动到前置位置优化 (在某些情况下). 这修复了[#10092](https://github.com/ClickHouse/ClickHouse/issues/10092) [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复本机 mysql-connector-java (JDBC) 的 SCRAMBLE 中出现分隔符的问题. [#10140](https://github.com/ClickHouse/ClickHouse/pull/10140) ([BohuTANG](https://github.com/BohuTANG)) -* 修复未指定数据库时使用当前数据库进行访问检查的问题. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)) -* 用合并部分修复表的 ALTER. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)) -* 添加了通过 `allow_nondeterministic_mutations` 设置放宽对突变中非确定性函数使用限制的能力. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)) -* 修复字典调用 `DROP TABLE` 的问题. [#10165](https://github.com/ClickHouse/ClickHouse/pull/10165) ([Azat Khuzhin](https://github.com/azat)) -* 如果在将 `INSERT` 插入分布式表时结构不匹配, 则转换块. [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)) -* 当插入的块被带有分区键的部分分割时,记录的行数不正确 (作为所有部分的总和). [#10138](https://github.com/ClickHouse/ClickHouse/pull/10138) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为 MySQL 数据库引擎添加一些参数检查和支持标识符参数. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)) -* 在创建新副本时修复不正确的 `index_granularity_bytes` 检查. 修复 [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)) -* 当表包含跳过索引时, 修复 `CHECK TABLE` 查询中的错误. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)) -* 修复嵌套表中只有一个分片的 Distributed-over-Distributed 问题. [#9997](https://github.com/ClickHouse/ClickHouse/pull/9997) ([Azat Khuzhin](https://github.com/azat)) -* 使用 `JOIN` 和 `UNION ALL` 修复查询可能丢失的行. 修复 [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). ... [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复使用本地 clickhouse 服务器作为源时字典中的错误. 如果字典和源中的类型不兼容, 可能会导致内存损坏. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)) -* 修复了从不存在 `/table/replicas/replica_name/metadata` 节点的旧 ClickHouse 版本更新时复制表启动的问题. 修复 [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)) -* 修复错误 `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform` . 当设置 `distributed_aggregation_memory_efficient` 被启用, 并且分布式查询从不同分片读取混合单级和二级聚合的聚合数据时发生这种情况. [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复具有实体化视图的数据库在启动时附加失败时的死锁. [#10054](https://github.com/ClickHouse/ClickHouse/pull/10054) ([Azat Khuzhin](https://github.com/azat)) -* 修复 GROUP BY 中可能发生的分段错误,该错误是包含尾随零字节的字符串键 ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). ... [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 当别名可以覆盖限定的列名时, 修复分布式查询的错误结果. 修复 [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714) [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复 `SYSTEM RESTART REPLICAS` 中可能出现的死锁. [#9955](https://github.com/ClickHouse/ClickHouse/pull/9955) ([tavplubix](https://github.com/tavplubix)) -* 修复用于远程查询执行的线程数 (性能回归, 自 20.3 起). 当来自 `Distributed` 表的查询在本地和远程分片上同时执行时, 就会发生这种情况. 修复 [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965) [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复了 `ATTACH PART` 中的 `DeleteOnDestroy` 逻辑, 这可能导致自动删除附加部分并添加了一些测试. [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复 `ON CLUSTER` DDL 查询在服务器启动时冻结的错误. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)) -* 修复了在某些数据库查询的处理阶段之一未检索到必要表的错误. 修复 [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)) -* 修复 `JOIN` 与 `TOTALS` 一起出现时 Not found column in block 错误. 修复 [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839) [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复解析 CREATE USER 命令中设置的多个主机. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)) -* 修复连接表引擎的 `TRUNCATE`. ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)) -* 修复 `ReplicatedMergeTree` 中删除和优化之间的竞争条件. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)) -* 当设置了 `optimize_skip_unused_shards` 时, 修复分布式的 `DISTINCT`. [#9808](https://github.com/ClickHouse/ClickHouse/pull/9808) ([Azat Khuzhin](https://github.com/azat)) -* 修复 ALTER 中的 `scalar does not exist` 错误 ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). ... [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)) -* 修复 `distributed_product_mode=\'local\'` 中的限定名称错误. 修复 [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756) [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)) -* 对于 INSERT 查询, 分片现在确实将启动器的设置限制到它们的约束, 而不是抛出异常. 此修复允许将 INSERT 查询发送到具有其他约束的分片. 此更改改进了修复 [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)) -* 在向 Kafka 代理提交偏移量时添加一些重试, 因为如果在 `offsets.commit.timeout.ms` 期间没有足够的副本可用于 `__consumer_offsets` 主题, 它可以拒绝提交. [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)) -* 修复在 `WHERE` 中使用基础表的虚拟列时的分布式引擎行为. [#9847](https://github.com/ClickHouse/ClickHouse/pull/9847) ([Azat Khuzhin](https://github.com/azat)) -* 修复了函数参数的时区使用不正确的一些情况. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)) -* 当查询在分布式表上具有 PREWHERE 和 WHERE 并且 `SET分布式产品模式 = 'local'` 时, 修复 `Different expressions with the same alias` 错误. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复具有复合主键的表的突变过多的内存消耗. 这修复了 [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)) -* 修复了从设置 `allow_introspection_functions` 计算内省函数的授权. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)) -* 修复 max_distributed_connections (带和不带处理器) [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)) -* 修复客户端上可能出现的异常 `Got 0 in totals chunk, expected 1` . 如果右连接表有零行, 则使用 `JOIN` 进行查询时会发生这种情况. 示例: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. 修复 [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). ... [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复 `COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query` 错误, 以防在表列表中使用 COMMA JOIN 进行子查询 (即在 WHERE 中). 修复 [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782) [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复了设置 `optimize_skip_unused_shards` 时服务器崩溃并且键的表达式无法转换为其字段类型的问题. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)) -* 修复 `splitByString` 中的空字符串处理. [#9767](https://github.com/ClickHouse/ClickHouse/pull/9767) ([hcz](https://github.com/hczhcz)) -* 修复损坏的 `ALTER TABLE DELETE COLUMN` 查询合并部分. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)) -* 修复了通过 http (使用处理器管道) 查询时缺少的 `rows_before_limit_at_least`. 修复 [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730) [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复 `ALTER` 查询 (突变) 中的过多内存消耗. 这修复了 [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) -* 修复可能的永久性 `Cannot schedule a task` 错误. [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) -* 修复外部字典 DDL 中反引号的错误. 修复 [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) -* 修复了 `text_log` 中的数据竞争. 它不对应任何真正的错误. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复复制中的错误, 如果用户在以前的版本上执行了更改, 则不允许复制工作. 这修复了 [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)) -* 修复了 `sumKahan` 和 `sumWithOverflow` 不正确的内部函数名称. 在远程查询中使用此函数时导致异常. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)) -* 添加设置 `use_compact_format_in_distributed_parts_names`, 它允许将`INSERT` 查询的文件以更紧凑的格式写入 `Distributed` 表. 这修复了 [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)) -* 在 JOIN 键中使用低基数修复 RIGHT 和 FULL JOIN. [#9610](https://github.com/ClickHouse/ClickHouse/pull/9610) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复 `MergeTreeRangeReader` 中可能出现的异常 `Size of filter does not match size of column` and `Invalid number of rows in Chunk` 和 `Invalid number of rows in Chunk`. 在某些情况下, 它们可能会在执行 `PREWHERE` 时出现. [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) -* 允许具有内部复制的分布式表的 `ALTER ON CLUSTER`. 这修复了 [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268) [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)) -* 如果您编写像 `time + 1` 这样的简单算术表达式 (与像 `time + INTERVAL 1 SECOND` 这样的表达式相反), 则修复了未保留时区的问题. 这修复了 [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743) [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 改进 -* 将 DateTime 与字符串文字进行比较时使用时区. 这修复了 [#5206](https://github.com/ClickHouse/ClickHouse/issues/5206). [#10515](https://github.com/ClickHouse/ClickHouse/pull/10515) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 如果无法从文本输入格式解析 Decimal 值, 则打印详细的诊断信息. [#10205](https://github.com/ClickHouse/ClickHouse/pull/10205) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为分布式/缓冲调度池添加任务/内存指标. [#10449](https://github.com/ClickHouse/ClickHouse/pull/10449) ([Azat Khuzhin](https://github.com/azat)) -* 一旦准备好在 clickhouse-local 和 HTTP 接口中进行 SELECT DISTINCT 查询, 就立即显示结果. 这修复了 [#8951](https://github.com/ClickHouse/ClickHouse/issues/8951) [#9559](https://github.com/ClickHouse/ClickHouse/pull/9559) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 允许使用 `SAMPLE OFFSET` 查询代替 `cityHash64(PRIMARY KEY) % N == n` 在 `clickhouse-copier` 中进行拆分. 要使用此功能, 请将 `--experimental-use-sample-offset 1` 作为命令行参数传递. [#10414](https://github.com/ClickHouse/ClickHouse/pull/10414) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 如果第一列的值中不能包含 BOM, 则允许在 TSV 中解析 BOM. 这修复了 [#10301](https://github.com/ClickHouse/ClickHouse/issues/10301) [#10424](https://github.com/ClickHouse/ClickHouse/pull/10424) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加 Avro 嵌套字段插入支持. [#10354](https://github.com/ClickHouse/ClickHouse/pull/10354) ([Andrew Onyshchuk](https://github.com/oandrew)) -* 指定相同类型时, 允许在非修改数据模式下更改列. [#10382](https://github.com/ClickHouse/ClickHouse/pull/10382) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 在 GROUP BY 分片键上自动 `distributed_group_by_no_merge` (如果设置了 `optimize_skip_unused_shards`). [#10341](https://github.com/ClickHouse/ClickHouse/pull/10341) ([Azat Khuzhin](https://github.com/azat)) -* 使用 LIMIT/LIMIT BY/ORDER BY 优化查询以使用 GROUP BY sharding_key 进行分布式. [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)) -* 添加了设置 `max_server_memory_usage` 以限制服务器的总内存使用量. 现在计算指标 `MemoryTracking` 没有漂移. 设置 `max_memory_usage_for_all_queries` 现在已经过时并且什么都不做. 这将关闭 [#10293](https://github.com/ClickHouse/ClickHouse/issues/10293). [#10362](https://github.com/ClickHouse/ClickHouse/pull/10362) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加配置选项 `system_tables_lazy_load`. 如果设置为 false, 则在服务器启动时加载带有日志的系统表. [Alexander Burmak](https://github.com/Alex-Burmak), [Svyatoslav Tkhon Il Pak](https://github.com/DeifyTheGod), [#9642](https://github.com/ClickHouse/ClickHouse/pull/9642) [#10359](https://github.com/ClickHouse/ClickHouse/pull/10359) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使用后台线程池 (background_schedule_pool_size) 进行分布式发送. [#10263](https://github.com/ClickHouse/ClickHouse/pull/10263) ([Azat Khuzhin](https://github.com/azat)) -* 使用后台线程池进行后台缓冲区刷新. [#10315](https://github.com/ClickHouse/ClickHouse/pull/10315) ([Azat Khuzhin](https://github.com/azat)) -* 支持删除不完整写入部分的一种特殊情况. 这修复了[#9940](https://github.com/ClickHouse/ClickHouse/issues/9940). [#10221](https://github.com/ClickHouse/ClickHouse/pull/10221) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使用 isInjective() 在此类函数的手动列表上进行 GROUP BY 优化. [#10342](https://github.com/ClickHouse/ClickHouse/pull/10342) ([Azat Khuzhin](https://github.com/azat)) -* 如果客户端在连接时立即发送 RST 数据包, 请避免在日志中打印错误消息. 这是具有 keepalived 和 VRRP 的 IPVS 平衡器的典型行为. 这修复了 [#1851](https://github.com/ClickHouse/ClickHouse/issues/1851) [#10274](https://github.com/ClickHouse/ClickHouse/pull/10274) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 允许为浮点类型解析 `+inf`. 这将关闭 [#1839](https://github.com/ClickHouse/ClickHouse/issues/1839) [#10272](https://github.com/ClickHouse/ClickHouse/pull/10272) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为嵌套类型实现了 `generateRandom` 表函数. 这将关闭 [#9903](https://github.com/ClickHouse/ClickHouse/issues/9903) [#10219](https://github.com/ClickHouse/ClickHouse/pull/10219) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 MySQL 兼容接口中提供 `max_allowed_packed` 将帮助一些客户端通过 MySQL 协议与 ClickHouse 通信. [#10199](https://github.com/ClickHouse/ClickHouse/pull/10199) ([BohuTANG](https://github.com/BohuTANG)) -* 允许 GLOBAL IN 的文字 (即`SELECT * FROM remote('localhost', system.one) WHERE dummy global in (0)`). [#10196](https://github.com/ClickHouse/ClickHouse/pull/10196) ([Azat Khuzhin](https://github.com/azat)) -* 修复 clickhouse-client 交互模式下的各种小问题. [#10194](https://github.com/ClickHouse/ClickHouse/pull/10194) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 避免多余的字典加载 (system.tables, DROP/SHOW CREATE TABLE). [#10164](https://github.com/ClickHouse/ClickHouse/pull/10164) ([Azat Khuzhin](https://github.com/azat)) -* 更新到 RWLock:getLock() + 实现的超时参数重新设计为阶段公平. [#10073](https://github.com/ClickHouse/ClickHouse/pull/10073) ([Alexander Kazakov](https://github.com/Akazz)) -* 增强与本机 mysql-connector-java(JDBC) 的兼容性. [#10021](https://github.com/ClickHouse/ClickHouse/pull/10021) ([BohuTANG](https://github.com/BohuTANG)) -* `toString` 函数被认为是单调的, 即使在使用 String 或 LowCardinality(String) 参数的重言式情况下也可以用于索引分析. [#10110](https://github.com/ClickHouse/ClickHouse/pull/10110) ([Amos Bird](https://github.com/amosbird)) -* 向命令 `{CREATE|DROP} USER/ROLE/ROW POLICY/SETTINGS PROFILE/QUOTA`、`GRANT` 添加 `ON CLUSTER` 子句支持. [#9811](https://github.com/ClickHouse/ClickHouse/pull/9811) ([Vitaly Baranov](https://github.com/vitlibar)) -* 对 S3 URI 的虚拟托管式支持. [#9998](https://github.com/ClickHouse/ClickHouse/pull/9998) ([Pavel Kovalenko](https://github.com/Jokser)) -* 现在可以在字典 DDL 查询中不用圆括号指定没有参数的字典的布局类型. 修复 [#10057](https://github.com/ClickHouse/ClickHouse/issues/10057). [#10064](https://github.com/ClickHouse/ClickHouse/pull/10064) ([alesapin](https://github.com/alesapin)) -* 添加在文件路径中使用带前导零的数字范围的功能. [#9989](https://github.com/ClickHouse/ClickHouse/pull/9989) ([Olga Khvostikova](https://github.com/stavrolia)) -* CROSS JOIN 中更好的内存使用. [#10029](https://github.com/ClickHouse/ClickHouse/pull/10029) ([Artem Zuikov](https://github.com/4ertus2)) -* 获取远程表的结构并设置 skip_unavailable_shards 时尝试连接到集群中的所有分片. [#7278](https://github.com/ClickHouse/ClickHouse/pull/7278) ([nvartolomei](https://github.com/nvartolomei)) -* 将 `total_rows`/`total_bytes` 添加到 `system.tables` 表中. [#9919](https://github.com/ClickHouse/ClickHouse/pull/9919) ([Azat Khuzhin](https://github.com/azat)) -* 系统日志表现在默认使用多态部分. [#9905](https://github.com/ClickHouse/ClickHouse/pull/9905) ([Anton Popov](https://github.com/CurtizJ)) -* 将类型列添加到 system.settings/merge_tree_settings. [#9909](https://github.com/ClickHouse/ClickHouse/pull/9909) ([Azat Khuzhin](https://github.com/azat)) -* 尽早在服务器启动时检查可用的 CPU 指令. [#9888](https://github.com/ClickHouse/ClickHouse/pull/9888) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 从突变中移除 `ORDER BY` 阶段, 因为我们从单个线程中的单个有序部分读取. 还添加检查突变中的行是否按排序键排序并且不违反此顺序. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)) -* 在左侧为 FixedString 实现运算符 LIKE. 这是更好地支持 TPC-DS 查询所必需的. [#9890](https://github.com/ClickHouse/ClickHouse/pull/9890) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加 `force_optimize_skip_unused_shards_no_nested` 将禁用嵌套分布式表的 `force_optimize_skip_unused_shards`. [#9812](https://github.com/ClickHouse/ClickHouse/pull/9812) ([Azat Khuzhin](https://github.com/azat)) -* 现在, 对于 MergeTree 数据部分, 列大小只计算一次. [#9827](https://github.com/ClickHouse/ClickHouse/pull/9827) ([alesapin](https://github.com/alesapin)) -* 评估`optimize_skip_unused_shards` 的常量表达式 (即`SELECT * FROM foo_dist WHERE key=xxHash32(0)`).[#8846](https://github.com/ClickHouse/ClickHouse/pull/8846) ([Azat Khuzhin](https://github.com/azat)) -* 删除了从 TTL 表达式中检查使用 `Date` 或 `DateTime` 列的方法. [#9967](https://github.com/ClickHouse/ClickHouse/pull/9967) ([Vladimir Chebotarev](https://github.com/excitoon)) -* DiskS3硬链接优化实现. [#9760](https://github.com/ClickHouse/ClickHouse/pull/9760) ([Pavel Kovalenko](https://github.com/Jokser)) -* 如果 `set multiple_joins_rewriter_version = 2` 启用多个 JOIN 重写的第二个版本, 以保持列名不冲突. 它支持多个带有 `USING` 的JOIN, 并允许 `select *` 用于带有子查询的JOIN. [#9739](https://github.com/ClickHouse/ClickHouse/pull/9739) ([Artem Zuikov](https://github.com/4ertus2)) -* StorageMergeTree 的 `non-blocking` 更改的实现. [#9606](https://github.com/ClickHouse/ClickHouse/pull/9606) ([alesapin](https://github.com/alesapin)) -* 添加 MergeTree 对 DiskS3 的全面支持. [#9646](https://github.com/ClickHouse/ClickHouse/pull/9646) ([Pavel Kovalenko](https://github.com/Jokser)) -* 扩展 `splitByString` 以支持空字符串作为分隔符. [#9742](https://github.com/ClickHouse/ClickHouse/pull/9742) ([hcz](https://github.com/hczhcz)) -* 将 `timestamp_ns` 列添加到 `system.trace_log` . 它包含跟踪事件的高清时间戳, 并允许构建线程配置文件的时间线 ("火焰图"). [#9696](https://github.com/ClickHouse/ClickHouse/pull/9696) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 启用设置 `send_logs_level` 时, 避免日志消息和查询进度混淆. [#9634](https://github.com/ClickHouse/ClickHouse/pull/9634) ([Azat Khuzhin](https://github.com/azat)) -* 添加了对`MATERIALIZE TTL IN PARTITION`的支持. [#9581](https://github.com/ClickHouse/ClickHouse/pull/9581) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 支持 Avro 嵌套字段内的复杂类型. [#10502](https://github.com/ClickHouse/ClickHouse/pull/10502) ([Andrew Onyshchuk](https://github.com/oandrew)) - -#### 性能改进 -* 为 Partial MergeJoin 的右表提供更好的插入逻辑. [#10467](https://github.com/ClickHouse/ClickHouse/pull/10467) ([Artem Zuikov](https://github.com/4ertus2)) -* 改进了面向行的格式的性能 (CSV 超过 10%,Avro 超过 35%, 在窄表的情况下). [#10503](https://github.com/ClickHouse/ClickHouse/pull/10503) ([Andrew Onyshchuk](https://github.com/oandrew)) -* 使用 IN 运算符右侧显式定义的集合和左侧的元组提高查询的性能. [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) -* 在 HashJoin 中为哈希表使用更少的内存. [#10416](https://github.com/ClickHouse/ClickHouse/pull/10416) ([Artem Zuikov](https://github.com/4ertus2)) -* StorageDictionary 上的特殊 HashJoin. 允许使用 JOIN 重写 `dictGet()` 函数. 它本身并不是向后不兼容的, 但可能会在某些安装中发现 [#8400](https://github.com/ClickHouse/ClickHouse/issues/8400). [#10133](https://github.com/ClickHouse/ClickHouse/pull/10133) ([Artem Zuikov](https://github.com/4ertus2)) -* 当目标表支持时启用物化视图的并行插入. [#10052](https://github.com/ClickHouse/ClickHouse/pull/10052) ([vxider](https://github.com/Vxider)) -* 使用单调函数改进指数分析的性能. [#9607](https://github.com/ClickHouse/ClickHouse/pull/9607)[#10026](https://github.com/ClickHouse/ClickHouse/pull/10026) ([Anton Popov](https://github.com/CurtizJ)) -* 使用 SSE2 或 SSE4.2 SIMD 内在函数来加速布隆过滤器中的标记化. [#9968](https://github.com/ClickHouse/ClickHouse/pull/9968) ([Vasily Nemkov](https://github.com/Enmk)) -* 在 `IN` 运算符的右侧使用显式定义的集合提高查询的性能. 这修复了 20.3 版中的性能回归. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740) ([Anton Popov](https://github.com/CurtizJ)) -* 现在, clickhouse-copier 将每个分区分成几部分并独立复制. [#9075](https://github.com/ClickHouse/ClickHouse/pull/9075) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 添加更多聚合方法. 例如,TPC-H 查询 1 现在将选择 `FixedHashMap` 并获得 25% 的性能提升 [#9829](https://github.com/ClickHouse/ClickHouse/pull/9829) ([Amos Bird](https://github.com/amosbird)) -* 在预限制转换中对多个流使用单行计数器. 这有助于避免在带有 `limit` 但没有 `order by` 的查询中联合管道流 (如 `select f(x) from (select x from t limit 1000000000)`) 并使用多个线程进行进一步处理. [#9602](https://github.com/ClickHouse/ClickHouse/pull/9602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### 构建/测试/打包改进 -* 使用来自 ClickHouse-Extras 的 AWS SDK 库的分支. [#10527](https://github.com/ClickHouse/ClickHouse/pull/10527) ([Pavel Kovalenko](https://github.com/Jokser)) -* 为新的 ALTER RENAME COLUMN 查询添加集成测试. [#10654](https://github.com/ClickHouse/ClickHouse/pull/10654) ([vzakaznikov](https://github.com/vzakaznikov)) -* 在使用错误参数调用函数 `now64` 时修复可能的有符号整数溢出. 这修复了 [#8973](https://github.com/ClickHouse/ClickHouse/issues/8973) [#10511](https://github.com/ClickHouse/ClickHouse/pull/10511) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 拆分 fuzzer 和 sanitizer 配置, 使构建配置与 Oss-fuzz 兼容. [#10494](https://github.com/ClickHouse/ClickHouse/pull/10494) ([kyprizel](https://github.com/kyprizel)) -* 修复了 clang-10 上的 clang-tidy. [#10420](https://github.com/ClickHouse/ClickHouse/pull/10420) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在错误消息中显示绝对路径. 否则 KDevelop 无法导航到正确的文件, 而是打开一个新文件. [#10434](https://github.com/ClickHouse/ClickHouse/pull/10434) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了 `ASAN_OPTIONS` 环境变量以使用 Address sanitizer 调查 CI 压力测试中的错误. [#10440](https://github.com/ClickHouse/ClickHouse/pull/10440) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 为 clang 构建启用 ThinLTO (实验性). [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 删除系统安装 Z3 求解器时可能引入的对 Z3 的意外依赖. [#10426](https://github.com/ClickHouse/ClickHouse/pull/10426) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将集成测试 docker 文件移动到 docker/ 目录. [#10335](https://github.com/ClickHouse/ClickHouse/pull/10335) ([Ilya Yatsishin](https://github.com/qoega)) -* 允许在 CI 中使用 `clang-10`. 它确保 [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) 是已修复的. [#10384](https://github.com/ClickHouse/ClickHouse/pull/10384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 OpenSSL 更新为上游主服务器. 修复了 TLS 连接可能失败并显示消息 `OpenSSL SSL_read: error:14094438:SSLroutines:ssl3_read_bytes:tlsv1 alert internal error` 和 `SSL Exception: error:2400006E:random number generator::error retrieving entropy` 时的问题. 该问题存在于 20.1 版中. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 clang-10 构建. [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)) -* 为[Parallel INSERT for materialized view] (https://github.com/ClickHouse/ClickHouse/pull/10052)添加性能测试. [#10345](https://github.com/ClickHouse/ClickHouse/pull/10345) ([vxider](https://github.com/Vxider)) -* 修复片状测试 `test_settings_constraints_distributed.test_insert_clamps_settings`. [#10346](https://github.com/ClickHouse/ClickHouse/pull/10346) ([Vitaly Baranov](https://github.com/vitlibar)) -* 添加 util 以在 CI ClickHouse 中上传测试结果. [#10330](https://github.com/ClickHouse/ClickHouse/pull/10330) ([Ilya Yatsishin](https://github.com/qoega)) -* 在 junit_to_html 工具中将测试结果转换为 JSONEachRow 格式. [#10323](https://github.com/ClickHouse/ClickHouse/pull/10323) ([Ilya Yatsishin](https://github.com/qoega)) -* 更新 cctz. [#10215](https://github.com/ClickHouse/ClickHouse/pull/10215) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 允许从最纯粹的 JUnit XML 报告创建 HTML 报告. [#10247](https://github.com/ClickHouse/ClickHouse/pull/10247) ([Ilya Yatsishin](https://github.com/qoega)) -* 更新对最小编译器版本的检查. 修复问题的根本原因 [#10250](https://github.com/ClickHouse/ClickHouse/issues/10250) [#10256](https://github.com/ClickHouse/ClickHouse/pull/10256) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 对分布式实时视图表的初始支持. [#10179](https://github.com/ClickHouse/ClickHouse/pull/10179) ([vzakaznikov](https://github.com/vzakaznikov)) -* 在 MergeTreeIndexFullText 中修复 (false) MSan 报告. 该问题首先出现在 [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* clickhouse-docker-util. [#10151](https://github.com/ClickHouse/ClickHouse/pull/10151) ([filimonov](https://github.com/filimonov)) -* 将 pdqsort 更新到最新版本. [#10171](https://github.com/ClickHouse/ClickHouse/pull/10171) ([Ivan](https://github.com/abyss7)) -* 将 libdivide 更新至 v3.0. [#10169](https://github.com/ClickHouse/ClickHouse/pull/10169) ([Ivan](https://github.com/abyss7)) -* 使用启用的多态部分添加检查. [#10086](https://github.com/ClickHouse/ClickHouse/pull/10086) ([Anton Popov](https://github.com/CurtizJ)) -* 为 FreeBSD 添加交叉编译构建. 这修复了 [#9465](https://github.com/ClickHouse/ClickHouse/issues/9465) [#9643](https://github.com/ClickHouse/ClickHouse/pull/9643) ([Ivan](https://github.com/abyss7)) -* 添加性能测试. [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924) [#6980](https://github.com/ClickHouse/ClickHouse/pull/6980) ([filimonov](https://github.com/filimonov)) -* 在 `File` 引擎中添加对 `/dev/null` 的支持以进行更好的性能测试. [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) -* 将 /dbms 中的所有文件夹向上移动一级. [#9974](https://github.com/ClickHouse/ClickHouse/pull/9974) ([Ivan](https://github.com/abyss7)) -* 添加一个测试, 检查从 MergeTree 中读取的单线程是否按顺序执行. 添加到 [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670) [#9762](https://github.com/ClickHouse/ClickHouse/pull/9762) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 `00964_live_view_watch_events_heartbeat.py` 测试以避免竞争条件. [#9944](https://github.com/ClickHouse/ClickHouse/pull/9944) ([vzakaznikov](https://github.com/vzakaznikov)) -* 修复集成测试 `test_settings_constraints`. [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)) -* 每个函数都在自己的文件中, 第 12 部分. [#9922](https://github.com/ClickHouse/ClickHouse/pull/9922) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 增加了对元组数组分析速度极慢的情况的性能测试. [#9872](https://github.com/ClickHouse/ClickHouse/pull/9872) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 zstd 更新到 1.4.4. 它在性能和压缩比方面有一些小的改进. 如果您使用不同版本的 ClickHouse 运行副本, 您可能会看到合理的错误消息 `Data after merge is not byte-identical to data on another replicas.` 并附有说明. 这些消息没问题, 您不必担心. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 `system.stack_trace` 中的 TSan 报告. [#9832](https://github.com/ClickHouse/ClickHouse/pull/9832) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 删除了对 `clock_getres` 的依赖. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了使用 clang-tidy 的标识符名称检查. [#9799](https://github.com/ClickHouse/ClickHouse/pull/9799) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更新 `builder` docker 镜像. 此镜像未在 CI 中使用, 但对开发人员有用. [#9809](https://github.com/ClickHouse/ClickHouse/pull/9809) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 删除 CI 中不再使用的旧的 `performance-test` 工具. `clickhouse-performance-test` 很棒, 但现在我们正在使用更高级的工具, 该工具通过复杂的统计公式进行比较测试, 无论环境如何变化, 都能获得可靠的结果. [#9796](https://github.com/ClickHouse/ClickHouse/pull/9796) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了大部分 clang-static-analyzer 检查. [#9765](https://github.com/ClickHouse/ClickHouse/pull/9765) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 Poco 更新到 1.9.3, 为 MongoDB URI 支持做准备. [#6892](https://github.com/ClickHouse/ClickHouse/pull/6892) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 使用 `-DUSE_STATIC_LIBRARIES=0 -DENABLE_JEMALLOC=0` 修复构建. [#9651](https://github.com/ClickHouse/ClickHouse/pull/9651) ([Artem Zuikov](https://github.com/4ertus2)) -* 对于变更日志脚本, 如果合并提交被挑选到发布分支, 则从提交描述中获取 PR 名称. [#9708](https://github.com/ClickHouse/ClickHouse/pull/9708) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 在 backport 脚本中支持 `vX.X-conflicts` 标签. [#9705](https://github.com/ClickHouse/ClickHouse/pull/9705) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复用于向后移植脚本的 `auto-label` . [#9685](https://github.com/ClickHouse/ClickHouse/pull/9685) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 在 Darwin 交叉构建中使用 libc++ 使其与原生构建一致. [#9665](https://github.com/ClickHouse/ClickHouse/pull/9665) ([Hui Wang](https://github.com/huiwang)) -* 修复不稳定的测试 `01017_uniqCombined_memory_usage`. 续 [#7236](https://github.com/ClickHouse/ClickHouse/issues/7236). [#9667](https://github.com/ClickHouse/ClickHouse/pull/9667) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复原生 MacOS Clang 编译器的构建. [#9649](https://github.com/ClickHouse/ClickHouse/pull/9649) ([Ivan](https://github.com/abyss7)) -* 允许在 `pthread_mutex_lock`、`pthread_mutex_unlock` 函数周围添加各种故障. [#9635](https://github.com/ClickHouse/ClickHouse/pull/9635) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 `packager` 脚本中添加对 `clang-tidy` 的支持. [#9625](https://github.com/ClickHouse/ClickHouse/pull/9625) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加使用未捆绑的 msgpack 的能力. [#10168](https://github.com/ClickHouse/ClickHouse/pull/10168) ([Azat Khuzhin](https://github.com/azat)) - - -## ClickHouse 版本 v20.3 - - -### ClickHouse 版本 v20.3.21.2-lts, 2020-11-02 - -#### Bug 修复 - -* 修复 sharding_key 中的 dictGet (以及类似的地方, 即当函数上下文被永久存储时). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). -* 如果查询具有`WHERE`、`PREWHERE` 和`GLOBAL IN`, 则修复来自`Distributed` 表的查询的错误空结果. 修复 [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `TSV/CSVWithNames` 格式中缺失或过多的标题. 这修复了 [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse 版本 v20.3.20.6-lts, 2020-10-09 - -#### Bug 修复 - -* Mutation 可能会在 `MOVE` 或 `REPLACE PARTITION` 之后等待一些不存在的部分, 或者在极少数情况下, 在 `DETACH` 或 `DROP PARTITION` 之后挂起。 已修复. [#15724](https://github.com/ClickHouse/ClickHouse/pull/15724), [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* 修复带有大量子查询的查询挂起到 `MySQL` 引擎的同一个表. 以前, 如果查询中有超过 16 个子查询到同一个 `MySQL` 表, 它会永远挂起. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). -* 当查询在 Merge 表上有 JOIN 时,修复 GROUP BY 中的 `Unknown identifier` . [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). -* 当子查询包含 finalizeAggregation 函数时, 修复使谓词下推工作. 修复 [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* 并发 `ALTER ... REPLACE/MOVE PARTITION ...` 查询可能会导致死锁. 已修复. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). - - -### ClickHouse 版本 v20.3.19.4-lts, 2020-09-18 - -#### Bug 修复 - -* 当被查询的列具有 `DEFAULT` 表达式时, 修复 `SELECT` 查询中的罕见错误, 该表达式依赖于另一列, 该列也具有 `DEFAULT` 并且不存在于选择查询中且不存在于磁盘上. 部分修复 [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* 修复了在赋值表达式和常量值 (如 `UPDATE x = 42`) 中使用 Nullable 列的 `ALTER UPDATE` 突变导致列或段错误中的值不正确时的错误. 修复 [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* 修复错误的十进制乘法结果导致结果列小数位数错误. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). - -#### 改进 - -* 支持合并部分的自定义编解码器. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse 版本 v20.3.18.10-lts, 2020-09-08 - -#### Bug 修复 - -* 如果 `PipelineExecutor` 本身发生异常, 则停止查询执行. 这可以防止罕见的查询挂起. 续 [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了有时缓存字典从源返回默认值而不是当前值时的行为. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 当数据库或表的名称包含点时, 修复从 users.xml 解析行策略. 这修复了[#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复 CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `text_log` 中的数据竞争. 它不对应任何真正的错误. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### 改进 - -* 修复长查询的错误错误. 为了正确查询, 可能会出现除 `Max query size exceeded` 之外的语法错误. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当在 parseDateTimeBestEffortOrNull/Zero 函数中没有完全解析值时, 返回 NULL/零. 这修复了 [#7876](https://github.com/ClickHouse/ClickHouse/issues/7876). [#11653](https://github.com/ClickHouse/ClickHouse/pull/11653) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### 性能改进 - -* 使用 LowCardinality 稍微优化非常短的查询. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). - -#### 构建/测试/打包改进 - -* 修复迁移到 clang-10 后出现的 HashTable 中的 UBSan 报告 (向 nullptr 添加零). [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse 版本 v20.3.17.173-lts, 2020-08-15 - -#### Bug 修复 - -* 使用 StorageMerge 和 `set enable_optimize_predicate_expression=1` 修复 JOIN 中的崩溃. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复无效的返回类型, 用于将元组与 `NULL` 元素进行比较. 修复[#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用常量列和主键的 `ORDER BY` 前缀修复查询. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). -* 返回在 roundUpToPowerOfTwoOrZero() 中设置 MSB 的数字的传递数字. [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse 版本 v20.3.16.165-lts 2020-08-10 - -#### Bug 修复 - -* 修复了当 unix 时间戳作为参数传递时 `parseDateTimeBestEffort` 函数中的错误. 这修复了 [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在具有 NaN 值的 Float 类型上调用的 `uniqExact`、`topK`、`sumDistinct` 和类似聚合函数的潜在低性能和稍微不正确的结果. 它还在调试版本中触发断言. 这修复了 [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果将可空 constexpr 作为不是文字 NULL 的 cond 的固定函数. 修复 [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 在数组元素为 Nullable 且数组下标也可为 Null 的情况下, 修复了 `arrayElement` 函数中的断言. 这修复了 [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了对从本地副本进行选择的线程数的不必要限制. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了查询 `WITH TOTALS` 时可能出现的数据中可能出现的额外溢出行. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了大元组的性能, 这些元组在 `IN` 部分被解释为函数. 当用户出于某种晦涩的原因编写 `WHERE x IN tuple(1, 2, ...)` 而不是 `WHERE x IN (1, 2, ...)` 的情况. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了 input_format_parallel_parsing 的内存跟踪 (通过将线程附加到组). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) 当子查询包含 with 子句时允许推送谓词. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了 [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) 使用 const 表达式修复布隆过滤器索引. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* 当代理不可用时 (不仅如此) 修复了 StorageKafka 中的 SIGSEGV. [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* 修复了带有缓存布局的外部字典中的竞争条件, 这可能导致服务器崩溃. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* 修复了当 `enable_mixed_granularity_parts=1` 时, 在 `ALTER DELETE` 查询后导致旧部件损坏的错误. 修复 [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* 对于参数数目无效的函数 `in` 更好的异常信息. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了性能问题,同时读取合并部分. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了启用 `text_log` 时的死锁. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 StorageMerge 时可能出现的段错误. 关闭 [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* 修复了带有 `-State` 和 `Nullable` 参数的聚合函数的 `TOTALS/ROLLUP/CUBE`. 这修复了 [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `WITH FILL` 修饰符中列的顺序. 以前不遵守 `ORDER BY` 语句的列顺序. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). -* 当存在按虚拟列 (如 `Merge`表中的`_table`) 或系统表中的 `index` 列过滤数据 (例如从 `system.tables` 查询时按数据库名称过滤) 时, 避免 `bad cast` 异常, 并且这个表达式返回 `Nullable` 类型. 这修复了 [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `TrieDictionary` 加载失败后显示错误. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). -* 对于可能导致崩溃的空数组, 函数 `arrayFill` 工作不正确. 这修复了 [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 实现到 `LowCardinality` 类型的通用类型的转换. 这允许对具有 LowCardinality 列和其他列的表执行 UNION ALL. 这修复了 [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在某些特殊类型的 `StorageFile` 标头中多次连续插入期间多次写入时的行为. 这已修复 [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了 UInt8 值不等于 0 或 1 时的逻辑函数. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). -* 在 GROUP BY 内射函数消除期间修复了 `dictGet` 参数检查. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `ALTER DELETE` 中的错误逻辑, 当条件评估为 NULL 时会导致删除记录. 这修复了 [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在存在别名的情况下发送到外部 DBMS (例如 MySQL、ODBC) 的查询转换. 这修复了 [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了整数除法中的潜在溢出. 这修复了 [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `greatCircleDistance`、`geoDistance` 中潜在的无限循环. 这修复了 [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 对于带有连接或附加到系统日志 (ystem.query_log、metric_log 等) 或 engine=Buffer 基础表的子查询的物化视图, 避免 `There is no query` 异常. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). -* 修复了由于线程总数限制错误导致的带有 `UNION` 的选择的性能. 修复 [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `-StateResample` 组合器修复了段错误. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了不必要的限制从 `VIEW` 中选择的线程数. 修复 [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了使用错误类型 `PREWHERE` 时可能发生的崩溃. 修复[#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了具有 `LowCardinality` 类型的函数 `defaultValueOfArgumentType` 的 `Expected single dictionary argument for function` 错误. 修复 [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了带有 `Tuple(LowCardinality)` 参数的高阶函数的错误 `Cannot capture column`. 修复 [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 加载数据库时并行解析表元数据. 这修复了有大量表时服务器启动缓慢的问题. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). -* 使 `topK` 聚合函数为 Enum 类型返回 Enum. 这修复了 [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了否为常量表达式的约束检查. 这修复了 [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了元组与 `Nullable` 列的不正确比较. 修复 [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在调用具有不同大小的 `FixedString` 类型参数的函数 `if` 时出现错误结果和潜在崩溃的问题. 这修复了[#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果使用偏移量 `-9223372036854775808` 调用函数, 则以函数 `neighbor` 作为唯一返回表达式的查询可能会返回空结果. 这修复了 [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 generateRandom 中可能导致崩溃的潜在数组大小溢出. 这修复了 [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了潜在的浮点异常. 这将关闭 [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了服务器启动时日志消息中错误的设置名称. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `Values` 格式中的 `Query parameter was not set` 问题. 修复 [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). -* 在查询 (参数化查询) 中保留替换的别名. 这修复了 [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了解析 DateTime64 时潜在的浮点异常. 这修复了 [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 通过 `HTTP` 接口修复内存记帐 (对于 `wait_end_of_query=1` 可能很重要). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). -* 修复了条件为 NULL 的 `if()` 的错误结果. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). -* 在检查相等性之前解析存储在 zookeeper 中的元数据. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `LIMIT n WITH TIES` 与 `ORDER BY` 语句一起使用的问题, 该语句包含别名. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). -* 修复缓存字典中未初始化内存的潜在读取. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### 性能改进 - -* 索引不用于带文字的 IN 运算符, 在 v19.3 前后引入了性能回归. 这修复了 [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). - - -### ClickHouse 版本 v20.3.12.112-lts 2020-06-25 - -#### Bug 修复 - -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. 续 [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 不允许在高阶函数中使用 arrayJoin. 它导致协议同步中断. 这关闭了 [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复使用过多线程进行查询. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复像 `SELECT *, xyz.*` 这样的查询的意外行为, 这些查询在预期错误时成功. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* 现在复制的提取将在元数据更改期间取消. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). -* 修复了由 Values 输入格式中复杂文字的错误类型推导导致的 LOGICAL_ERROR. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* 修复 `ORDER BY ... WITH FILL` 在 const 列上. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* 与 XDBC 桥通信时传递适当的超时. 最近在检查桥接活性和接收元信息时没有遵守超时. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复导致 `system.mutations` 状态不正确的错误. 它可能表明整个突变已经完成, 但服务器在复制队列中仍有 `MUTATE_PART` 任务并尝试执行它们. 这修复了 [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). -* 添加对带有不区分大小写标志的正则表达式的支持. 这修复了 [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果设置了行级安全性,则删除琐碎的计数查询优化. 在以前的版本中, 用户获取表中记录的总数而不是过滤. 这修复了[#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复字符串的布隆过滤器 (数据跳过索引). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. (可能它以某种方式与 [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复错误 `Block structure mismatch` , 用于从 `Buffer` 表中采样读取的查询. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 clickhouse-client 的错误退出代码, 当 exception.code() % 256 = 0 时. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* 修复服务器启动时有关 `Mark cache size was lowered` 的日志消息中的小错误. 这将关闭 [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复错误 `Size of offsets does not match size of column` , 用于查询中的 `PREWHERE column in (subquery)` 和 `ARRAY JOIN` . [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* HTTP 会话中的所有查询都具有相同的 query_id. 已修复. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). -* 现在 clickhouse-server docker 容器将更喜欢 IPv6 检查服务器活动性. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* 修复 `` 的 shard_num/replica_num (破坏 use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). -* 使用 -State 函数在聚合过程中抛出异常时修复内存泄漏. 这修复了 [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当别名可以覆盖限定的列名, 修复分布式查询的错误结果. 修复 [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714). [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)). - - -### ClickHouse 版本 v20.3.11.97-lts 2020-06-10 - -#### 新特征 - -* 现在, ClickHouse 在其一侧控制字典源的超时. 缓存字典配置中添加了两个新设置: `strict_max_lifetime_seconds`, 默认为 `max_lifetime` 和 `query_wait_timeout_milliseconds`, 默认为一分钟. 第一个设置对于 `allow_read_expired_keys` 设置也很有用 (禁止读取非常过期的键). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - -#### Bug 修复 - -* 修复了在启用 `min_bytes_to_use_direct_io` 且 PREWHERE 处于活动状态并使用 SAMPLE 或大量线程时可能发生的 `Data compressed with different methods` 错误. 这修复了 [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复编解码器的返回压缩大小. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当列具有带有非文字参数的压缩编解码器时修复服务器崩溃. 修复 [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* 以 nan 为点修复 pointInPolygon. 修复 [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* 修复 JOIN 在 LowCarinality(T) 和 Nullable(T) 上的崩溃. [#11380](https://github.com/ClickHouse/ClickHouse/issues/11380). [#11414](https://github.com/ClickHouse/ClickHouse/pull/11414) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复错误 `USING` 键的错误代码. [#11373](https://github.com/ClickHouse/ClickHouse/issues/11373). [#11404](https://github.com/ClickHouse/ClickHouse/pull/11404) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了带有纬度/经度范围之外的参数的 geohashesInBox. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* `joinGet()` 函数更好的错误. [#11389](https://github.com/ClickHouse/ClickHouse/pull/11389) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复具有外部排序和限制的查询可能出现的 `Pipeline stuck` 错误. 修复 [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 在 ReplicatedMergeTree 中发送零件期间删除冗余锁. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). -* 修复了多行模式下 clickhouse-client 对 `\G`(垂直输出)的支持. 这将关闭 [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复直接从 StorageJoin 选择 (没有 JOIN) 时的崩溃和错误的可空性. [#11340](https://github.com/ClickHouse/ClickHouse/pull/11340) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复 `quantilesExactWeightedArray` 中的崩溃问题. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 现在合并在 `ALTER` 查询中更改元数据之前停止. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). -* 通过再次设置 `parallel_view_processing = 1` 并行写入 `MATERIALIZED VIEW` . 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当提取的 JSON 具有不平衡的 { 或 [ 字符串时, 修复visitParamExtractRaw. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* 修复 ThreadPool 中非常罕见的竞争条件. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复转换中潜在的未初始化内存. 示例: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了如果表的主键中有 Array 列并且查询正在使用 `empty` 或 `notEmpty` 函数按此列过滤时索引分析无法工作的问题. 这修复了 [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复当查询速度估计可能不正确时的错误, 并且如果查询受到 `max_network_bandwidth`、`max_execution_speed` 或 `priority` 设置的限制, `min_execution_speed` 的限制可能无法工作或无法正常工作. 将 `timeout_before_checking_execution_speed` 的默认值更改为非零值, 否则设置 `min_execution_speed` 和 `max_execution_speed` 无效. 这修复了 [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). 这修复了 [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). 这修复了 [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). 可用性改进:避免在 `clickhouse-client` 中将异常消息与进度条串联. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复读取 Protobuf 格式错误数据时崩溃的问题. 这修复了 [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复了缓存字典可以返回默认值而不是正常值 (只有过期键时) 的错误. 这仅影响字符串字段. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复错误 `Block structure mismatch in QueryPipeline` , 同时从内部查询中的常量读取 `VIEW` . 修复 [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复可能的异常 `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `Array(Array(LowCardinality))` 捕获参数修复高阶函数的可能错误 `Cannot capture column` . [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 S3 globbing 在超过 1000 个密钥和一些后端的情况下可能会失败的问题. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). -* 如果数据跳过索引依赖于将在后台合并期间修改的列 (对于 SummingMergeTree、AggregatingMergeTree 以及 TTL GROUP BY), 则计算错误. 此问题已通过合并后移动索引计算得到修复, 因此索引是在合并数据上计算的. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* 修复简单查询过多的线程预留 (优化减少线程数量, 在管道更改后部分中断). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). -* 修复分布式查询的谓词优化 (`enable_optimize_predicate_expression=1`) 对于带有 `HAVING` 部分的查询 (即需要在服务器启动器上进行过滤时), 通过保留表达式的顺序 (这足以修复), 并且还强制 聚合器在索引上使用列名. 修复: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). -* 引入提交重试逻辑, 以减少在极少数情况下, 偏移提交失败时从 Kafka 获取重复项的可能性. [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)). - -#### 性能改进 - -* 每次调用读取外部字典的任何函数时, 仅获取一次字典并检查访问权限. [#10928](https://github.com/ClickHouse/ClickHouse/pull/10928) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### 构建/测试/打包改进 - -* 修复几个不稳定的集成测试. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). - -### ClickHouse 版本 v20.3.10.75-lts 2020-05-23 - -#### Bug 修复 - -* 如果没有最终确定, 则从突变终止任务中删除日志记录. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* 修复了 `parseDateTime64BestEffort` 参数解析错误. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* 修复了方法 `getRawData()` 中不正确的原始数据大小. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* 修复了 20.1 及更早版本之间两级聚合的不兼容问题. 当在启动器节点和远程节点上使用不同版本的 ClickHouse 并且 `GROUP BY` 结果的大小很大并且聚合由单个 `String` 字段执行时, 就会发生这种不兼容性. 结果导致单个键的多个未合并的行. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了与 `Distributed` 表中元组的向后兼容性. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* 如果这样的键不存在, 则修复了 `StringHashTable` 中的 `SIGSEGV`. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `ReplicatedMergeTree` 中的错误, 该错误可能导致 `OPTIMIZE` 查询上的某些 `ALTER` 在它变为非活动状态后挂起等待某些副本. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* 修复 `Block::sortColumns()` 之后的列顺序. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* 修复了当不要求引用标识符时 `ODBC` 桥的问题. 修复 [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `DateLUT` 中的 `UBSan` 和 `MSan` 报告. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了关键条件下错误的类型转换. 修复[#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)) -* 修复了 `parallel_view_processing` 行为. 现在, 如果发生异常, 所有对 `MATERIALIZED VIEW` 的插入都应该无一例外地完成. 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了组合符 `-OrNull` 和 `-OrDefault` 与 `-State` 组合时的问题. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* 修复了带有嵌套类型的 `generateRandom` 中的崩溃. 修复 [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了合并后可能发生的 `SummingMergeTree` 中 `LowCardinality(FixedString)` 键列的数据损坏. 修复 [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了函数 `h3EdgeAngle` 中可能出现的缓冲区溢出. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了消失的总数. 如果查询具有连接或具有外部 where 条件的子查询, 则总计可能已被过滤. 修复 [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在一个查询中具有相同集合的 `IN` 运算符的多次使用. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了当 `readonly=2` 和 `cancel_http_readonly_queries_on_client_close=1` 时导致 http 请求卡在客户端关闭的错误. 修复 [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `AggregateTransform` 构造函数中参数的顺序. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* 修复了启用 `distributed_aggregation_memory_efficient` 时远程查询缺乏并行执行的问题. 修复[#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了使用 `LIMIT` 的查询可能不正确的行数. 修复 [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了当表有很多部分时锁定并发更改的错误. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* 修复了在 `SYSTEM DROP DNS CACHE` 查询也删除缓存时的错误, 用于检查是否允许用户从某些 IP 地址连接. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `MATERIALIZED VIEW` 内部查询中不正确的标量结果, 以防该查询包含相关表. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了列 `ALIAS` 的 `SELECT`, 它的默认表达式类型与列类型不同. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* 实现了 DateTime64 和 String 值之间的比较. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* 修复索引损坏, 在某些情况下将紧凑部分合并到另一个紧凑部分后可能会发生. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). -* 修复了所有部分mutation完成后, 在 `is_done=0` 中挂掉的情况. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). -* 修复了 unix 纪元开始时与 `UTC` 有小数偏移的时区溢出. 这修复了 [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了 `Distributed` 存储的不当关闭. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `simpleLinearRegression` 中大整数上的数字溢出. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). - - -#### 构建/测试/打包改进 - -* 修复 LZ4 库中的 UBSan 报告. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 clang-10 构建. [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238). [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)). -* 添加了关于 `max_rows_to_sort` 设置的失败测试. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加了在输入格式中打印诊断信息的一些改进. 修复 [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). -* 将 CA 证书添加到 clickhouse-server docker 镜像. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). - -#### Bug 修复 - -* 修复错误 `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). - - -### ClickHouse 版本 v20.3.8.53, 2020-04-23 - -#### Bug 修复 -* 修复了时区的日期时间函数的错误行为, 这些时区在 UTC 的正负偏移量之间发生了变化 (例如 Pacific/Kiritimati). 这修复了[#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复启用 `distributed_group_by_no_merge` 时可能出现的段错误 (在 20.3.7.46 中由 [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131) 引入). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复 `Array(Tuple(...))` 数据类型的错误扁平化. 这修复了 [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在聚合器中删除磁盘预留. 这修复了磁盘空间预留的错误, 该错误可能导致大型外部聚合即使成功完成也可能失败. [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) -* 修复了 `ReplicatedMergeTree` 中的 `DROP` 与 `OPTIMIZE` 竞赛. 如果存在并发的 OPTIMIZE 查询, 则 DROP 可能会在 ZooKeeper 的副本路径中留下一些垃圾. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) -* 修复更改列默认值后服务器无法附加表的错误. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) -* 在加载表之前附加数据库失败时不要删除元数据目录. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) -* 修复了一些错误, 当某些数据以法定人数插入, 然后以某种方式删除 (DROP PARTITION, TTL) 时, 这会导致插入卡住或 SELECT 中的误报异常. 这修复了 [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 修复 `ConcatProcessor` 中可能发生的 `Pipeline stuck` 错误,该错误可能发生在远程查询中. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复了 HashTable 中尝试从缓冲区读取 HashMap 时导致编译错误的错误行为. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)) -* 允许对多个 JOIN 使用 `count(*)`. 修复 [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) -* 更喜欢 `fallback_to_stale_replicas` 而不是 `skip_unavailable_shards`, 否则当两个设置都指定并且没有最新的副本时, 查询将失败 (来自@alex-zaitsev的补丁). 修复: [#2564](https://github.com/ClickHouse/ClickHouse/issues/2564). [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) -* 修复带有 ARRAY JOIN、ORDER BY 和 LIMIT 的查询可能返回不完整结果的问题. 这修复了 [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). 作者: [Vadim Plakhtinskiy](https://github.com/VadimPlh). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 创建 BloomFilter 索引时检查参数的数量和类型. [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623) [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)) - -#### 性能改进 -* 在 `IN` 运算符的右侧显式定义的集合和左侧的元组提高了查询的性能. 这修复了 20.3 版中的性能回归. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740), [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) - -### ClickHouse 版本 v20.3.7.46, 2020-04-17 - -#### Bug 修复 - -* 修复 `Logical error: CROSS JOIN has expressions` 错误, 用于使用逗号和名称连接混合的查询. [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)). -* 使用 `max_bytes_before_external_group_by` 修复查询. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)). -* 在存在数组连接函数的情况下修复移动到前置位置优化 (在某些情况下). 这修复了[#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加放宽对使用 `allow_nondeterministic_mutations` 设置的突变中非确定性函数使用限制的能力. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). - -### ClickHouse 版本 v20.3.6.40, 2020-04-16 - -#### 新特征 - -* 添加函数 `isConstant`. 此函数检查其参数是否为常量表达式并返回 1 或 0. 用于开发、调试和演示目的. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Bug 修复 - -* 使用 `max_rows_to_group_by` 和 `group_by_overflow_mode = 'break'` 修复错误 `Pipeline stuck`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复罕见的可能异常 `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了 ClickHouse 会抛出 `Unknown function lambda.` 的错误. 当用户尝试在 ENGINE = Replicated* 的表上运行 ALTER UPDATE/DELETE 时出现错误消息. 检查非确定性函数现在可以正确处理 lambda 表达式. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复了 Date 类型的 `generateRandom` 函数. 这修复了 [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). 修复了将 2106 年的日期插入到使用旧式分区的 MergeTree 表但分区以 1970 年命名的边缘情况. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 如果 View 的表定义与 SELECT 查询不对应,则转换类型. 这修复了 [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022). [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 RFC-2822 中字符串的 `parseDateTimeBestEffort`, 当一周中的某一天是星期二或星期四时. 这修复了[#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 JOIN 内的常量列名可能与 JOIN 外的常量名冲突. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 当查询实际上应该在 LIMIT 停止时, 修复可能的无限查询执行, 同时从无限源 (如 `system.numbers` 或 `system.zeros` ) 读取. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复未指定数据库时使用当前数据库进行访问检查的问题. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)). -* 如果结构在 INSERT 中不匹配, 则将块转换为 Distributed(). [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). -* 修复处理器管道中极端情况可能导致的错误结果. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 用合并部分修复某些类型的改动. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)). -* 在创建新副本时修复不正确的 `index_granularity_bytes` 检查. 修复 [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)). -* 当其结构与基础表不同时, 修复 INSERT 上的 SIGSEGV 到分布式表中. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). -* 使用 `JOIN` 和 `UNION ALL` 修复查询可能丢失的行. 修复 [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了从不存在 `/table/replicas/replica_name/metadata` 节点的旧 ClickHouse 版本更新时复制表启动的问题. 修复 [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)). -* 为 MySQL 数据库引擎添加一些参数检查和支持标识符参数. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). -* 修复来自 localhost clickhouse 服务器的 clickhouse 字典源中的错误. 如果字典和源中的类型不兼容, 该错误可能会导致内存损坏. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). -* 当表包含跳过索引时, 修复`CHECK TABLE`查询中的错误. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)). -* 修复错误 `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. 当设置 `distributed_aggregation_memory_efficient` 被启用, 并且分布式查询从不同分片读取不同级别的聚合数据时发生这种情况 (混合单级和二级聚合). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 GROUP BY 中可能发生的分段错误, 该错误是包含尾随零字节的字符串键 ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复用于远程查询执行的线程数 (性能回归, 自 20.3 起). 当来自 `Distributed` 表的查询在本地和远程分片上同时执行时, 就会发生这种情况. 修复 [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了在某些数据库查询的处理阶段之一未检索到必要表的错误. 修复 [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). -* 修复 `JOIN` 与 `TOTALS` 一起出现时 `Not found column in block` 错误. 修复 [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复 `ON CLUSTER` DDL 查询在服务器启动时冻结的错误 [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). -* 修复解析 CREATE USER 命令中设置的多个主机, 例如 `CREATE USER user6 HOST NAME REGEXP 'lo.?*host', NAME REGEXP 'lo*host'`. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复连接表引擎的 `TRUNCATE` ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). -* 修复 ALTER 中的 `scalar does not exist` 错误 ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). -* 修复 `ReplicatedMergeTree` 中删除和优化之间的竞争条件. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). -* 修复 `distributed_product_mode='local'` 中的限定名称错误. 修复 [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756). [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复了设置 `allow_introspection_functions` 中内省函数的计算授权. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### 构建/测试/打包改进 - -* 修复集成测试`test_settings_constraints`. [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)). -* 删除了对 `clock_getres` 的依赖. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse 版本 v20.3.5.21, 2020-03-27 - -#### Bug 修复 - -* 当查询在分布式表上具有 PREWHERE 和 WHERE 并且 `SET distributed_product_mode = 'local'` 时, 修复 `Different expressions with the same alias` 错误. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复具有复合主键的表的突变过多的内存消耗. 这修复了[#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). -* 对于 INSERT 查询, 分片现在将从发起者获得的设置限制在分片的约束中, 而不是抛出异常. 此修复允许将 INSERT 查询发送到具有其他约束的分片. 此更改改进了修复 [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复 `COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query` 错误, 以防在表列表中使用 COMMA JOIN 进行子查询 (即在 WHERE 中). 修复[#9782](https://github.com/ClickHouse/ClickHouse/issues/9782). [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复客户端上可能出现的异常 `Got 0 in totals chunk, expected 1` on client. 如果右连接表有零行, 则使用“JOIN”进行查询时会发生这种情况. 示例: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. 修复 [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 无法转换类型时使用 optimize_skip_unused_shards 修复 SIGSEGV. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). -* 修复损坏的 `ALTER TABLE DELETE COLUMN` 查询合并部分. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)). -* 修复 max_distributed_connections (带和不带处理器). [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)). -* 修复了函数参数的时区使用不正确的一些情况. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). - -#### 改进 - -* 由于我们从单个线程中的单个有序部分读取, 因此逐阶段从突变中删除顺序. 还添加检查突变中的行顺序是否按排序键顺序排序并且不违反此顺序. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse 版本 v20.3.4.10, 2020-03-20 - -#### Bug 修复 -* 此版本还包含 20.1.8.41 的所有错误修复. -* 修复丢失的 `rows_before_limit_at_least` 用于通过 http (使用处理器管道) 的查询. 这修复了 [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - - -### ClickHouse 版本 v20.3.3.6, 2020-03-17 - -#### Bug 修复 -* 此版本还包含 20.1.7.38 的所有错误修复. -* 修复复制中的错误, 如果用户在以前的版本上执行了更改, 则不允许复制工作. 这修复了 [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. -* 添加设置 `use_compact_format_in_distributed_parts_names` , 它允许将 `INSERT` 查询的文件以更紧凑的格式写入 `Distributed` 表. 这修复了 [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). 它使 20.3 版再次向后兼容. - -### ClickHouse 版本 v20.3.2.1, 2020-03-12 - -#### 向后不兼容变更 - -* 修复了为大量副本的 `Distributed` 表发送数据时 `file name too long` 的问题. 修复了副本凭据在服务器日志中暴露的问题. 磁盘目录名格式改为 `[shard{shard_index}[_replica{replica_index}]]`. [#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([Mikhail Korotov](https://github.com/millb)) 升级到新版本后, 您将不会无需人工干预即可降级, 因为旧服务器版本无法识别新目录格式. 如果要降级, 则必须手动将相应目录重命名为旧格式. 仅当您对 `Distributed` 表使用异步 `INSERT` 时, 此更改才相关. 在 20.3.3 版本中, 我们将引入一个设置, 允许您逐步启用新格式. -* 更改了突变命令的复制日志条目的格式. 在安装新版本之前, 您必须等待旧的突变处理. -* 实现简单的内存分析器, 在超过软分配限制 [#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([Ivan](https:// github.com/abyss7)) [#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([alexey-milovidov](https://github.com/alexey-milovidov)) 专栏 `system.trace_log` 的内容从 `timer_type` 重命名为 `trace_type`. 这将需要更改第三方性能分析和火焰图处理工具. -* 在任何地方使用 OS 线程 id 而不是内部线程号. 这修复了 [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) 当启用设置 `send_logs_level` 时, 旧的 `clickhouse-client` 无法接收从服务器发送的日志, 因为名称和结构化日志消息的类型已更改. 另一方面, 不同的服务器版本可以相互发送不同类型的日志. 当你不使用 `send_logs_level` 设置时, 你应该不在乎. [#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 移除 `indexHint` 函数. [#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 移除 `findClusterIndex`、`findClusterValue` 函数. 这修复了 [#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). 如果您正在使用这些功能, 请发送电子邮件至 `clickhouse-feedback@yandex-team.com`. [#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 现在不允许使用 `SELECT` 子查询作为默认表达式创建列或添加列. [#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([alesapin](https://github.com/alesapin)) -* JOIN 中的子查询需要别名. [#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) -* 改进了 `ALTER MODIFY/ADD` 查询逻辑. 现在你不能在没有类型的情况下 `ADD` 列, `MODIFY` 默认表达式不会改变列的类型, `MODIFY` 类型不会丢失默认表达式值. 修复 [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([alesapin](https://github.com/alesapin)) -* 需要重新启动服务器以应用日志配置中的更改. 这是一个临时解决方法, 以避免服务器记录到已删除的日志文件的错误 (请参阅 [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 默认情况下启用设置 `experimental_use_processors`. 此设置启用新查询管道的使用. 这是内部重构, 我们预计不会有明显的变化. 如果您会看到任何问题, 请将其设置为零. [#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 新特征 -* 添加 `Avro` 和 `AvroConfluent` 输入/输出格式. [#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* `cache` 字典中过期键的多线程和非阻塞更新 (具有读取旧键的可选权限). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 添加查询 `ALTER ... MATERIALIZE TTL`. 它运行突变, 强制通过 TTL 删除过期数据并重新计算有关 TTL 的所有部分的元信息. [#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([Anton Popov](https://github.com/CurtizJ)) -* 如果需要, 从 HashJoin 切换到 MergeJoin (在磁盘上). [#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) -* 为 `ALTER TABLE` 添加了 `MOVE PARTITION` 命令. [#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([Guillaume Tassery](https://github.com/YiuRULE)) -* 即时从配置文件重新加载存储配置. [#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 允许将 `storage_policy` 更改为不那么丰富的一个. [#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 添加了对 S3 存储和表功能的通配符/通配符的支持. [#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 为 `FixedString(N)` 数据类型实现 `bitAnd`、`bitOr`、`bitXor`、`bitNot`. [#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([Guillaume Tassery](https://github.com/YiuRULE)) -* 添加函数 `bitCount` . 这修复了 [#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([ikopylov](https://github.com/ikopylov)) -* 添加 `generateRandom` 表函数以生成具有给定模式的随机行. 允许用数据填充任意测试表. [#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) -* `JSONEachRowFormat`: 当对象包含在顶级数组中时支持特殊情况. [#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([Kruglov Pavel](https://github.com/Avogar)) -* 现在可以使用 `DEFAULT` 表达式创建一个列, 该列依赖于具有默认 `ALIAS` 表达式的列. [#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([alesapin](https://github.com/alesapin)) -* 允许在 `clickhouse-obfuscator` 中指定大于源数据大小的 `--limit`. 数据将用不同的随机种子重复. [#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了带有储层采样算法的 `groupArraySample` 函数 (类似于 `group Array` ). [#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([Amos Bird](https://github.com/amosbird)) -* 现在您可以通过系统指标监控 `cache`/`complex_key_cache` 字典中更新队列的大小. [#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 允许使用 CRLF 作为 CSV 输出格式的行分隔符, 并将 `output_format_csv_crlf_end_of_line` 设置为 1 [#8934](https://github.com/ClickHouse/ClickHouse/pull/8934) [#8935](https://github.com/ClickHouse/ClickHouse/pull/8935) [#8963](https://github.com/ClickHouse/ClickHouse/pull/8963) ([Mikhail Korotov](https://github.com/millb)) -* 实现[H3](https://github.com/uber/h3) API的更多功能: `h3GetBaseCell`、`h3HexAreaM2`、`h3IndexesAreNeighbors`、`h3ToChildren`、`h3ToString`和`stringToH3`. [#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) -* 引入了新设置: `max_parser_depth` 以控制最大堆栈大小并允许大型复杂查询. 这修复了 [#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) 和 [#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([Maxim Smirnov](https://github.com/qMBQx8GH)) -* 如果无法跳过未使用的分片, 则添加设置 `force_optimize_skip_unused_shards` 设置以抛出. [#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) -* 允许配置多个磁盘/卷来存储数据以在 `Distributed` 引擎中发送. [#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) -* 支持存储临时数据的存储策略 (``). [#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) -* 添加了 `X-ClickHouse-Exception-Code` HTTP 标头, 如果在发送数据之前抛出异常, 则会设置该标头. 这实现了 [#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([Mikhail Korotov](https://github.com/millb)) -* 添加函数 `ifNotFinite`. 它只是一个语法糖: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 `system.dictionaries` 表中添加了 `last_successful_update_time` 列. [#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 添加 `blockSerializedSize` 函数 (未压缩的磁盘大小). [#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) -* 添加函数 `moduloOrZero`. [#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([hcz](https://github.com/hczhcz)) -* 添加了系统表`system.zeros` 和`system.zeros_mt` 以及故事函数`zeros()` 和`zeros_mt()`. 表 (和表函数) 包含名称为 `zero` 且类型为 `UInt8` 的单列. 此列包含零. 出于测试目的需要它作为生成多行的最快方法. 这修复了 [#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### 实验功能 -* 在`MergeTree`-family 表中添加新的零件紧凑格式, 其中所有列都存储在一个文件中. 它有助于提高小而频繁的插入的性能. 旧格式 (每列一个文件) 现在称为宽格式. 数据存储格式由设置 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part` 控制. [#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([Anton Popov](https://github.com/CurtizJ)) -* 支持对 `Log`、`TinyLog` 和`StripeLog` 表的S3 存储. [#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([Pavel Kovalenko](https://github.com/Jokser)) - -#### Bug 修复 -* 修复了日志消息中不一致的空格. [#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复未命名元组数组在表创建时被展平为嵌套结构的错误. [#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) -* 修复了 `File` 表或 `file` 表函数中匹配 glob 模式的文件过多时, 可能会出现 `Too many open files` 错误的问题. 现在文件被懒惰地打开. 这修复了 [#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* DROP TEMPORARY TABLE 现在只删除临时表. [#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([Vitaly Baranov](https://github.com/vitlibar)) -* 当我们关闭服务器或 DETACH/ATTACH 表时删除过时的分区. [#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([Guillaume Tassery](https://github.com/YiuRULE)) -* 关于默认磁盘如何计算 `data` 子目录中的可用空间. 修复了如果将 `data` 目录安装到单独的设备 (罕见情况), 则无法正确计算可用空间量的问题. 这修复了 [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([Mikhail Korotov](https://github.com/millb)) -* 允许逗号 (交叉) 与 IN() 内连接. [#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) -* 如果 WHERE 部分中有 [NOT] LIKE 运算符, 则允许将 CROSS 重写为 INNER JOIN. [#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) -* 在启用设置 `distributed_aggregation_memory_efficient` 的情况下修复` GROUP BY` 后可能出现的错误结果. 修复 [#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 找到的键在缓存字典的度量中被计为遗漏. [#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 修复 [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598) 中引入的复制协议不兼容性. [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) -* 修复了 `ReplicatedMergeTree` 表启动时 `queue_task_handle` 上的竞争条件. [#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 令牌 `NOT` 在 `SHOW TABLES NOT LIKE` 查询中不起作用. [#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加范围检查功能 `h3EdgeLengthM`. 如果没有这个检查, 缓冲区溢出是可能的. [#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了多个参数 (超过 10 个) 的三元逻辑 OP 的批量计算中的错误. [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) -* 修复 PREWHERE 优化错误,可能导致段错误或 `Inconsistent number of columns got from MergeTreeRangeReader` 异常. [#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([Anton Popov](https://github.com/CurtizJ)) -* 修复意外的 `Timeout exceeded while reading from socket` 异常, 该异常在实际超过超时之前和启用查询分析器时在安全连接上随机发生. 同时添加 `connect_timeout_with_failover_secure_ms`设置 (默认100ms), 与 `connect_timeout_with_failover_ms` 类似, 但用于安全连接 (因为SSL握手比普通TCP连接慢). [#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([tavplubix](https://github.com/tavplubix)) -* 修复突变终止的错误, 当突变可能挂在 `parts_to_do=0` 和 `is_done=0` 状态时. [#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([alesapin](https://github.com/alesapin)) -* 使用带有 `partial_merge_join` 设置的新 ANY JOIN 逻辑. 现在可以使用 `partial_merge_join=1` 进行 `ANY|ALL|SEMI LEFT` 和 `ALL INNER` 连接. [#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) -* 分片现在将从发起者获得的设置限制到分片的约束, 而不是抛出异常. 此修复允许将查询发送到具有其他约束的分片. [#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([Vitaly Baranov](https://github.com/vitlibar)) -* 修复了 `MergeTreeReadPool` 中的内存管理问题. [#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 使用字符串 `e` 调用时修复 `toDecimal*OrNull()` 函数族. 修复[#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) -* 确保 `FORMAT Null` 没有向客户端发送数据. [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 修复 `LiveViewBlockInputStream` 中的时间戳不会更新的错误. `LIVE VIEW` 是一项实验性功能. [#8644](https://github.com/ClickHouse/ClickHouse/pull/8644) ([vxider](https://github.com/Vxider)) [#8625](https://github.com/ClickHouse/ClickHouse/pull/8625) ([vxider](https://github.com/Vxider)) -* 修复了不允许删除旧 TTL 表达式的 `ALTER MODIFY TTL` 错误行为. [#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复了 MergeTreeIndexSet 中的 UBSan 报告. 这修复了 [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了当 haystack 有零字节时 `match` 和 `extract` 函数的行为. 当 haystack 不变时, 行为是错误的. 这修复了 [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 避免从 Apache Avro 3rd 方库中的析构函数抛出. [#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([Andrew Onyshchuk](https://github.com/oandrew)) -* 不要提交部分从 `Kafka` 轮询的批处理, 因为它可能导致数据漏洞. [#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) -* 使用可为空的返回类型修复 `joinGet`. [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) [#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([Amos Bird](https://github.com/amosbird)) -* 修复使用 `T64`编解码器压缩时数据不兼容的问题. [#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2)) 修复`T64`压缩编解码器中导致的数据类型ID 在受影响的版本中错误 (解) 压缩 [#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) -* 添加设置 `enable_early_constant_folding` 并在某些导致错误的情况下禁用它. [#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) -* 使用 VIEW 修复下推谓词优化器并启用测试. [#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([Winter Zhang](https://github.com/zhang2014)) -* 修复 `Merge` 表中的段错误, 这在从 `File` 存储读取时可能发生. [#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([tavplubix](https://github.com/tavplubix)) -* 在 `ATTACH PARTITION FROM`、`REPLACE PARTITION`、`MOVE TO TABLE`中添加了对存储策略的检查. 否则会导致重启后部分数据无法访问, 导致ClickHouse无法启动. [#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 如果为表设置了 TTL, 则修复更改. [#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([Anton Popov](https://github.com/CurtizJ)) -* 修复在修改/添加/删除某些字典时执行 `SYSTEM RELOAD ALL DICTIONARIES` 时可能发生的竞争条件. [#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([Vitaly Baranov](https://github.com/vitlibar)) -* 在以前的版本 `Memory` 数据库引擎使用空数据路径, 所以表在 `path `目录 (例如 `/var/lib/clickhouse/`) 中创建, 而不是在数据库的数据目录中 (例如. `/var/lib/clickhouse/db_name`). [#8753](https://github.com/ClickHouse/ClickHouse/pull/8753) ([tavplubix](https://github.com/tavplubix)) -* 修复了有关缺少默认磁盘或策略的错误日志消息. [#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复 not(has()) 数组类型的bloom_filter索引. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -* 允许带有 `Log` 引擎的表中的第一列作为别名 [#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([Ivan](https://github.com/abyss7)) -* 在一个线程中从 `MergeTree` 表读取时修复范围的顺序. 它可能导致 `MergeTreeRangeReader` 的异常或错误的查询结果. [#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([Anton Popov](https://github.com/CurtizJ)) -* 使 `reinterpretAsFixedString` 返回 `FixedString` 而不是 `String`. [#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([Andrew Onyshchuk](https://github.com/oandrew)) -* 避免极少数情况下用户可能会收到错误的错误消息 (`Success`而不是详细的错误描述). [#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使用带有空行模板的 `Template` 格式时不会崩溃. [#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 系统表的元数据文件可能创建在错误的位置 [#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([tavplubix](https://github.com/tavplubix)) Fixes [#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). -* 修复缓存字典中 exception_ptr 的数据竞争 [#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 不要为查询 `ATTACH TABLE IF NOT EXISTS` 抛出异常. 以前, 如果表已经存在, 则抛出它, 尽管有 `IF NOT EXISTS` 子句. [#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([Anton Popov](https://github.com/CurtizJ)) -* 修复了异常消息中缺少关闭括号的问题. [#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在交互模式下启动 clickhouse-client 时避免消息 `Possible deadlock avoid` . [#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了 base64 编码值末尾填充可能格式错误的问题. 更新 base64 库. 这修复了 [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在读取后缀之后但提交之前发生异常时, 在极少数情况下防止在 `Kafka` 中丢失数据. 修复 [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) -* 修复了`DROP TABLE IF EXISTS` 中的异常. [#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([Nikita Vasilev](https://github.com/nikvas0)) -* 修复当用户尝试为旧格式的 `MergeTree` 表引擎系列 `ALTER MODIFY SETTING` 时崩溃. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) -* 支持 JSON 相关函数中不适合 Int64 的 UInt64 数字. 将 SIMDJSON 更新为 master. 这修复了[#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了使用非严格单调函数索引时反向谓词的执行. [#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([Alexander Kazakov](https://github.com/Akazz)) -* 不要尝试在 `GROUP BY` 中折叠 `IN` 常量. [#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([Amos Bird](https://github.com/amosbird)) -* 修复 `ALTER DELETE` 突变中导致索引损坏的错误. 这修复了 [#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) and [#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). Additionally fix extremely rare race conditions in `ReplicatedMergeTree` `ALTER` queries. [#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([alesapin](https://github.com/alesapin)) -* 当设置 `compile_expressions` 启用时,当我们使用 `Nullable` 类型时,你会在 `LLVMExecutableFunction` 中得到 `unexpected column`. [#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([Guillaume Tassery](https://github.com/YiuRULE)) -* `Kafka` 引擎的多项修复: 1) 修复在消费者组重新平衡期间出现的重复项. 2) 修复了当数据从多个分区轮询一次轮询并部分提交时出现的罕见 `holes` (现在我们总是处理/提交整个轮询的消息块). 3) 按块大小修复刷新 (在此之前, 只有超时刷新才能正常工作). 4) 更好的订阅程序 (带有分配反馈). 5) 使测试工作更快 (使用默认间隔和超时). 由于数据之前没有按块大小刷新 (根据文档应该如此), 该 PR 可能会导致默认设置的性能下降 (由于更频繁和更小的刷新不是最佳的). 如果您在更改后遇到性能问题 - 请将表中的 `kafka_max_block_size` 增加到更大的值 (例如 `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). 修复 [#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) -* 修复 PREWHERE 优化后某些查询中的 `Parameter out of bound` 异常. [#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Baudouin Giard](https://github.com/bgiard)) -* 修复了函数 `arrayZip` 的参数混合常量的情况. [#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 执行 `CREATE` 查询时,在存储引擎参数中折叠常量表达式. 用当前数据库替换空数据库名称. 修复 [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492) [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) ([tavplubix](https://github.com/tavplubix)) -* 现在无法创建或添加具有简单循环别名的列, 例如 `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) -* 修复了可能会损坏原始部分的双重移动错误。 如果您使用 `ALTER TABLE MOVE`, 这是相关的. [#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 允许 `interval` 标识符正确解析而无需反引号. 修复了即使 `interval` 标识符包含在反引号或双引号中也无法执行查询的问题. 这修复了 [#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了 `bitTestAll`/`bitTestAny` 函数的模糊测试和错误行为. [#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 当有很多行等于第 n 行时, 修复 `LIMIT n WITH TIES` 中可能的崩溃/错误的行数. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -* 修复使用启用的 `insert_quorum` 编写的部分的突变. [#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([alesapin](https://github.com/alesapin)) -* 修复 `Poco::HTTPServer` 破坏时的数据竞争. 当服务器启动并立即关闭时可能会发生这种情况 [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) -* 修复运行 `SHOW CREATE TABLE a_table_that_does_not_exist` 时显示误导性错误消息的错误. [#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) -* 当我们有一个 ORDER BY 和一个 LIMIT 子句时, 当我们在 SELECT 子句中有一个常量时, 在一些罕见的情况下修复了 `Parameters is out of bound` 异常. [#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([Guillaume Tassery](https://github.com/YiuRULE)) -* 修复突变完成, 当已经完成突变时, 可以有状态`is_done=0`. [#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([alesapin](https://github.com/alesapin)) -* 防止使用旧语法对 MergeTree 表执行 `ALTER ADD INDEX` , 因为它不起作用. [#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([Mikhail Korotov](https://github.com/millb)) -* 在服务器启动期间不要访问 `LIVE VIEW` 所依赖的表, 因此服务器将能够启动. 分离 `LIVE VIEW` 时也要删除 `LIVE VIEW` 依赖项. `LIVE VIEW` 是一项实验性功能. [#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([tavplubix](https://github.com/tavplubix)) -* 修复了在执行 PREWHERE 时在 MergeTreeRangeReader 中可能出现的段错误. [#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([Anton Popov](https://github.com/CurtizJ)) -* 使用列 TTL 修复可能不匹配的校验和. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) -* 修复了在只有一个卷的情况下部件不会通过 TTL 规则在后台移动的错误. [#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复了 `Method createColumn() is not implemented for data type Set` 实现的问题. 这修复了 [#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 现在我们将尝试更频繁地完成突变. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) -* 通过减去一个常量来修复 `intDiv`. [#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([hcz](https://github.com/hczhcz)) -* 修复 `BlockIO` 中可能的竞争条件. [#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复在尝试使用/删除使用错误参数创建的 `Kafka` 表时导致服务器终止的错误. [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) -* 如果 OS 为 `timer_create` 函数返回错误结果, 则添加了解决方法. [#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了使用 `min_marks_for_seek` 参数时的错误. 修复了分布式表中没有分片键并且我们尝试跳过未使用的分片时的错误消息. [#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) - -#### 改进 -* 在 `ReplicatedMergeTree*` 引擎系列的突变之上实施 `ALTER MODIFY/DROP` 查询. 现在 ALTERS 仅在元数据更新阶段阻塞, 之后不再阻塞. [#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([alesapin](https://github.com/alesapin)) -* 添加使用包含不合格名称的 `WHERE` 部分重写 CROSS 到 INNER JOIN 的能力. [#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) -* 使 `SHOW TABLES` 和 `SHOW DATABASES` 查询支持 `WHERE` 表达式和`FROM`/`IN`. [#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([sundyli](https://github.com/sundy-li)) -* 添加了一个设置 `deduplicate_blocks_in_dependent_materialized_views`. [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) -* 在最近的更改之后, MySQL 客户端开始以十六进制打印二进制字符串, 从而使它们不可读 ([#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). ClickHouse 中的解决方法是将字符串列标记为 UTF-8, 这并非总是如此, 但通常如此. [#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([Yuriy Baranov](https://github.com/yurriy)) -* 添加对 `sumMap` 的 String 和 FixedString 键的支持. [#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Baudouin Giard](https://github.com/bgiard)) -* 支持 SummingMergeTree 映射中的字符串键. [#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Baudouin Giard](https://github.com/bgiard)) -* 即使线程抛出异常, 也会向线程池发出线程终止信号. [#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* 允许在 `clickhouse-benchmark` 中设置 `query_id`. [#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([Anton Popov](https://github.com/CurtizJ)) -* 不允许在 `ALTER TABLE ... PARTITION partition` 查询中出现奇怪的表达式. 这个地址 [#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 表 `system.table_engines` 现在提供有关功能支持的信息 (例如 `supports_ttl` 或 `supports_sort_order`). [#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) -* 默认启用 `system.metric_log`. 它将包含具有 ProfileEvents 值的行, CurrentMetrics 以 `collect_interval_milliseconds` 间隔 (默认为一秒) 收集. 表很小 (通常以兆字节为单位), 默认收集这些数据是合理的. [#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries. Fixes [#6964](https://github.com/ClickHouse/ClickHouse/issues/6964) [#8874](https://github.com/ClickHouse/ClickHouse/pull/8874) ([Ivan](https://github.com/abyss7)) -* 现在临时的 `LIVE VIEW` 是由 `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` 创建的, 而不是 `CREATE TEMPORARY LIVE VIEW ...`, 因为之前的语法与 `CREATE TEMPORARY TABLE ...` 不一致. [#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([tavplubix](https://github.com/tavplubix)) -* 添加 text_log.level 配置参数以限制进入 `system.text_log` 表的条目. [#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) -* 允许根据 TTL 规则将下载的部分放入磁盘/卷. [#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 对于外部 MySQL 字典, 允许相互化 MySQL 连接池以在字典之间 `share` 它们. 此选项显着减少了与 MySQL 服务器的连接数. [#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) -* 在 `clickhouse-benchmark` 输出中显示分位数的最近查询执行时间, 而不是内插值. 最好显示与某些查询的执行时间相对应的值. [#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将数据插入 Kafka 时,可以为消息添加密钥和时间戳. 修复 [#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) -* 如果服务器从终端运行, 则按颜色突出显示线程编号、查询 ID 和日志优先级. 这是为了提高开发人员对相关日志消息的可读性. [#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 加载 `Ordinary` 数据库表时更好的异常消息. [#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为具有聚合函数状态的数组实现 `arraySlice`. 这修复了 [#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 允许在 IN 运算符的右侧使用常量函数和常量数组. [#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([Anton Popov](https://github.com/CurtizJ)) -* 如果在为 system.replicas 获取数据时发生 zookeeper 异常, 则将其显示在单独的列中. 这实现了 [#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在销毁时原子地删除 MergeTree 数据部分. [#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 支持分布式表的行级安全. [#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([Ivan](https://github.com/abyss7)) -* 现在我们在设置值中识别后缀 (如 KB、KiB...). [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([Mikhail Korotov](https://github.com/millb)) -* 在构建大型 JOIN 的结果时防止内存不足. [#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) -* 在 `clickhouse-client` 中以交互模式向建议添加集群名称. [#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为组中的所有线程初始化查询分析器, 例如它允许完全分析插入查询. [#8820](https://github.com/ClickHouse/ClickHouse/pull/8820) ([Ivan](https://github.com/abyss7)) -* 在 `system.query_log` 表中添加了列 `exception_code`. [#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([Mikhail Korotov](https://github.com/millb)) -* 在默认服务器配置文件中的端口 `9004` 上启用 MySQL 兼容性服务器. 配置示例中的固定密码生成命令. [#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([Yuriy Baranov](https://github.com/yurriy)) -* 如果文件系统是只读的,则防止在关机时中止. 这修复了 [#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 HTTP POST 查询中需要长度时更好的异常消息. [#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 `_path` 和 `_file` 虚拟列添加到 `HDFS` 和 `File` 引擎以及 `hdfs` 和 `file` 表函数. [#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) -* 如果将新列添加到视图的内部表中,则在插入到 `MATERIALIZED VIEW` 时修复错误 `Cannot find column` . [#8766](https://github.com/ClickHouse/ClickHouse/pull/8766) [#8788](https://github.com/ClickHouse/ClickHouse/pull/8788) ([vzakaznikov](https://github.com/vzakaznikov)) [#8788](https://github.com/ClickHouse/ClickHouse/issues/8788) [#8806](https://github.com/ClickHouse/ClickHouse/pull/8806) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 通过在最终更新 (如日志) 后发送进度来修复本地客户端 - 服务器协议的进度. 这可能仅与某些使用本机协议的第三方工具相关. [#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) -* 添加使用 MySQL 协议跟踪客户端连接数的系统指标. ([#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([Eugene Klimov](https://github.com/Slach)) -* 从现在开始, HTTP 响应将把 `X-ClickHouse-Timezone` 标头设置为 `SELECT timezone()` 将报告的相同时区值. [#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) - -#### 性能改进 -* 提高使用 IN 分析索引的性能. [#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([Anton Popov](https://github.com/CurtizJ)) -* 逻辑函数 + 代码清理中更简单、更高效的代码. [#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([Alexander Kazakov](https://github.com/Akazz)) -* 通过使用 C++20 功能确保更严格的别名, 整体性能改进 (受影响查询的范围为 5%..200%). [#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([Amos Bird](https://github.com/amosbird)) -* 比较函数的内部循环更严格的别名. [#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 算术函数的内部循环更严格的别名. [#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* ColumnVector::replicate() 的实现速度提高了约 3 倍, 通过它实现了 ColumnConst::convertToFullColumn(). 在实现常量的测试中也很有用. [#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([Alexander Kazakov](https://github.com/Akazz)) -* `ColumnVector::replicate()` 的另一个小性能改进 (这加速了 `materialize` 函数和高阶函数) . 进一步改进了 [#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([Alexander Kazakov](https://github.com/Akazz)) -* 改进了 `stochasticLinearRegression` 聚合函数的性能. 此补丁由英特尔提供. [#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 提高 `reinterpretAsFixedString` 函数的性能. [#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Do not send blocks to client for `Null` format in processors pipeline. [#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -#### 构建/测试/打包改进 -* 异常处理现在可以在适用于 Linux 的 Windows 子系统上正常工作. 参见 https://github.com/ClickHouse-Extras/libunwind/pull/3 . 这修复了 [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) -* 将 `readline` 替换为 `replxx` 以在 `clickhouse-client` 中进行交互式行编辑. [#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([Ivan](https://github.com/abyss7)) -* FunctionsComparison 中更好的构建时间和更少的模板实例化. [#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 CI 中添加了与 `clang-tidy` 的集成. 另见 [#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 现在我们在 CI 中使用 `lld` 链接 ClickHouse, 即使对于 `gcc`. [#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([alesapin](https://github.com/alesapin)) -* 允许在设置 `THREAD_FUZZER_*` 环境变量时随机化线程调度并插入故障. 这有助于测试. [#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在无状态测试中启用安全套接字. [#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([tavplubix](https://github.com/tavplubix)) -* 使 SPLIT_SHARED_LIBRARIES=OFF 更健壮. [#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) -* 使 `performance_introspection_and_logging` 测试对随机服务器卡住是可靠的. 这可能发生在 CI 环境中. 另见 [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在样式检查中验证 XML. [#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了测试 `00738_lock_for_inner_table` 中的竞争条件. 这个测试依赖于睡眠. [#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 删除类型为 `once` 的性能测试. 这是在统计比较模式下运行所有性能测试所必需的 (更可靠). [#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了算术函数的性能测试. [#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 增加了 `sumMap` 和 `summapwithverflow` 聚合函数的性能测试. 后续的 [#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 通过样式检查确保错误代码的样式. [#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为测试历史添加脚本. [#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([alesapin](https://github.com/alesapin)) -* 添加 GCC 警告 `-Wsuggest-override` 以定位和修复所有必须使用 `override` 关键字的地方. [#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* 忽略 Mac OS X 下的弱符号,因为它必须被定义. [#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([Deleted user](https://github.com/ghost)) -* 在性能测试中规范一些查询的运行时间. 这样做是为了准备在比较模式下运行所有性能测试. [#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复一些测试以支持 pytest 与查询测试. [#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([Ivan](https://github.com/abyss7)) -* 使用 MSan 在构建中启用 SSL, 因此在运行无状态测试时服务器不会在启动时失败. [#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([tavplubix](https://github.com/tavplubix)) -* 修复测试结果中的数据库替换. [#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) -* 为其他平台构建修复. [#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([proller](https://github.com/proller)) [#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([proller](https://github.com/proller)) [#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([proller](https://github.com/proller)) -* 将磁盘部分添加到 stateless-with-coverage 测试 docker 映像. [#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([Pavel Kovalenko](https://github.com/Jokser)) -* 使用 GRPC 构建时摆脱源代码树文件. [#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([Amos Bird](https://github.com/amosbird)) -* 通过从上下文中删除 SessionCleaner, 可以稍微加快构建时间. 让SessionCleaner的代码更简单. [#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更新了对 clickhouse-test 脚本中挂起查询的检查. [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz)) -* 从存储库中删除了一些无用的文件. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将数学性能测试的类型从 `once` 更改为 `loop` . [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 添加 docker 图像,它允许为我们的代码库构建交互式代码浏览器 HTML 报告. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/ClickHouse/dbms/index.html) -* 抑制 MSan 下的一些测试失败. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 加速 `exception while insert` 测试. 此测试经常在 debug-with-coverage 构建中超时. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 `libcxx` 和 `libcxxabi` 更新为 master. 准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复不稳定的测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 清理重复的链接器标志. 确保链接器不会查找意外的符号. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)) -* 将 `clickhouse-odbc` 驱动程序添加到测试图像中. 这允许通过其自己的 ODBC 驱动程序测试 ClickHouse 与 ClickHouse 的交互. [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) -* 修复单元测试中的几个错误. [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([alesapin](https://github.com/alesapin)) -* 启用 `-Wmissing-include-dirs` GCC 警告以消除所有不存在的包含 - 主要是由于 CMake 脚本错误. [#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* 如果查询分析器无法工作, 请描述原因. 这适用于 [#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 OpenSSL 更新为上游主服务器. 修复了 TLS 连接可能失败并显示消息 `OpenSSL SSL_read: error:14094438:SSLroutines:ssl3_read_bytes:tlsv1 alert internal error` 和 `SSL Exception: error:2400006E:random number generator::error retrieving entropy` 的问题. 该问题存在于 20.1 版中. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更新服务器的 Dockerfile. [#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) -* build-gcc-from-sources 脚本中的小修复. [#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) -* 在不使用 `number` 列的性能测试中, 将 `numbers` 替换为 `zeros`. 这将导致更干净的测试结果. [#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复在 Column 构造函数中使用 initializer_list 时的堆栈溢出问题. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)) -* 将 librdkafka 升级到 v1.3.0。 在 Mac OS X 上启用捆绑的 `rdkafka` 和 `gsasl` 库. [#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([Andrew Onyshchuk](https://github.com/oandrew)) -* 在 GCC 9.2.0 上构建修复. [#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) - - -## ClickHouse 版本 v20.1 - -### ClickHouse 版本 v20.1.16.120-stable 2020-60-26 - -#### Bug 修复 - -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. 续 [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 不允许在高阶函数中使用 arrayJoin. 它导致协议同步中断. 这关闭了 [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复像 `SELECT *, xyz.*` 这样的查询的意外行为, 这些查询在预期错误时成功. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* 修复了由 Values 输入格式中复杂文字的错误类型推导导致的 LOGICAL_ERROR. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* 修复 `ORDER BY ... WITH FILL` 在 const 列上. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* 与 XDBC 桥通信时传递适当的超时. 最近在检查桥接活性和接收元信息时没有遵守超时. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加对带有不区分大小写标志的正则表达式的支持. 这修复了 [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复字符串的布隆过滤器 (数据跳过索引). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* 修复在 prewhere 条件下使用 `Nullable` 列导致的罕见崩溃. (可能它以某种方式与 [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) 相关联). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 clickhouse-client 的错误退出代码, 当 exception.code() % 256 = 0 时. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* 修复服务器启动时有关 `Mark cache size was lowered` 的日志消息中的小错误. 这将关闭 [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 现在 clickhouse-server docker 容器将更喜欢 IPv6 检查服务器活动性. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* 使用 -State 函数在聚合过程中抛出异常时修复内存泄漏. 这修复了 [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了使用 `FINAL` 修饰符和 `ORDER BY` 优化包装到函数中的主键的使用. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse 版本 v20.1.15.109-stable 2020-06-19 - -#### Bug 修复 - -* 在更改期间修复结构的多余锁定. [#11790](https://github.com/ClickHouse/ClickHouse/pull/11790) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse 版本 v20.1.14.107-stable 2020-06-11 - -#### Bug 修复 - -* 修复错误 `Size of offsets does not match size of column` , 用于查询中的 `PREWHERE column in (subquery)` 和 `ARRAY JOIN` . [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). - - -### ClickHouse 版本 v20.1.13.105-stable 2020-06-10 - -#### Bug 修复 - -* 修复了在启用 `min_bytes_to_use_direct_io` 且 PREWHERE 处于活动状态并使用 SAMPLE 或大量线程时可能发生的 `Data compressed with different methods` 错误. 这修复了 [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复编解码器的返回压缩大小. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了当列具有非文字参数的压缩编解码器时服务器崩溃的问题.修复 [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* 以 nan 为点修复 pointInPolygon. 修复 [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* 修复了带有纬度/经度范围之外的参数的 geohashesInBox. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* 修复具有外部排序和限制的查询可能出现的 `Pipeline stuck` 错误. 修复 [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复 `quantilesExactWeightedArray` 中的崩溃问题. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 通过再次设置 `parallel_view_processing = 1` 并行写入 `MATERIALIZED VIEW` . 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当提取的 JSON 具有不平衡的 { 或 [ 字符串时, 修复visitParamExtractRaw. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* 修复 ThreadPool 中非常罕见的竞争条件. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复转换中潜在的未初始化内存. 示例: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了如果表的主键中有 Array 列并且查询正在使用 `empty` 或 `notEmpty` 函数按此列过滤时索引分析无法工作的问题. 这修复了 [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复当查询速度估计可能不正确时的错误, 并且如果查询受到 `max_network_bandwidth`、`max_execution_speed` 或 `priority` 设置的限制,`min_execution_speed` 的限制可能无法工作或无法正常工作. 将 `timeout_before_checking_execution_speed` 的默认值更改为非零值, 否则设置 `min_execution_speed` 和 `max_execution_speed` 无效. 这修复了 [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). 这修复了 [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). 这修复了 [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). 可用性改进: 避免在 `clickhouse-client` 中将异常消息与进度条串联. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复读取 Protobuf 格式错误数据时崩溃的问题. 这修复了 [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), 修复了 [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* 使用 `Array(Array(LowCardinality))` 捕获参数修复高阶函数的可能错误 `Cannot capture column` . [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 如果数据跳过索引依赖于将在后台合并期间修改的列 (对于 SummingMergeTree、AggregatingMergeTree 以及 TTL GROUP BY), 则计算错误. 此问题已通过合并后移动索引计算得到修复, 因此索引是在合并数据上计算的. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* 如果没有最终确定,则从突变终止任务中删除日志记录. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* 修复了 parseDateTime64BestEffort 参数解析错误. [#10925](https://github.com/ClickHouse/ClickHouse/issues/10925). [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* 修复方法 getRawData() 中不正确的原始数据大小. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* 修复与分布式表中元组的向后兼容性. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 StringHashTable 中的 SIGSEGV (如果这样的键不存在). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `ReplicatedMergeTree` 中的错误, 该错误可能导致 `OPTIMIZE` 查询上的某些 `ALTER` 在它变为非活动状态后挂起等待某些副本. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* 在 Block::sortColumns() 之后修复列顺序 (还添加了一个测试,表明它会影响一些实际用例 - 缓冲区引擎). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* 修复 ODBC 桥在不要求引用标识符时的问题. 这修复了[#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 DateLUT 中的 UBSan 和 MSan 报告. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* - 在关键条件下使用 `src_type` 进行正确的类型转换. 修复[#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* 修复 `parallel_view_processing` 行为. 现在, 如果发生异常, 所有对 `MATERIALIZED VIEW` 的插入都应该无一例外地完成. 修复 [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 与 -State 结合时修复组合器 -OrNull 和 -OrDefault. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* 修复消失的总数. 如果查询具有连接或具有外部 where 条件的子查询, 则总计可能已被过滤. 修复 [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 在一个查询中使用相同的集合修复 `IN` 运算符的多次使用. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). -* 修复 AggregateTransform 构造函数中参数的顺序. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* 修复启用了 `distributed_aggregation_memory_efficient` 的远程查询缺乏并行执行的问题. 修复[#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复分布式查询的谓词优化 (`enable_optimize_predicate_expression=1`) 对于带有 `HAVING` 部分的查询 (即需要在服务器启动器上进行过滤时), 通过保留表达式的顺序 (这足以修复), 并且还强制 聚合器在索引上使用列名. 修复: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). -* 修复错误 `the BloomFilter false positive must be a double number between 0 and 1`. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). -* 修复 SELECT 列 ALIAS 的默认表达式类型与列类型不同. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* 实现了 DateTime64 和 String 值之间的比较 (就像 DateTime 一样). [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). - - -### ClickHouse 版本 v20.1.12.86, 2020-05-26 - -#### Bug 修复 - -* 修复了 20.1 及更早版本之间两级聚合的不兼容问题. 当在发起节点和远程节点上使用不同版本的 ClickHouse 并且 GROUP BY 结果的大小很大并且由单个 String 字段执行聚合时, 就会发生这种不兼容. 结果导致单个键的多个未合并的行. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了合并后可能发生的 `SummingMergeTree` 中 `LowCardinality(FixedString)` 键列的数据损坏. 修复[#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了当 `readonly=2` 和 `cancel_http_readonly_queries_on_client_close=1` 时导致 http 请求卡在客户端关闭的错误. 修复 [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* 修复了在 `SYSTEM DROP DNS CACHE` 查询也删除缓存时的错误, 用于检查是否允许用户从某些 IP 地址连接. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). -* 修复了 `MATERIALIZED VIEW` 内部查询中不正确的标量结果, 以防该查询包含相关表. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复所有部分 mutation 完成, 但是在 `is_done=0` 中挂掉的情况. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). -* 修复了 unix 纪元开始时与 UTC 有小数偏移的时区的溢出. 这修复了 [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了分布式存储的不当关闭. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `simpleLinearRegression` 中大整数上的数字溢出. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). -* 修复了附加数据库失败时删除元数据目录的问题. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)). -* 在创建 `BloomFilter` 索引时添加了对参数数量和类型的检查 [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). -* 修复了使用 `ARRAY JOIN`、`ORDER BY` 和 `LIMIT` 的查询可能返回不完整结果的问题. 这修复了 [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 更喜欢 `fallback_to_stale_replicas` 而不是 `skip_unavailable_shards`. [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)). -* 修复了 `Array(Tuple(...))` 数据类型的错误扁平化. 这修复了[#10259](https://github.com/ClickHouse/ClickHouse/issues/10259). [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了在尝试从缓冲区读取 HashMap 时导致编译错误的 `HashTable` 中的错误行为. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)). -* 修复了 `ConcatProcessor` 中可能发生的 `Pipeline stuck` 错误, 该错误可能发生在远程查询中. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 使用 `max_rows_to_group_by` 和 `group_by_overflow_mode = 'break'` 修复了错误 `Pipeline stuck`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了一些错误, 当某些数据以法定人数插入, 然后以某种方式删除 (DROP PARTITION, TTL) 时, 这会导致插入卡住或 SELECT 中的误报异常. 这修复了 [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946). [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* 修复了在远程服务器上使用 18.12.17 之前的版本而在启动服务器上使用较新版本时不兼容的问题, 以及 GROUP BY 固定和非固定键, 以及激活两级分组方法时. [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### 构建/测试/打包改进 - -* 将 CA 证书添加到 clickhouse-server docker 镜像. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). - - -### ClickHouse 版本 v20.1.10.70, 2020-04-17 - -#### Bug 修复 - -* 修复罕见的可能异常 `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复了当用户尝试在带有 `ENGINE = Replicated*` 的表上运行 `ALTER UPDATE/DELETE` 时, ClickHouse 会抛出 `Unknown function lambda.` 错误消息的错误. 检查非确定性函数现在可以正确处理 lambda 表达式. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). -* 修复 RFC-2822 中字符串的 `parseDateTimeBestEffort`, 当一周中的某一天是星期二或星期四时. 这修复了 [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复 `JOIN` 内的常量列名可能与 `JOIN` 外的常量名冲突. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 修复了可能的无限查询执行, 当查询实际上应该在LIMIT上停止, 而从无限源读取, 如 `system.numbers` 或 `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 在存在`array Join`函数的情况下修复移动到prewhere优化 (在某些情况下). 这修复了[#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 添加放宽对使用 `allow_nondeterministic_mutations` 设置的突变中非确定性函数使用限制的能力. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). -* 如果结构在 `INSERT` 上不匹配,则将块转换为带有 `Distributed` 引擎的表. [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). -* 当其结构与底层表不同时, 将 `INSERT` 上的 `SIGSEGV` 修复到 `Distributed` 表中. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). -* 使用 `JOIN` 和 `UNION ALL` 修复查询可能丢失的行. 修复 [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 为 MySQL 数据库引擎添加参数检查和支持标识符参数. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). -* 修复来自 localhost clickhouse 服务器的 clickhouse 字典源中的错误. 如果字典和源中的类型不兼容, 该错误可能会导致内存损坏. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). -* 修复错误 `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. 当设置 `distributed_aggregation_memory_efficient` 被启用, 并且分布式查询从不同分片读取不同级别的聚合数据时发生这种情况 (混合单级和二级聚合). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 修复在包含尾随零字节的字符串键上的 `GROUP BY` 中可能发生的分段错误 ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* 修复了在某些数据库查询的处理阶段之一未检索到必要表的错误. 修复 [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). -* 修复 `JOIN` 与 `TOTALS` 一起出现时的 `Not found column in block` 错误. 修复 [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复 `ON CLUSTER` DDL 查询在服务器启动时冻结的错误. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). -* 修复连接表引擎的 `TRUNCATE` ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). -* 修复 ALTER 查询中的 `scalar does not exist` 错误 ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). -* 修复 `ReplicatedMergeTree` 中删除和优化之间的竞争条件. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). -* 修复了 `ATTACH PART` 中的 `DeleteOnDestroy` 逻辑, 这可能导致自动删除附加部分并添加了一些测试. [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### 构建/测试/打包改进 - -* 修复单元测试 `collapsing_sorted_stream`. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)). - -### ClickHouse 版本 v20.1.9.54, 2020-03-28 - -#### Bug 修复 - -* 修正了在分布式表上有 `PREWHERE` 和 `WHERE` 以及 `SET distributed_product_mode = 'local'` 时出现的 `Different expressions with the same alias` 错误. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). -* 修复具有复合主键的表的突变过多的内存消耗. 这修复了 [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). -* 对于 INSERT 查询, 分片现在将从发起者获得的设置限制在分片的约束中, 而不是抛出异常. 此修复允许将 `INSERT` 查询发送到具有另一个约束的分片. 此更改改进了修复 [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). -* 修复客户端上可能出现的异常 `Got 0 in totals chunk, expected 1` on client. 如果右连接表有零行, 则使用“JOIN”进行查询时会发生这种情况. 示例: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. 修复 [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 当类型无法转换时, 使用 `optimize_skip_unused_shards` 修复 `SIGSEGV`. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). -* 修复了函数参数的时区使用不正确的一些情况. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). - -#### 改进 - -* 从突变中移除 `ORDER BY` 阶段, 因为我们从单个线程中的单个有序部分读取. 还添加检查突变中的行顺序是否按排序键顺序排序并且不违反此顺序. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). - -#### 构建/测试/打包改进 - -* 清理重复的链接器标志. 确保链接器不会查找意外的符号. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)). - -### ClickHouse 版本 v20.1.8.41, 2020-03-20 - -#### Bug 修复 -* 修复可能的永久性 `Cannot schedule a task` 错误 (由于 `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread` 中的未处理异常). 这修复了 [#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) -* 修复 `ALTER` 查询 (突变) 中的过多内存消耗. 这修复了 [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) -* 修复外部字典 DDL 中反引号的错误. 这修复了 [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) - -### ClickHouse 版本 v20.1.7.38, 2020-03-18 - -#### Bug 修复 -* 修复了 `sumKahan` 和 `sumWithOverflow` 不正确的内部函数名称. 在远程查询中使用此函数时导致异常. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). This issue was in all ClickHouse releases. -* 允许具有内部复制的 `Distributed` 表的 `ALTER ON CLUSTER` . 这修复了[#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). This issue was in all ClickHouse releases. -* 修复 `MergeTreeRangeReader` 中可能出现的异常 `Size of filter does not match size of column` 和 `Invalid number of rows in Chunk` . 在某些情况下, 它们可能会在执行 `PREWHERE` 时出现. 修复 [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) -* 修正了这个问题: 如果你写一个简单的算术表达式, 比如 `time + 1` (与像 `time + INTERVAL 1 SECOND` 这样的表达式相反), 时区不会被保留. 这修复了 [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov))。 此问题存在于所有 ClickHouse 版本中. -* 现在无法创建或添加具有简单循环别名的列, 例如 `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) -* 修复了 base64 编码值末尾填充可能格式错误的问题. 更新 base64 库. 这修复了 [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 `Poco::HTTPServer` 破坏时的数据竞争. 服务器启动并立即关闭时可能会发生这种情况. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) -* 当有很多行等于第 n 行时, 修复 `LIMIT n WITH TIES` 中可能的崩溃/错误的行数. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -* 使用列 TTL 修复可能不匹配的校验和. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) -* 修复当用户尝试为旧格式的 `MergeTree` 表引擎系列 `ALTER MODIFY SETTING` 时崩溃. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) -* 现在我们将尝试更频繁地完成突变. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) -* 修复 [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598) 中引入的复制协议不兼容问题. [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) -* 修复 not(has()) 数组类型的bloom_filter索引. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -* 修复了当 haystack 有零字节时`match` 和`extract` 函数的行为. 当 haystack 不变时, 行为是错误的. 这修复了 [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 构建/测试/打包改进 - -* 异常处理现在可以在适用于 Linux 的 Windows 子系统上正常工作. 参见 https://github.com/ClickHouse-Extras/libunwind/pull/3 这修复了 [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) - - -### ClickHouse 版本 v20.1.6.30, 2020-03-05 - -#### Bug 修复 - -* 修复使用 `T64` 编解码器压缩时数据不兼容的问题. -[#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) -* 在一个线程中从 MergeTree 表读取时修复范围的顺序. 修复 [#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). -[#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ)](https://github.com/CurtizJ) -* 修复在执行 `PREWHERE` 时在 `MergeTreeRangeReader` 中可能出现的段错误. 修复[#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). -[#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ)](https://github.com/CurtizJ) -* 修复 `reinterpretAsFixedString` 以返回 `FixedString` 而不是 `String`. -[#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) -* 使用可为空的返回类型修复 `joinGet`. 修复 [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) -[#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) -* 修复模糊测试和 bitTestAll/bitTestAny 函数的错误行为. -[#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* 修复 haystack 为零字节时匹配和提取函数的行为. 当 haystack 不变时, 行为是错误的. 修复 [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) -[#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* 修复了使用非严格单调函数索引时反向谓词的执行. 修复 [#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) -[#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz)](https://github.com/Akazz) -* 如果 `WHERE` 部分中有 `[NOT] LIKE` 运算符, 则允许将 `CROSS` 重写为 `INNER JOIN`. 修复 [#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) -[#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) -* 允许带有日志引擎的表中的第一列作为别名. -[#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) -* 允许在内部使用逗号连接 `IN()`. 修复 [#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). -[#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) -* 改进`ALTER MODIFY/ADD`查询逻辑. 现在你不能在没有类型的情况下 `ADD` 列, `MODIFY` 默认表达式不会改变列的类型, `MODIFY` 类型不会丢失默认表达式值. 修复 [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). -[#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) -* 修复突变完成, 当已经完成突变时, 状态可以为 is_done=0. -[#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) -* 支持 system.numbers 和 system.numbers_mt 的 `Processors` 管道. 这也修复了不遵守 `max_execution_time` 时的错误. -[#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) -* 修复 `DictCacheKeysRequestedFound` 指标的错误计数. -[#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) -* 在`ATTACH PARTITION FROM`、`REPLACE PARTITION`、`MOVE TO TABLE` 中添加了对存储策略的检查, 否则可能会使重启后部分数据无法访问并阻止ClickHouse启动. -[#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) -* 修复了 `MergeTreeIndexSet` 中的 UBSan 报告. 这修复了 [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) -[#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* 修复 BlockIO 中可能的数据竞争. -[#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) -* 支持在 JSON 相关函数中不适合 Int64 的 `UInt64` 数字. 将 `SIMDJSON` 更新为 master. 这修复了 [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) -[#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* 修复了如果数据目录安装到单独的设备时无法正确计算可用空间量的问题. 对于默认磁盘, 从数据子目录计算可用空间. 这修复了 [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) -[#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(millb)](https://github.com/millb) -* 修复 TLS 连接可能失败并显示消息 `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error and SSL Exception: error:2400006E:random number generator::error retrieving entropy.` . 更新 OpenSSL 到上游掌握. -[#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* 执行 `CREATE` 查询时, 在存储引擎参数中折叠常量表达式. 用当前数据库替换空数据库名称. 修复 [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508)、[#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). 还修复了 ClickHouseDictionarySource 中本地地址的检查. -[#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) -* 修复 `StorageMerge` 中的段错误, 这在从 StorageFile 读取时可能发生 -[#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) -* 在读取后缀之后但提交之前发生异常时, 在极少数情况下防止在 `Kafka` 中丢失数据. 修复[#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Related: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) -[#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) -* 修复在尝试使用/删除使用错误参数创建的 `Kafka` 表时导致服务器终止的错误. 修复 [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). -[#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) - -#### 新特征 -* 添加 `deduplicate_blocks_in_dependent_materialized_views` 选项以控制幂等插入到具有物化视图的表中的行为. 应 Altinity 的特殊要求, 此新功能已添加到错误修复版本中. -[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) - -### ClickHouse 版本 v20.1.2.4, 2020-01-22 - -#### 向后不兼容变更 -* 使设置 `merge_tree_uniform_read_distribution` 过时. 服务器仍然可以识别此设置, 但没有效果. [#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将函数 `greatCircleDistance` 的返回类型更改为 `Float32` , 因为现在计算的结果是 `Float32` . [#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 现在预计查询参数以 `escaped` 格式表示. 例如, 要传递字符串 `ab` , 您必须在URL 中分别写入 `a\tb` 或 `a\b` 以及 `a%5Ctb` 或 `a%5C%09b`. 这需要添加将 NULL 作为 `\N` 传递的可能性. 这修复了 [#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 默认情况下为 `ReplicatedMergeTree` 启用 `use_minimalistic_part_header_in_zookeeper` 设置. 这将显着减少存储在 ZooKeeper 中的数据量. 从 19.1 版开始支持此设置, 我们已经在多个服务的生产中使用它半年多没有任何问题. 如果您有机会降级到 19.1 之前的版本, 请禁用此设置. [#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 数据跳过索引已准备好生产并默认启用. 设置 `allow_experimental_data_skipping_indices`、`allow_experimental_cross_to_join_conversion` 和 `allow_experimental_multiple_joins_emulation` 现在已经过时并且什么都不做. [#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为与 `JOIN` 操作一致的 `StorageJoin` 添加新的 `ANY JOIN` 逻辑. 要在不改变行为的情况下进行升级, 您需要将 `SETTINGS any_join_distinct_right_table_keys = 1` 添加到引擎连接表元数据或在升级后重新创建这些表. [#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) -* 需要重新启动服务器以应用日志配置中的更改. 这是一个临时解决方法, 以避免服务器记录到已删除的日志文件的错误 (请参阅 [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -#### 新特征 -* 向 `system.merges` 添加了有关部件路径的信息. [#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 添加在 `ON CLUSTER` 模式下执行 `SYSTEM RELOAD DICTIONARY` 查询的能力. [#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([Guillaume Tassery](https://github.com/YiuRULE)) -* 添加在 `ON CLUSTER` 模式下执行 `CREATE DICTIONARY` 查询的能力. [#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([alesapin](https://github.com/alesapin)) -* 现在用户在 `users.xml` 中的配置文件可以继承多个配置文件. [#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -* 添加了 `system.stack_trace` 表, 允许查看所有服务器线程的堆栈跟踪. 这对于开发人员自省服务器状态很有用. 这修复了[#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加具有可配置亚秒级精度的 `DateTime64` 数据类型. [#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([Vasily Nemkov](https://github.com/Enmk)) -* 添加表函数 `clusterAllReplicas`, 允许查询集群中的所有节点. [#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([kiran sunkari](https://github.com/kiransunkari)) -* 添加聚合函数 `categoricalInformationValue`, 用于计算离散特征的信息值. [#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([hcz](https://github.com/hczhcz)) -* 通过并行执行来加速对 `CSV`、`TSV` 和 `JSONEachRow` 格式的数据文件的解析. [#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 添加执行银行家舍入的函数`bankerRound`. [#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([hcz](https://github.com/hczhcz)) -* 在区域名称的嵌入式字典中支持更多语言: 'ru', 'en', 'ua', 'uk', 'by', 'kz', 'tr', 'de', 'uz', 'lv', 'lt', 'et', 'pt', 'he', 'vi'. [#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 改进了 `ANY JOIN` 逻辑的一致性. 现在 `t1 ANY LEFT JOIN t2` 等于 `t2 ANY RIGHT JOIN t1`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* 添加设置 `any_join_distinct_right_table_keys` 以启用 `ANY INNER JOIN` 的旧行为. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* 添加新的 `SEMI` 和 `ANTI JOIN`. 旧的 `ANY INNER JOIN` 行为现在可用作 `SEMI LEFT JOIN` . [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* 为 `File` 引擎和 `file` 表功能添加了 `Distributed` 格式, 允许从异步插入到 `Distributed `表中生成的 `.bin` 文件中读取. [#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 为 `runningAccumulate` 添加可选的重置列参数, 允许重置每个新键值的聚合结果. [#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([Sergey Kononenko](https://github.com/kononencheg)) -* 添加使用 ClickHouse 作为 Prometheus 端点的功能. [#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) -* 在 `config.xml` 中添加 `` 部分, 限制远程表引擎和表函数 `URL`、`S3`、`HDFS`的允许主机. [#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([Mikhail Korotov](https://github.com/millb)) -* 添加了函数 `greatCircleAngle`, 以度数计算球体上的距离. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更改地球半径与 H3 库一致. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为输入和输出添加了 `JSONCompactEachRow` 和 `JSONCompactEachRowWithNamesAndTypes` 格式. [#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([Mikhail Korotov](https://github.com/millb)) -* 添加了与文件相关的表引擎和表函数 (`File`、`S3`、`URL`、`HDFS`) 的功能, 允许根据附加引擎参数或文件扩展名读写 `gzip` 文件. [#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([Andrey Bodrov](https://github.com/apbodrov)) -* 添加了 `randomASCII(length)` 函数, 生成一个带有随机 [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) 可打印字符集的字符串. [#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([BayoNet](https://github.com/BayoNet)) -* 添加了函数 `JSONExtractArrayRaw`, 它从 `JSON` 字符串返回未解析的 json 数组元素的数组. [#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) -* 添加 `arrayZip` 函数, 该函数允许将多个长度相等的数组组合成一个元组数组. [#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([Winter Zhang](https://github.com/zhang2014)) -* 添加根据为 `*MergeTree` 表引擎系列配置的 `TTL` 表达式在磁盘之间移动数据的能力. [#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 添加了新的聚合函数 `avgWeighted`, 它允许计算加权平均值. [#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([Andrey Bodrov](https://github.com/apbodrov)) -* 现在默认为 `TSV`、`TSKV`、`CSV` 和 `JSONEachRow` 格式启用并行解析. [#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 从`H3` 库添加几个地理函数: `h3GetResolution`、`h3EdgeAngle`、`h3EdgeLength`、`h3IsValid` 和 `h3kRing`. [#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) -* 在文件相关的存储和表函数中添加了对 brotli (`br`) 压缩的支持. 这修复了 [#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为 `SimpleAggregationFunction` 类型添加 `groupBit*` 函数. [#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([Guillaume Tassery](https://github.com/YiuRULE)) - -#### Bug 修复 -* 使用 `Distributed` 引擎修复表的重命名. 修复问题 [#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) -* 现在字典支持非 ClickHouse SQL 方言中任意字符串中的属性的 `EXPRESSION`. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) -* 修复损坏的 `INSERT SELECT FROM mysql(...)` 查询. 这修复了 [#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) 和 [#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) -* 修复从 `JSONEachRow` 插入默认 `Tuple` 时出现的错误 `Mismatch column sizes`. 这修复了[#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([tavplubix](https://github.com/tavplubix)) -* 现在,如果将 `WITH TIES` 与 `LIMIT BY` 一起使用, 则会引发异常. 还添加了将 `TOP` 与 `LIMIT BY` 一起使用的功能. 这修复了 [#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 在 `clickhouse-odbc-bridge` 二进制文件中修复来自新 glibc 版本的意外依赖. [#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([Amos Bird](https://github.com/amosbird)) -* 修复 `*MergeTree` 引擎家族的检查功能错误. 现在, 如果我们在最后一个颗粒和最后一个标记 (非最终) 中有相同数量的行, 它不会失败. [#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([alesapin](https://github.com/alesapin)) -* 当基础数字类型等于表指定类型时, 修复在 `ALTER` 查询后插入 `Enum*` 列. 这修复了 [#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([Anton Popov](https://github.com/CurtizJ)) -* 允许函数 `substring` 的非常量负 `size` 参数. 这是不允许的. 这修复了 [#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复传递给 `(O|J)DBC` 表引擎的参数数量错误时的解析错误. [#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([alesapin](https://github.com/alesapin)) -* 将日志发送到 syslog 时使用正在运行的 clickhouse 进程的命令名称. 在以前的版本中, 使用空字符串代替命令名称. [#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) -* 修复对 `localhost` 允许主机的检查. 此 PR 修复了 [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) 中提供的解决方案. [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) -* 修复在 `argMin` 和 `argMax` 函数中, 当结果用于 `runningAccumulate` 函数时, 长字符串参数的罕见崩溃. 这修复了 [#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([dinosaur](https://github.com/769344359)) -* 使用 `Buffer` 引擎修复表的内存过量使用. [#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) -* 修复了函数中可以将 `NULL` 作为参数之一并返回非 NULL 的潜在错误. [#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 线程池中更好的度量计算, 用于 `MergeTree` 表引擎的后台进程. [#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 当存在行级表过滤器时, 在 `WHERE` 语句中修复函数 `IN`. 修复 [#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([Ivan](https://github.com/abyss7)) -* 现在, 如果未完全解析设置值的整数值, 则会引发异常. [#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([Mikhail Korotov](https://github.com/millb)) -* 修复使用聚合函数查询具有两个以上本地分片的分布式表时的异常. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) -* 现在布隆过滤器可以处理零长度数组并且不执行冗余计算. [#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([achimbab](https://github.com/achimbab)) -* 修复了通过将客户端主机与 `users.xml` 中指定的 `host_regexp` 匹配来检查是否允许客户端主机. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([Vitaly Baranov](https://github.com/vitlibar)) -* 放宽不明确的列检查, 导致多个 `JOIN ON` 部分出现误报. [#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复了当服务器无法以 `JSON` 或 `XML` 格式发送或写入数据为 `String` 数据类型 (需要 `UTF-8` 验证) 或压缩时可能出现的服务器崩溃 (`std::terminate`) Brotli 算法或其他一些罕见情况下的结果数据. 这修复了 [#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 CI 发现的 `StorageDistributedDirectoryMonitor` 中的竞争条件. 这修复了 [#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 现在 `*MergeTree` 表引擎族中的后台合并更准确地保留存储策略卷顺序. [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 现在表引擎 `Kafka` 可以正常使用 `Native` 格式. 这修复了[#6731](https://github.com/ClickHouse/ClickHouse/issues/6731) [#7337](https://github.com/ClickHouse/ClickHouse/issues/7337) [#8003](https://github.com/ClickHouse/ClickHouse/issues/8003). [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -* 修复了带有标题的格式 (如 `CSVWithNames`), 这些格式会为表引擎 `Kafka` 抛出有关 EOF 的异常. [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -* 修复了从 `IN` 部分右侧的子查询进行设置的错误. 这修复了[#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) and [#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 修复从存储 `File` 读取时可能发生的崩溃. [#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 固定读取包含类型为 `list` 的列的 `arquet` 格式的文件. [#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([maxulan](https://github.com/maxulan)) -* 如果`max_parallel_replicas > 1`, 则使用依赖于采样键的 `PREWHERE` 条件修复分布式查询的错误 `Not found column` . [#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 如果查询使用依赖于表别名的 `PREWHERE`, 并且由于主键条件导致结果集为空, 则修复错误 `Not found column`. [#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复了函数 `rand` 和 `randConstant` 在 `Nullable` 参数的情况下的返回类型. 现在函数总是返回 `UInt32` 而不是 `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 禁用了`WITH FILL` 表达式的谓词下推. 这修复了[#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([Winter Zhang](https://github.com/zhang2014)) -* 修复了使用 `FINAL` 部分时 `SummingMergeTree` 的错误 `count()` 结果. [#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 修复来自远程服务器的常量函数可能的错误结果. 它发生在具有诸如 `version()`、`uptime()` 等函数的查询中, 这些函数为不同的服务器返回不同的常量值. 这修复了 [#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复下推谓词优化中导致错误结果的复杂错误. 这修复了很多关于下推谓词优化的问题. [#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([Winter Zhang](https://github.com/zhang2014)) -* 修复 `CREATE TABLE .. AS dictionary` 查询中的崩溃. [#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) -* `.g4` 文件中的多项改进 ClickHouse 语法. [#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([taiyang-li](https://github.com/taiyang-li)) -* 修复导致带有引擎 `Join` 的表在 `JOIN` 中崩溃的错误. 这修复了 [#7556](https://github.com/ClickHouse/ClickHouse/issues/7556) [#8254](https://github.com/ClickHouse/ClickHouse/issues/8254) [#7915](https://github.com/ClickHouse/ClickHouse/issues/7915) [#8100](https://github.com/ClickHouse/ClickHouse/issues/8100). [#8298](https://github.com/ClickHouse/ClickHouse/pull/8298) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复冗余字典重新加载到 `CREATE DATABASE` . [#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) -* 限制从 `StorageFile` 和 `StorageHDFS` 读取的最大流数。 修复 [#7650](https://github.com/ClickHouse/ClickHouse/issues/7650). [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) -* 当用户同时指定默认表达式和编解码器时, 修复 `ALTER ... MODIFY ... CODEC` 查询中的错误. 修复 [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([alesapin](https://github.com/alesapin)) -* 修复具有 `SimpleAggregateFunction(LowCardinality)` 类型的列的后台合并错误 [#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 固定类型检查函数 `toDateTime64`. [#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([Vasily Nemkov](https://github.com/Enmk)) -* 现在服务器不会在 `LEFT` 或 `FULL JOIN` 和Join 引擎和不支持的 `join_use_nulls` 设置上崩溃. [#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) -* 现在, 如果 `db` 不存在, `DROP DICTIONARY IF EXISTS db.dict` 查询不会抛出异常. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) -* 修复由于使用对已删除的 `IStorage` 对象的引用而导致的表函数 (`file`、`mysql`、`remote`)中可能的崩溃. 修复插入表函数时指定的列的错误解析. [#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([tavplubix](https://github.com/tavplubix)) -* 在启动 `clickhouse-server` 之前确保网络已启动. 这修复了 [#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([Zhichang Yu](https://github.com/yuzhichang)) -* 修复安全连接的超时处理, 因此查询不会无限期挂起. 这修复了[#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复并发工作者之间的 `clickhouse-copier` 冗余争用. [#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* 现在突变不会跳过附加的部分, 即使它们的突变版本大于当前的突变版本. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) -* 移动到另一个磁盘和服务器重启后, 忽略 `*MergeTree` 数据部分的冗余副本. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 使用 `JOIN` 键中的 `Low Cardinality` 修复 `FULL JOIN` 中的崩溃. [#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) -* 禁止在插入查询中多次使用列名, 例如 `INSERT INTO tbl (x, y, x)`. 这修复了[#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([alesapin](https://github.com/alesapin)) -* 添加回退以检测未知 CPU 的物理 CPU 内核数 (使用逻辑 CPU 内核数) . 这修复了 [#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复物化列和别名列的 `There's no column` 错误. [#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复了在没有 `TABLE` 或 `DICTIONARY` 限定符的情况下使用 `EXISTS` 查询时服务器崩溃的问题. 就像 `EXISTS t`。 这修复了 [#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). 这个错误是在 19.17 版本中引入的. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了使用 `SimpleAggregateFunction` 列时可能出现的带有错误 `"Sizes of columns not match"` 的罕见错误. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) -* 修复具有空 `allow_databases` 的用户可以访问所有数据库的错误 (对于 `allow_dictionaries` 也是如此). [#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) -* 当服务器已经与客户端断开连接时修复客户端崩溃. [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) -* 在按主键前缀和非主键后缀排序的情况下修复 `ORDER BY` 行为. [#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([Anton Popov](https://github.com/CurtizJ)) -* 检查表中是否存在合格的列. 这修复了 [#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) -* 修复了在合并完成移动指定的超部分后立即运行的 `ALTER MOVE` 行为. 修复 [#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复使用具有不同列数的 `UNION` 时可能出现的服务器崩溃. 修复 [#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 修复具有负大小的函数 `substr` 的结果子字符串的大小. [#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 现在,如果后台池中没有足够的空闲线程, 服务器不会在 `MergeTree` 中执行部分突变. [#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([tavplubix](https://github.com/tavplubix)) -* 修复格式化`UNION ALL` AST 时的一个小错误. [#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([litao91](https://github.com/litao91)) -* 修复了负数的错误布隆过滤器结果. 这修复了 [#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([Winter Zhang](https://github.com/zhang2014)) -* 修复了解压缩中潜在的缓冲区溢出. 恶意用户可以传递伪造的压缩数据, 这将导致缓冲区后读取. 这个问题是由 Yandex 信息安全团队的 Eldar Zaitov 发现的. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复由于 `arrayIntersect` 中整数溢出而导致的错误结果. [#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 现在 `OPTIMIZE TABLE` 查询不会等待离线副本执行操作. [#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) -* 修复了 `Replicated*MergeTree` 表的 `ALTER TTL` 解析器. [#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复服务器和客户端之间的通信, 以便服务器在查询失败后读取临时表信息. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) -* 修复聚合位图和标量位图相交时的 `bitmapAnd` 函数错误. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) -* 根据修复了 `clickhouse-cluster-copier` 中的错误的 ZooKeeper 程序员指南细化 `ZXid` 的定义. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* `odbc` 表函数现在尊重 `external_table_functions_use_nulls` 设置. [#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([Vasily Nemkov](https://github.com/Enmk)) -* 修复了导致罕见数据竞争的错误. [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) -* 现在`SYSTEM RELOAD DICTIONARY` 完全重新加载字典, 忽略`update_field`. 这修复了 [#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([Vitaly Baranov](https://github.com/vitlibar)) -* 添加检查创建查询中是否存在字典的功能. [#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([alesapin](https://github.com/alesapin)) -* 以 `Values` 格式修复 `Float*` 解析. 这修复了 [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) -* 修复当我们无法在 `*MergeTree` 表引擎系列的某些后台操作中保留空间时崩溃. [#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 修复当表包含 `SimpleAggregateFunction(LowCardinality)` 列时合并操作的崩溃. 这修复了 [#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) -* 恢复对所有 ICU 语言环境的支持, 并添加对常量表达式应用排序规则的功能. 还将语言名称添加到 `system.collations` 表. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) -* 修复了最小生命周期为零 (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`) 的外部字典不在后台更新时的错误. [#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([alesapin](https://github.com/alesapin)) -* 修复带有 ClickHouse 源的外部字典在查询中有子查询时崩溃. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 使用引擎 `URL` 修复表中文件扩展名的错误解析. 这修复了 [#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([Andrey Bodrov](https://github.com/apbodrov)) -* 修复没有键的 `*MergeTree` 表的 `CHECK TABLE` 查询. 修复[#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) -* 修复了 `Float64` 到 MySQL 类型的转换. [#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([Yuriy Baranov](https://github.com/yurriy)) -* 现在如果表因为服务器崩溃而没有完全删除, 服务器将尝试恢复并加载它. [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) -* 修复了表函数 `file` 在插入不存在的文件时崩溃的问题. 现在在这种情况下将创建文件, 然后将处理插入. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) -* 修复启用 `trace_log` 时可能发生的罕见死锁. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) -* 在从 DDL 查询创建的 `RangeHashed` 外部字典中添加除 `Date` 之外的不同类型的处理能力. 修复 [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) -* 修复了当使用另一个函数的结果调用 `now64()` 时崩溃的问题. [#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([Vasily Nemkov](https://github.com/Enmk)) -* 修复了通过 mysql 线路协议检测连接的客户端 IP 的错误. [#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) -* 修复 `arraySplit` 函数中的空数组处理. 这修复了 [#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([hcz](https://github.com/hczhcz)) -* 修复了另一个正在运行的 `clickhouse-server` 的 `pid-file` 可能会被删除的问题. [#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([Weiqing Xu](https://github.com/weiqxu)) -* 修复字典重新加载, 如果它有 `invalidate_query`,它会停止更新, 并且在之前的更新尝试中出现一些异常. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) -* 修复了函数 `arrayReduce` 中可能导致 `double free` 的错误和聚合函数组合器 `Resample` 中可能导致内存泄漏的错误. 添加了聚合函数 `aggThrow`. 此功能可用于测试目的. [#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 改进 -* 使用 `S3` 表引擎时改进了日志记录. [#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -* 调用 `clickhouse-local` 时没有传递参数时打印帮助消息. 这修复了 [#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([Andrey Nagorny](https://github.com/Melancholic)) -* 添加设置 `mutations_sync` 允许同步等待 `ALTER UPDATE/DELETE` 查询. [#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([alesapin](https://github.com/alesapin)) -* 允许在 `config.xml` 中设置相关的 `user_files_path` (类似于 `format_schema_path` ). [#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([hcz](https://github.com/hczhcz)) -* 为带有 `-OrZero` 后缀的转换函数添加非法类型的例外. [#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([Andrey Konyaev](https://github.com/akonyaev90)) -* 简化分布式查询中发送到分片的数据头格式. [#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([Vitaly Baranov](https://github.com/vitlibar)) -* `Live View` 表引擎重构. [#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) -* 为从 DDL 查询创建的外部词典添加额外检查. [#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([alesapin](https://github.com/alesapin)) -* 修复错误 `Column ... already exists` , 同时使用 `FINAL` 和 `SAMPLE`, 例如 `select count() from table final sample 1/2`. 修复 [#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 现在表 `joinGet` 函数的第一个参数可以是表标识符. [#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([Amos Bird](https://github.com/amosbird)) -* 允许将 `MaterializedView` 与 `Kafka` 表上方的子查询一起使用. [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) -* 现在磁盘之间的后台移动在单独的线程池中运行它. [#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) -* `SYSTEM RELOAD DICTIONARY` 现在同步执行. [#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([Vitaly Baranov](https://github.com/vitlibar)) -* 堆栈跟踪现在显示物理地址 (目标文件中的偏移量) 而不是虚拟内存地址 (加载目标文件的位置). 当二进制文件与位置无关且 ASLR 处于活动状态时, 这允许使用 `addr2line`. 这修复了[#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 支持行级安全过滤器的新语法: `
...
`. 修复 [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([Ivan](https://github.com/abyss7)) -* 现在 `cityHash` 函数可以处理 `Decimal` 和 `UUID` 类型. 修复 [#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([Mikhail Korotov](https://github.com/millb)) -* 从系统日志中删除了固定索引粒度 (它是 1024), 因为它在实现自适应粒度后已过时. [#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在不使用 SSL 的情况下编译 ClickHouse 时启用 MySQL 兼容性服务器. [#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([Yuriy Baranov](https://github.com/yurriy)) -* 现在服务器校验和分布式批处理, 这会在批处理损坏的数据的情况下提供更详细的错误. [#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) -* 支持 `DROP DATABASE`、`DETACH TABLE`、`DROP TABLE`和 `ATTACH TABLE` 用于 `MySQL` 数据库引擎. [#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([Winter Zhang](https://github.com/zhang2014)) -* 在 S3 表函数和表引擎中添加身份验证. [#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 添加了对不同磁盘上 `MergeTree` 额外部分的检查, 以免遗漏未定义磁盘上的数据部分. [#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 为 Mac 客户端和服务器启用 SSL 支持. [#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([Ivan](https://github.com/abyss7)) -* 现在 ClickHouse 可以用作 MySQL 联合服务器 (请参阅 https://dev.mysql.com/doc/refman/5.7/en/federated-create-server.html). [#7717](https://github.com/ClickHouse/ClickHouse/pull/7717) ([Maxim Fedotov](https://github.com/MaxFedotov)) -* `clickhouse-client` 现在仅在开启多查询且关闭多行时启用 `bracketed-paste` . 这修复了 [#7757](https://github.com/ClickHouse/ClickHouse/issues/7757). [#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([Amos Bird](https://github.com/amosbird)) -* 在 `if` 函数中支持 `Array(Decimal)`. [#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) -* 在 `arrayDifference`、`arrayCumSum` 和 `arrayCumSumNegative` 函数中支持小数. [#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) -* 向 `system.dictionaries` 表添加了 `lifetime` 列. [#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) -* 改进了对 `*MergeTree` 表引擎不同磁盘上现有部分的检查. 地址 [#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 与用于 `S3` 交互的 `AWS SDK` 集成, 允许开箱即用地使用所有 S3 功能. [#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([Pavel Kovalenko](https://github.com/Jokser)) -* 添加了对 `Live View` 表中子查询的支持. [#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) -* 删除了从 `TTL` 表达式中检查使用 `Date` 或 `DateTime` 列的情况. [#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 有关磁盘的信息已添加到 `system.detached_parts` 表. [#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 现在可以在不重启的情况下更改设置 `max_(table|partition)_size_to_drop`. [#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -* 错误消息的可用性略好. 要求用户不要删除 `Stack trace:` 下面的行. [#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 [#7935](https://github.com/ClickHouse/ClickHouse/issues/7935) 之后,更好地读取来自 `Kafka` 引擎的各种格式的消息. [#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([Ivan](https://github.com/abyss7)) -* 更好地兼容不支持 `sha2_password` 认证插件的 MySQL 客户端. [#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([Yuriy Baranov](https://github.com/yurriy)) -* 在 MySQL 兼容服务器中支持更多的列类型. [#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([Yuriy Baranov](https://github.com/yurriy)) -* 为带有底层 `MergeTree` 表的 `Merge`、`Buffer` 和 `Materialized View` 存储实现 `ORDER BY` 优化. [#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([Anton Popov](https://github.com/CurtizJ)) -* 现在我们总是使用 `getrandom` 的POSIX实现来更好地兼容旧内核 (< 3.17). [#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([Amos Bird](https://github.com/amosbird)) -* 更好地检查移动 TTL 规则中的有效目的地. [#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 更好地检查 `Distributed` 表引擎的损坏插入批次. [#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) -* 将带有突变必须在未来处理的部分名称数组的列添加到 `system.mutations` 表. [#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([alesapin](https://github.com/alesapin)) -* 处理器的并行归并排序优化. [#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 设置 `mark_cache_min_lifetime` 现在已经过时并且什么都不做. 在以前的版本中, 标记缓存可以在大于 `mark_cache_size` 的内存中增长, 以在 `mark_cache_min_lifetime` 秒内容纳数据. 这导致混乱和比预期更高的内存使用量, 这在内存受限的系统上尤其糟糕. 如果您在安装此版本后发现性能下降, 您应该增加 `mark_cache_size`. [#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 准备在任何地方使用 `tid`. 这是 [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) 所必需的. [#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### 性能改进 -* 处理器管道中的性能优化. [#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 缓存字典中过期键的非阻塞更新 (具有读取旧键的权限). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 全局编译不带 `-fno-omit-frame-pointer` 的 ClickHouse 以节省一个寄存器. [#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([Amos Bird](https://github.com/amosbird)) -* 加速 `greatCircleDistance` 函数并为其添加性能测试. [#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) -* 改进功能 `roundDown` 的性能. [#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 改进了 `DateTime64` 数据类型的 `max`、`min`、`argMin`、`argMax`的性能. [#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([Vasily Nemkov](https://github.com/Enmk)) -* 改进了无限制或大限制和外部排序的排序性能. [#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 格式化浮点数的性能提高了 6 倍. [#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 改进了 `modulo` 函数的性能. [#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([Amos Bird](https://github.com/amosbird)) -* 优化了`ORDER BY` 并与单列键合并. [#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更好地实现 `arrayReduce`、`-Array` 和 `-State` 组合器. [#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([Amos Bird](https://github.com/amosbird)) -* 现在 `PREWHERE` 应该被优化为至少和 `WHERE` 一样有效. [#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([Amos Bird](https://github.com/amosbird)) -* 改进 `round` 和 `roundBankers` 处理负数的方式. [#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([hcz](https://github.com/hczhcz)) -* 将 `DoubleDelta` 和 `Gorilla` 编解码器的解码性能提高了大约 30-40%. 这修复了 [#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([Vasily Nemkov](https://github.com/Enmk)) -* 改进了 `base64` 相关功能的性能. [#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 添加了一个功能`geoDistance`. 它类似于 `greatCircleDistance`, 但使用对 WGS-84 椭球模型的近似. 两种功能的性能几乎相同. [#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* `Decimal` 数据类型的更快的 `min` 和 `max` 聚合函数. [#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) -* 向量化处理`arrayReduce`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) -* `if` 链现在优化为 `multiIf`. [#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) -* 修复了 19.15 中引入的 `Kafka` 表引擎的性能回归. 这修复了 [#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) -* 删除了 Debian 软件包中的 `gcc` 偶尔会在默认情况下带来的 'pie' 代码生成. [#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 并行解析数据格式. [#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* 默认情况下启用带有表达式的“值”的优化解析器 (`input_format_values_deduce_templates_of_expressions=1`). [#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([tavplubix](https://github.com/tavplubix)) - -#### 构建/测试/打包改进 -* 为 `ARM` 和最小模式构建修复. [#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([proller](https://github.com/proller)) -* 未调用 std::atexit 时为 `clickhouse-server` 添加覆盖文件刷新. 还略微改进了具有覆盖率的无状态测试的日志记录. [#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([alesapin](https://github.com/alesapin)) -* 在 contrib 中更新 LLVM 库. 避免使用 OS 包中的 LLVM. [#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使捆绑的 `curl` 构建完全安静. [#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([Pavel Kovalenko](https://github.com/Jokser)) -* 修复一些 `MemorySanitizer` 警告. [#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 在 `CMakeLists.txt` 中使用 `add_warning` 和 `no_warning` 宏. [#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([Ivan](https://github.com/abyss7)) -* 添加对 Minio S3 兼容对象 (https://min.io/) 的支持以进行更好的集成测试. [#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([Pavel Kovalenko](https://github.com/Jokser)) -* 将 `libc` 头文件导入到 contrib. 它允许在各种系统之间使构建更加一致 (仅适用于 `x86_64-linux-gnu`). [#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 从某些库中删除 `-fPIC`. [#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为 curl 清理 `CMakeLists.txt`. 见 https://github.com/ClickHouse/ClickHouse/pull/8011#issuecomment-569478910 [#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* `CapNProto` 库中的静默警告. [#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为短字符串优化哈希表添加性能测试. [#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([Amos Bird](https://github.com/amosbird)) -* 现在, 即使 `MADV_FREE` 不可用, ClickHouse 也将构建在 `AArch64` 上. 这修复了 [#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([Amos Bird](https://github.com/amosbird)) -* 更新 `zlib-ng` 以修复内存清理器问题. [#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 在非 Linux 系统上启用内部 MySQL 库, 因为 OS 包的使用非常脆弱, 通常根本不起作用. 这修复了 [#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了启用 `libc++` 后在某些系统上的构建. 这取代了 [#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使 `Field` 方法更加类型安全以查找更多错误. [#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 将缺失的文件添加到 `libc-headers` 子模块. [#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复性能测试输出中错误的 `JSON` 引用. [#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 现在为 `std::exception` 和 `Poco::Exception` 显示堆栈跟踪. 在以前的版本中, 它仅适用于`DB::Exception`. 这改进了诊断. [#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 为新的 glibc 版本移植 `clock_gettime` 和 `clock_nanosleep`. [#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([Amos Bird](https://github.com/amosbird)) -* 在示例配置中为开发人员启用`part_log`. [#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复 `01036_no_superfluous_dict_reload_on_create_database*` 中重新加载的异步性质. [#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) -* 修复编解码器性能测试. [#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([Vasily Nemkov](https://github.com/Enmk)) -* 为 `.tgz` 构建和文档添加安装脚本. [#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([alesapin](https://github.com/alesapin)) -* 删除了旧的 `ZSTD` 测试 (它是在 2016 年创建的, 用于重现 ZSTD 1.0 版之前的错误). 这修复了 [#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了 Mac OS Catalina 上的构建. [#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) -* 增加编解码器性能测试中的行数, 使结果更显着. [#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([Vasily Nemkov](https://github.com/Enmk)) -* 在调试版本中, 将 `LOGICAL_ERROR` 异常视为断言失败, 以便它们更容易被注意到. [#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 使与格式相关的性能测试更具确定性. [#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 更新 `lz4` 以修复 MemorySanitizer 故障. [#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 在异常处理中抑制已知的 MemorySanitizer 误报. [#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 将 `gcc` 和 `g++` 更新到 `build/docker/build.sh` 中的版本 9. [#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) -* 添加性能测试用例来测试 `PREWHERE` 比 `WHERE` 差. [#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([Amos Bird](https://github.com/amosbird)) -* 修复一个不稳定的测试的进展. [#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 避免对来自`libunwind` 的数据进行 MemorySanitizer 报告. [#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 将 `libc++` 更新到最新版本. [#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 从源代码构建 ICU 库. 这修复了 [#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 从 `libressl` 切换到 `openssl`. 在此更改后,ClickHouse 应支持 TLS 1.3 和 SNI. 这修复了[#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复了从 SSL 使用 `chacha20_poly1305` 时的 UBSan 报告 (发生在连接到 https://yandex.ru/ 时). [#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复`.deb` linux 发行版的默认密码文件模式. [#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([proller](https://github.com/proller)) -* 改进了在 `clickhouse-test` 中获取 `clickhouse-server` PID 的表达式. [#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([Alexander Kazakov](https://github.com/Akazz)) -* 将 contrib/googletest 更新到 v1.10.0. [#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alexander Burmak](https://github.com/Alex-Burmak)) -* 修复了 `base64` 库中的 ThreadSaninitizer 报告. 也把这个库更新到了最新版本, 不过没关系. 这修复了 [#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 修复处理器的 `00600_replace_running_query` . [#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 删除对 `tcmalloc` 的支持, 使 `CMakeLists.txt` 更简单. [#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 发布 gcc 构建现在使用 `libc++` 而不是 `libstdc++`. 最近 `libc++` 只与clang 一起使用. 这将提高构建配置和可移植性的一致性. [#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 启用 ICU 库以使用 MemorySanitizer 进行构建. [#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 抑制来自 `CapNProto` 库的警告. [#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 删除了 `tcmalloc` 代码的特殊情况, 因为它不再受支持. [#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 CI 覆盖任务中, 优雅地杀死服务器以允许它保存覆盖报告. 这修复了我们最近看到的不完整的覆盖报告. [#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([alesapin](https://github.com/alesapin)) -* 针对 `Float64` 和 `UInt64` 值对所有编解码器进行性能测试. [#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([Vasily Nemkov](https://github.com/Enmk)) -* `termcap` 非常不推荐使用并导致各种问题 (例如, 缺少“up”cap 和回显 `^J` 而不是多行). 支持`terminfo` 或捆绑 `ncurses`. [#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([Amos Bird](https://github.com/amosbird)) -* 修复 `test_storage_s3` 集成测试. [#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 支持 `StorageFile(, null)` 将块插入给定的格式文件而不实际写入磁盘. 这是性能测试所必需的. [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) -* 向功能测试添加了参数 `--print-time`, 用于打印每个测试的执行时间. [#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 在评估 RPN 时向 `KeyCondition` 添加断言. 这将修复来自 gcc-9 的警告. [#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 在 CI 构建中转储 cmake 选项. [#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 不要为某些胖库生成调试信息. [#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 使 `log_to_console.xml` 始终登录到 stderr, 无论它是否交互. [#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* 从 `clickhouse-performance-test` 工具中删除了一些未使用的功能. [#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 现在我们还将搜索带有相应 `clang-X` 版本的 `lld-X`. [#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([alesapin](https://github.com/alesapin)) -* Parquet 构建改进. [#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([maxulan](https://github.com/maxulan)) -* 更多 GCC 警告. [#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Arch Linux 软件包现在允许运行 ClickHouse 服务器, 而不仅仅是客户端. [#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) -* 使用处理器修复测试. 微小的性能修复. [#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 更新 contrib/protobuf. [#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V. Kornilov](https://github.com/matwey)) -* 准备切换到 c++20 作为新年庆祝活动. "May the C++ force be with ClickHouse". [#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([Amos Bird](https://github.com/amosbird)) - -#### 实验功能 -* 添加了实验设置 `min_bytes_to_use_mmap_io`. 它允许读取大文件而无需将数据从内核复制到用户空间. 默认情况下禁用该设置. 推荐的阈值大约是 64 MB, 因为 mmap/munmap 很慢. [#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* 重新设计配额作为访问控制系统的一部分. 添加了新表 `system.quotas`、新函数 `currentQuota`、`currentQuotaKey`、新SQL语法 `CREATE QUOTA`、`ALTER QUOTA`、`DROP QUOTA`、`SHOW QUOTA`. [#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([Vitaly Baranov](https://github.com/vitlibar)) -* 允许跳过带有警告的未知设置而不是抛出异常. [#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([Vitaly Baranov](https://github.com/vitlibar)) -* 重新设计行策略作为访问控制系统的一部分. 添加新表`system.row_policies`、新函数 `currentRowPolicies()`、新SQL语法 `CREATE POLICY`、`ALTER POLICY`、`DROP POLICY`、`SHOW CREATE POLICY`、`SHOW POLICIES`. [#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([Vitaly Baranov](https://github.com/vitlibar)) - -#### Security Fix -* 修复了使用 `File` 表引擎读取表中目录结构的可能性. 这修复了 [#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## [Changelog for 2019](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2019.md) From 69792adfa22ff258ee56e0684b5eab15b70424fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Sep 2022 05:32:23 +0200 Subject: [PATCH 524/582] Add a test for #4242 --- .../02418_tautological_if_index.reference | 2 ++ .../02418_tautological_if_index.sql | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02418_tautological_if_index.reference create mode 100644 tests/queries/0_stateless/02418_tautological_if_index.sql diff --git a/tests/queries/0_stateless/02418_tautological_if_index.reference b/tests/queries/0_stateless/02418_tautological_if_index.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02418_tautological_if_index.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02418_tautological_if_index.sql b/tests/queries/0_stateless/02418_tautological_if_index.sql new file mode 100644 index 00000000000..c37f24b29f6 --- /dev/null +++ b/tests/queries/0_stateless/02418_tautological_if_index.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS constCondOptimization; + +CREATE TABLE constCondOptimization +( + d Date DEFAULT today(), + time DateTime DEFAULT now(), + n Int64 +) +ENGINE = MergeTree ORDER BY (time, n) SETTINGS index_granularity = 1; + +INSERT INTO constCondOptimization (n) SELECT number FROM system.numbers LIMIT 10000; + +-- The queries should use index. +SET max_rows_to_read = 2; + +SELECT count() FROM constCondOptimization WHERE if(0, 1, n = 1000); +SELECT count() FROM constCondOptimization WHERE if(0, 1, n = 1000) AND 1 = 1; + +DROP TABLE constCondOptimization; From 2b6724c5e3f4e4129f8bf4dd48aec13f0adf3992 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 01:58:55 +0200 Subject: [PATCH 525/582] Fix example --- src/Core/examples/field.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/examples/field.cpp b/src/Core/examples/field.cpp index 3190a7fcb7d..110e11d0cb1 100644 --- a/src/Core/examples/field.cpp +++ b/src/Core/examples/field.cpp @@ -37,7 +37,7 @@ int main(int argc, char ** argv) std::cerr << applyVisitor(to_string, field) << std::endl; } - get(field).push_back(field); + field.get().push_back(field); std::cerr << applyVisitor(to_string, field) << std::endl; std::cerr << (field < field2) << std::endl; @@ -71,7 +71,7 @@ int main(int argc, char ** argv) size_t sum = 0; for (size_t i = 0; i < n; ++i) - sum += safeGet(array[i]).size(); + sum += array[i].safeGet().size(); watch.stop(); std::cerr << std::fixed << std::setprecision(2) From cc38ac37640928737586b83decb20208cb32bd95 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 03:03:37 +0200 Subject: [PATCH 526/582] Better parallel hash JOIN for floats --- src/Columns/ColumnVector.cpp | 2 +- src/Common/HashTable/Hash.h | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index d89917fc6c2..d4408bb7f27 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -90,7 +90,7 @@ void ColumnVector::updateWeakHash32(WeakHash32 & hash) const while (begin < end) { - *hash_data = intHashCRC32(*begin, *hash_data); + *hash_data = hashCRC32(*begin, *hash_data); ++begin; ++hash_data; } diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index bf691458d27..9d240bee461 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -220,7 +220,7 @@ template struct HashCRC32; template requires (sizeof(T) <= sizeof(UInt64)) -inline size_t hashCRC32(T key) +inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) { union { @@ -229,14 +229,14 @@ inline size_t hashCRC32(T key) } u; u.out = 0; u.in = key; - return intHashCRC32(u.out); + return intHashCRC32(u.out, updated_value); } template requires (sizeof(T) > sizeof(UInt64)) -inline size_t hashCRC32(T key) +inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) { - return intHashCRC32(key, -1); + return intHashCRC32(key, updated_value); } #define DEFINE_HASH(T) \ From 512fb998f196e9a5372e3c6d8bfe9db2132f95ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 03:53:04 +0200 Subject: [PATCH 527/582] Fix ridiculous trash in Base58 --- src/Common/base58.cpp | 126 ++++++++++++++++++++++++++++++++++++++++++ src/Common/base58.h | 114 ++------------------------------------ 2 files changed, 131 insertions(+), 109 deletions(-) create mode 100644 src/Common/base58.cpp diff --git a/src/Common/base58.cpp b/src/Common/base58.cpp new file mode 100644 index 00000000000..8c024b35d93 --- /dev/null +++ b/src/Common/base58.cpp @@ -0,0 +1,126 @@ +#include + + +namespace DB +{ + +size_t encodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) +{ + const char * base58_encoding_alphabet = "123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz"; + + size_t processed = 0; + size_t zeros = 0; + + for (;*src == '\0' && processed < srclen - 1; ++src) + { + ++processed; + ++zeros; + *dst = '1'; + ++dst; + } + + size_t idx = 0; + + while (processed < srclen - 1) + { + UInt8 carry = static_cast(*src); + + for (size_t j = 0; j < idx; ++j) + { + carry += static_cast(dst[j] << 8); + dst[j] = static_cast(carry % 58); + carry /= 58; + } + + while (carry > 0) + { + dst[idx] = static_cast(carry % 58); + ++idx; + carry /= 58; + } + ++src; + ++processed; + } + + size_t c_idx = idx >> 1; + for (size_t i = 0; i < c_idx; ++i) + { + char s = base58_encoding_alphabet[static_cast(dst[i])]; + dst[i] = base58_encoding_alphabet[static_cast(dst[idx - (i + 1)])]; + dst[idx - (i + 1)] = s; + } + + if ((idx & 1)) + { + dst[c_idx] = base58_encoding_alphabet[static_cast(dst[c_idx])]; + } + + dst[idx] = '\0'; + return zeros + idx + 1; +} + + +size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) +{ + static const Int8 map_digits[128] = + { + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, -1, -1, -1, -1, -1, -1, + -1, 9, 10, 11, 12, 13, 14, 15, 16, -1, 17, 18, 19, 20, 21, -1, + 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, -1, -1, -1, -1, -1, + -1, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, -1, 44, 45, 46, + 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, -1, -1, -1, -1 + }; + + size_t processed = 0; + size_t zeros = 0; + + while (*src == '1' && processed < srclen - 1) + { + ++processed; + ++zeros; + *dst = '\0'; + ++dst; + ++src; + } + + size_t idx = 0; + + while (processed < srclen-1) + { + UInt32 carry = map_digits[*src]; + if (carry == static_cast(-1)) + { + return 0; + } + for (size_t j = 0; j < idx; ++j) + { + carry += static_cast(dst[j]) * 58; + dst[j] = static_cast(carry & 0xFF); + carry >>= 8; + } + while (carry > 0) + { + dst[idx] = static_cast(carry & 0xFF); + ++idx; + carry >>= 8; + } + ++src; + ++processed; + } + + size_t c_idx = idx >> 1; + for (size_t i = 0; i < c_idx; ++i) + { + char s = dst[i]; + dst[i] = dst[idx - (i + 1)]; + dst[idx - (i + 1)] = s; + } + + dst[idx] = '\0'; + return zeros + idx + 1; +} + +} diff --git a/src/Common/base58.h b/src/Common/base58.h index bc3c3c7aee8..3eb4466f086 100644 --- a/src/Common/base58.h +++ b/src/Common/base58.h @@ -1,116 +1,12 @@ #pragma once -#include -#include + +#include + namespace DB { -inline size_t encodeBase58(const char8_t * src, size_t srclen, char8_t * dst) -{ - const char * base58_encoding_alphabet = "123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz"; - - size_t processed = 0; - size_t zeros = 0; - for (;*src == '\0' && processed < srclen-1; ++src) - { - ++processed; - ++zeros; - *dst++ = '1'; - } - - size_t idx = 0; - - while (processed < srclen-1) - { - unsigned int carry = static_cast(*src); - for (size_t j = 0; j < idx; ++j) - { - carry += static_cast(dst[j] << 8); - dst[j] = static_cast(carry % 58); - carry /= 58; - } - while (carry > 0) - { - dst[idx++] = static_cast(carry % 58); - carry /= 58; - } - ++src; - ++processed; - } - - size_t c_idx = idx >> 1; - for (size_t i = 0; i < c_idx; ++i) - { - char s = base58_encoding_alphabet[static_cast(dst[i])]; - dst[i] = base58_encoding_alphabet[static_cast(dst[idx - (i + 1)])]; - dst[idx - (i + 1)] = s; - } - if ((idx & 1)) - { - dst[c_idx] = base58_encoding_alphabet[static_cast(dst[c_idx])]; - } - - dst[idx] = '\0'; - return zeros + idx + 1; -} - -inline size_t decodeBase58(const char8_t * src, size_t srclen, char8_t * dst) -{ - const signed char uint_max = UINT_MAX; - const signed char map_digits[128] - = {uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, - uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, - uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, - uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, 0, 1, 2, - 3, 4, 5, 6, 7, 8, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, - 9, 10, 11, 12, 13, 14, 15, 16, uint_max, 17, 18, 19, 20, - 21, uint_max, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, 33, 34, 35, 36, 37, 38, 39, - 40, 41, 42, 43, uint_max, 44, 45, 46, 47, 48, 49, 50, 51, - 52, 53, 54, 55, 56, 57, uint_max, uint_max, uint_max, uint_max, uint_max}; - - size_t processed = 0; - size_t zeros = 0; - for (;*src == '1' && processed < srclen-1; ++src) - { - ++processed; - ++zeros; - *dst++ = '\0'; - } - - size_t idx = 0; - - while (processed < srclen-1) - { - unsigned int carry = map_digits[*src]; - if (unlikely(carry == UINT_MAX)) - { - return 0; - } - for (size_t j = 0; j < idx; ++j) - { - carry += static_cast(dst[j]) * 58; - dst[j] = static_cast(carry & 0xff); - carry >>= 8; - } - while (carry > 0) - { - dst[idx++] = static_cast(carry & 0xff); - carry >>= 8; - } - ++src; - ++processed; - } - - size_t c_idx = idx >> 1; - for (size_t i = 0; i < c_idx; ++i) - { - char s = dst[i]; - dst[i] = dst[idx - (i + 1)]; - dst[idx - (i + 1)] = s; - } - dst[idx] = '\0'; - return zeros + idx + 1; -} +size_t encodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst); +size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst); } From 33635e015d0b28daab3f895d55233da50d1f3887 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 06:11:02 +0200 Subject: [PATCH 528/582] Fix style --- base/base/shift10.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/base/base/shift10.cpp b/base/base/shift10.cpp index 286c3be85f0..5429bcc23b2 100644 --- a/base/base/shift10.cpp +++ b/base/base/shift10.cpp @@ -14,10 +14,10 @@ static T shift10Impl(T x, int exponent) static const long double powers10[] = { 1e-323L, 1e-322L, 1e-321L, 1e-320L, 1e-319L, 1e-318L, 1e-317L, 1e-316L, 1e-315L, 1e-314L, 1e-313L, 1e-312L, 1e-311L, - 1e-310L, 1e-309L, 1e-308L, 1e-307L, 1e-306L, 1e-305L, 1e-304L, 1e-303L, 1e-302L, 1e-301L, 1e-300L, 1e-299L, 1e-298L, 1e-297L, 1e-296L, 1e-295L, 1e-294L, 1e-293L, 1e-292L, 1e-291L, - 1e-290L, 1e-289L, 1e-288L, 1e-287L, 1e-286L, 1e-285L, 1e-284L, 1e-283L, 1e-282L, 1e-281L, 1e-280L, 1e-279L, 1e-278L, 1e-277L, 1e-276L, 1e-275L, 1e-274L, 1e-273L, 1e-272L, 1e-271L, - 1e-270L, 1e-269L, 1e-268L, 1e-267L, 1e-266L, 1e-265L, 1e-264L, 1e-263L, 1e-262L, 1e-261L, 1e-260L, 1e-259L, 1e-258L, 1e-257L, 1e-256L, 1e-255L, 1e-254L, 1e-253L, 1e-252L, 1e-251L, - 1e-250L, 1e-249L, 1e-248L, 1e-247L, 1e-246L, 1e-245L, 1e-244L, 1e-243L, 1e-242L, 1e-241L, 1e-240L, 1e-239L, 1e-238L, 1e-237L, 1e-236L, 1e-235L, 1e-234L, 1e-233L, 1e-232L, 1e-231L, + 1e-310L, 1e-309L, 1e-308L, 1e-307L, 1e-306L, 1e-305L, 1e-304L, 1e-303L, 1e-302L, 1e-301L, 1e-300L, 1e-299L, 1e-298L, 1e-297L, 1e-296L, 1e-295L, 1e-294L, 1e-293L, 1e-292L, 1e-291L, + 1e-290L, 1e-289L, 1e-288L, 1e-287L, 1e-286L, 1e-285L, 1e-284L, 1e-283L, 1e-282L, 1e-281L, 1e-280L, 1e-279L, 1e-278L, 1e-277L, 1e-276L, 1e-275L, 1e-274L, 1e-273L, 1e-272L, 1e-271L, + 1e-270L, 1e-269L, 1e-268L, 1e-267L, 1e-266L, 1e-265L, 1e-264L, 1e-263L, 1e-262L, 1e-261L, 1e-260L, 1e-259L, 1e-258L, 1e-257L, 1e-256L, 1e-255L, 1e-254L, 1e-253L, 1e-252L, 1e-251L, + 1e-250L, 1e-249L, 1e-248L, 1e-247L, 1e-246L, 1e-245L, 1e-244L, 1e-243L, 1e-242L, 1e-241L, 1e-240L, 1e-239L, 1e-238L, 1e-237L, 1e-236L, 1e-235L, 1e-234L, 1e-233L, 1e-232L, 1e-231L, 1e-230L, 1e-229L, 1e-228L, 1e-227L, 1e-226L, 1e-225L, 1e-224L, 1e-223L, 1e-222L, 1e-221L, 1e-220L, 1e-219L, 1e-218L, 1e-217L, 1e-216L, 1e-215L, 1e-214L, 1e-213L, 1e-212L, 1e-211L, 1e-210L, 1e-209L, 1e-208L, 1e-207L, 1e-206L, 1e-205L, 1e-204L, 1e-203L, 1e-202L, 1e-201L, 1e-200L, 1e-199L, 1e-198L, 1e-197L, 1e-196L, 1e-195L, 1e-194L, 1e-193L, 1e-192L, 1e-191L, 1e-190L, 1e-189L, 1e-188L, 1e-187L, 1e-186L, 1e-185L, 1e-184L, 1e-183L, 1e-182L, 1e-181L, 1e-180L, 1e-179L, 1e-178L, 1e-177L, 1e-176L, 1e-175L, 1e-174L, 1e-173L, 1e-172L, 1e-171L, From 90fd5072929f229f752e4e3e77ebfa787594411f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 06:18:43 +0200 Subject: [PATCH 529/582] Fix test --- src/AggregateFunctions/QuantileTDigest.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index b61e4874d00..ce87e6817c6 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -103,8 +103,8 @@ class QuantileTDigest static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2) { /// Symmetric interpolation for better results with infinities. - Value k = (x - x1) / (x2 - x1); - return (1 - k) * y1 + k * y2; + double k = (x - x1) / (x2 - x1); + return static_cast((1 - k) * y1 + k * y2); } struct RadixSortTraits From 3d8a2130cf4348c167831ada20a06ab22064ecbc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 06:31:15 +0200 Subject: [PATCH 530/582] Continue fixing the trash --- src/Common/base58.cpp | 7 ++++--- src/Common/base58.h | 4 ++-- tests/queries/0_stateless/02337_base58.sql | 2 -- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Common/base58.cpp b/src/Common/base58.cpp index 8c024b35d93..fa11810d89f 100644 --- a/src/Common/base58.cpp +++ b/src/Common/base58.cpp @@ -4,24 +4,25 @@ namespace DB { -size_t encodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) +size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst) { const char * base58_encoding_alphabet = "123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz"; size_t processed = 0; size_t zeros = 0; - for (;*src == '\0' && processed < srclen - 1; ++src) + while (*src == '\0' && processed < src_length - 1) { ++processed; ++zeros; *dst = '1'; ++dst; + ++src; } size_t idx = 0; - while (processed < srclen - 1) + while (processed < src_length - 1) { UInt8 carry = static_cast(*src); diff --git a/src/Common/base58.h b/src/Common/base58.h index 3eb4466f086..b4ed96e4169 100644 --- a/src/Common/base58.h +++ b/src/Common/base58.h @@ -6,7 +6,7 @@ namespace DB { -size_t encodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst); -size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst); +size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst); +size_t decodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst); } diff --git a/tests/queries/0_stateless/02337_base58.sql b/tests/queries/0_stateless/02337_base58.sql index 42b032c7601..3d6da61c50a 100644 --- a/tests/queries/0_stateless/02337_base58.sql +++ b/tests/queries/0_stateless/02337_base58.sql @@ -1,7 +1,5 @@ -- Tags: no-fasttest -SET send_logs_level = 'fatal'; - SELECT base58Encode('Hold my beer...'); SELECT base58Encode('Hold my beer...', 'Second arg'); -- { serverError 42 } SELECT base58Decode('Hold my beer...'); -- { serverError 36 } From e0a9ae04968e434ca0948a2c9d057881ab889c2d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 08:09:14 +0200 Subject: [PATCH 531/582] Fix base58 trash --- src/Common/base58.cpp | 43 +++++++++++++----------- src/Common/base58.h | 3 +- src/Functions/FunctionBase58Conversion.h | 42 ++++++++++++----------- 3 files changed, 48 insertions(+), 40 deletions(-) diff --git a/src/Common/base58.cpp b/src/Common/base58.cpp index fa11810d89f..fb2c98c82b8 100644 --- a/src/Common/base58.cpp +++ b/src/Common/base58.cpp @@ -9,9 +9,10 @@ size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst) const char * base58_encoding_alphabet = "123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz"; size_t processed = 0; + size_t idx = 0; size_t zeros = 0; - while (*src == '\0' && processed < src_length - 1) + while (processed < src_length && *src == 0) { ++processed; ++zeros; @@ -20,15 +21,13 @@ size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst) ++src; } - size_t idx = 0; - - while (processed < src_length - 1) + while (processed < src_length) { - UInt8 carry = static_cast(*src); + UInt32 carry = *src; for (size_t j = 0; j < idx; ++j) { - carry += static_cast(dst[j] << 8); + carry += static_cast(dst[j]) << 8; dst[j] = static_cast(carry % 58); carry /= 58; } @@ -39,6 +38,7 @@ size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst) ++idx; carry /= 58; } + ++src; ++processed; } @@ -56,14 +56,13 @@ size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst) dst[c_idx] = base58_encoding_alphabet[static_cast(dst[c_idx])]; } - dst[idx] = '\0'; - return zeros + idx + 1; + return zeros + idx; } -size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) +std::optional decodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst) { - static const Int8 map_digits[128] = + static const Int8 map_digits[256] = { -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, @@ -72,13 +71,22 @@ size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) -1, 9, 10, 11, 12, 13, 14, 15, 16, -1, 17, 18, 19, 20, 21, -1, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, -1, -1, -1, -1, -1, -1, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, -1, 44, 45, 46, - 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, -1, -1, -1, -1 + 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, }; size_t processed = 0; + size_t idx = 0; size_t zeros = 0; - while (*src == '1' && processed < srclen - 1) + while (processed < src_length && *src == '1') { ++processed; ++zeros; @@ -87,14 +95,12 @@ size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) ++src; } - size_t idx = 0; - - while (processed < srclen-1) + while (processed < src_length) { UInt32 carry = map_digits[*src]; if (carry == static_cast(-1)) { - return 0; + return {}; } for (size_t j = 0; j < idx; ++j) { @@ -115,13 +121,12 @@ size_t decodeBase58(const UInt8 * src, size_t srclen, UInt8 * dst) size_t c_idx = idx >> 1; for (size_t i = 0; i < c_idx; ++i) { - char s = dst[i]; + UInt8 s = dst[i]; dst[i] = dst[idx - (i + 1)]; dst[idx - (i + 1)] = s; } - dst[idx] = '\0'; - return zeros + idx + 1; + return zeros + idx; } } diff --git a/src/Common/base58.h b/src/Common/base58.h index b4ed96e4169..cf5341a1efb 100644 --- a/src/Common/base58.h +++ b/src/Common/base58.h @@ -1,12 +1,13 @@ #pragma once #include +#include namespace DB { size_t encodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst); -size_t decodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst); +std::optional decodeBase58(const UInt8 * src, size_t src_length, UInt8 * dst); } diff --git a/src/Functions/FunctionBase58Conversion.h b/src/Functions/FunctionBase58Conversion.h index bc166f2c9f5..14e316d2864 100644 --- a/src/Functions/FunctionBase58Conversion.h +++ b/src/Functions/FunctionBase58Conversion.h @@ -41,23 +41,24 @@ struct Base58Encode const auto * src = src_column.getChars().data(); auto * dst = dst_data.data(); - auto * dst_pos = dst; - size_t src_offset_prev = 0; + size_t prev_src_offset = 0; + size_t current_dst_offset = 0; for (size_t row = 0; row < input_rows_count; ++row) { - size_t srclen = src_offsets[row] - src_offset_prev; - auto encoded_size = encodeBase58(src, srclen, dst_pos); + size_t current_src_offset = src_offsets[row]; + size_t src_length = current_src_offset - prev_src_offset - 1; + size_t encoded_size = encodeBase58(&src[prev_src_offset], src_length, &dst[current_dst_offset]); + prev_src_offset = current_src_offset; + current_dst_offset += encoded_size; + dst[current_dst_offset] = 0; + ++current_dst_offset; - src += srclen; - dst_pos += encoded_size; - - dst_offsets[row] = dst_pos - dst; - src_offset_prev = src_offsets[row]; + dst_offsets[row] = current_dst_offset; } - dst_data.resize(dst_pos - dst); + dst_data.resize(current_dst_offset); } }; @@ -82,26 +83,27 @@ struct Base58Decode const auto * src = src_column.getChars().data(); auto * dst = dst_data.data(); - auto * dst_pos = dst; - size_t src_offset_prev = 0; + size_t prev_src_offset = 0; + size_t current_dst_offset = 0; for (size_t row = 0; row < input_rows_count; ++row) { - size_t srclen = src_offsets[row] - src_offset_prev; - - auto decoded_size = decodeBase58(src, srclen, dst_pos); + size_t current_src_offset = src_offsets[row]; + size_t src_length = current_src_offset - prev_src_offset - 1; + std::optional decoded_size = decodeBase58(&src[prev_src_offset], src_length, &dst[current_dst_offset]); if (!decoded_size) throw Exception("Invalid Base58 value, cannot be decoded", ErrorCodes::BAD_ARGUMENTS); - src += srclen; - dst_pos += decoded_size; + prev_src_offset = current_src_offset; + current_dst_offset += *decoded_size; + dst[current_dst_offset] = 0; + ++current_dst_offset; - dst_offsets[row] = dst_pos - dst; - src_offset_prev = src_offsets[row]; + dst_offsets[row] = current_dst_offset; } - dst_data.resize(dst_pos - dst); + dst_data.resize(current_dst_offset); } }; From 88e917883066fb6e5be150a9fc094d52b3aedba0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Sep 2022 08:46:05 +0200 Subject: [PATCH 532/582] Faster build --- .../AggregateFunctionQuantile.cpp | 143 ++---------------- .../AggregateFunctionQuantile.h | 12 -- .../AggregateFunctionQuantileBFloat16.cpp | 60 ++++++++ ...regateFunctionQuantileBFloat16Weighted.cpp | 60 ++++++++ ...AggregateFunctionQuantileDeterministic.cpp | 71 +++++++++ .../AggregateFunctionQuantileExact.cpp | 72 +++++++++ ...ggregateFunctionQuantileExactExclusive.cpp | 57 +++++++ .../AggregateFunctionQuantileExactHigh.cpp | 67 ++++++++ ...ggregateFunctionQuantileExactInclusive.cpp | 57 +++++++ .../AggregateFunctionQuantileExactLow.cpp | 66 ++++++++ ...AggregateFunctionQuantileExactWeighted.cpp | 71 +++++++++ .../AggregateFunctionQuantileTDigest.cpp | 60 ++++++++ ...gregateFunctionQuantileTDigestWeighted.cpp | 60 ++++++++ .../AggregateFunctionQuantileTiming.cpp | 60 ++++++++ ...ggregateFunctionQuantileTimingWeighted.cpp | 60 ++++++++ 15 files changed, 833 insertions(+), 143 deletions(-) create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileBFloat16.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileDeterministic.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileExact.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileExactExclusive.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileExactLow.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileTDigest.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileTDigestWeighted.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileTimingWeighted.cpp diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 6783a55418a..38b3c91be69 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -1,8 +1,9 @@ #include - +#include #include #include - +#include +#include #include @@ -21,70 +22,6 @@ namespace template using FuncQuantile = AggregateFunctionQuantile, NameQuantile, false, std::conditional_t, false>; template using FuncQuantiles = AggregateFunctionQuantile, NameQuantiles, false, std::conditional_t, true>; -template using FuncQuantileDeterministic = AggregateFunctionQuantile, NameQuantileDeterministic, true, std::conditional_t, false>; -template using FuncQuantilesDeterministic = AggregateFunctionQuantile, NameQuantilesDeterministic, true, std::conditional_t, true>; - -template using FuncQuantileExact = AggregateFunctionQuantile, NameQuantileExact, false, void, false>; -template using FuncQuantilesExact = AggregateFunctionQuantile, NameQuantilesExact, false, void, true>; - -template using FuncQuantileExactLow = AggregateFunctionQuantile, NameQuantileExactLow, false, void, false>; -template using FuncQuantilesExactLow = AggregateFunctionQuantile, NameQuantilesExactLow, false, void, true>; -template using FuncQuantileExactHigh = AggregateFunctionQuantile, NameQuantileExactHigh, false, void, false>; -template using FuncQuantilesExactHigh = AggregateFunctionQuantile, NameQuantilesExactHigh, false, void, true>; - -template using FuncQuantileExactExclusive = AggregateFunctionQuantile, NameQuantileExactExclusive, false, Float64, false>; -template using FuncQuantilesExactExclusive = AggregateFunctionQuantile, NameQuantilesExactExclusive, false, Float64, true>; - -template using FuncQuantileExactInclusive = AggregateFunctionQuantile, NameQuantileExactInclusive, false, Float64, false>; -template using FuncQuantilesExactInclusive = AggregateFunctionQuantile, NameQuantilesExactInclusive, false, Float64, true>; - -template using FuncQuantileExactWeighted = AggregateFunctionQuantile, NameQuantileExactWeighted, true, void, false>; -template using FuncQuantilesExactWeighted = AggregateFunctionQuantile, NameQuantilesExactWeighted, true, void, true>; - -template using FuncQuantileTiming = AggregateFunctionQuantile, NameQuantileTiming, false, Float32, false>; -template using FuncQuantilesTiming = AggregateFunctionQuantile, NameQuantilesTiming, false, Float32, true>; - -template using FuncQuantileTimingWeighted = AggregateFunctionQuantile, NameQuantileTimingWeighted, true, Float32, false>; -template using FuncQuantilesTimingWeighted = AggregateFunctionQuantile, NameQuantilesTimingWeighted, true, Float32, true>; - -template using FuncQuantileTDigest = AggregateFunctionQuantile, NameQuantileTDigest, false, std::conditional_t, false>; -template using FuncQuantilesTDigest = AggregateFunctionQuantile, NameQuantilesTDigest, false, std::conditional_t, true>; - -template using FuncQuantileTDigestWeighted = AggregateFunctionQuantile, NameQuantileTDigestWeighted, true, std::conditional_t, false>; -template using FuncQuantilesTDigestWeighted = AggregateFunctionQuantile, NameQuantilesTDigestWeighted, true, std::conditional_t, true>; - -template using FuncQuantileBFloat16 = AggregateFunctionQuantile, NameQuantileBFloat16, false, std::conditional_t, false>; -template using FuncQuantilesBFloat16 = AggregateFunctionQuantile, NameQuantilesBFloat16, false, std::conditional_t, true>; - -template using FuncQuantileBFloat16Weighted = AggregateFunctionQuantile, NameQuantileBFloat16Weighted, true, std::conditional_t, false>; -template using FuncQuantilesBFloat16Weighted = AggregateFunctionQuantile, NameQuantilesBFloat16Weighted, true, std::conditional_t, true>; - -template