From 6e296d034272503acef51a979d28cacc1e5a6a2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Oct 2021 13:44:52 +0300 Subject: [PATCH 001/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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/100] 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 b4c4448f0feb27abe7665ab2b3e570f75b03ac8c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 2 Aug 2022 12:07:38 +0200 Subject: [PATCH 015/100] 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 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 016/100] 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 017/100] 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 018/100] 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 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 019/100] 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 020/100] 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 021/100] 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 022/100] 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 023/100] 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 024/100] 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 025/100] 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 026/100] 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 0bad9425d26d08fd2087940374ac5cee5b5f14f4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Aug 2022 19:27:40 +0200 Subject: [PATCH 027/100] 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 028/100] 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 0433b801d2d8e157aca800319755510082b930ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Aug 2022 12:27:15 +0200 Subject: [PATCH 029/100] 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 030/100] 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 031/100] 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 032/100] 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 033/100] 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 d81462c0952c35d5d4345883b881c16136d690de Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Tue, 23 Aug 2022 15:08:35 +0800 Subject: [PATCH 034/100] 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 ac3cf0beb4cce89bee54dec35848031ff64449aa Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 23 Aug 2022 20:58:57 +0800 Subject: [PATCH 035/100] 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 52ec268d75bce0a29c6883d829eaac63a76cc5bb Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 23 Aug 2022 21:26:15 +0800 Subject: [PATCH 036/100] 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 037/100] 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 038/100] 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 72bc451a8a6b59e75beed134806f8347f2db4da8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 16:50:48 +0200 Subject: [PATCH 039/100] 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 040/100] 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 041/100] 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 042/100] 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 043/100] 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 e85e885d40f1940d38b6b156a37146b6402387fd Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Thu, 25 Aug 2022 18:15:01 +0800 Subject: [PATCH 044/100] 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 10fdc1e286a717a368b2f0a9933f4ec83ec6aa05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Aug 2022 14:52:01 +0200 Subject: [PATCH 045/100] 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 704c4b2c5bf3d653378a2e513cfb56cc2917bb4a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Aug 2022 11:54:54 +0200 Subject: [PATCH 046/100] 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 40fb163e8cc84dab39607810cc6fe525b8b45f54 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Aug 2022 11:37:19 +0200 Subject: [PATCH 047/100] 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 92f88c1429be0ee9c352439cb140aea65a88dc83 Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Tue, 30 Aug 2022 20:02:47 +0800 Subject: [PATCH 048/100] 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 0e6b3b870acb12efbbd27eaef67c9ed1f267b704 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 13:08:27 +0000 Subject: [PATCH 049/100] 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 f043790e6dac08681fbe05c57c622f64599e3111 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 14:33:53 +0000 Subject: [PATCH 050/100] 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 acec516271f8f540048a9ddea56a64752d57004c Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Thu, 1 Sep 2022 19:31:01 +0800 Subject: [PATCH 051/100] 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 36454a340d857f3a983b5cd818642f6f981796fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Sep 2022 17:12:16 +0300 Subject: [PATCH 052/100] 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 053/100] 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 054/100] 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 055/100] 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 056/100] 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 dbe3b777ec51b886a07dc3202a2579fb6a10bf7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Sep 2022 17:07:56 +0200 Subject: [PATCH 057/100] 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 bcb6475c4ae70d6597367cd893b9d55da87a7e60 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 1 Sep 2022 18:30:55 +0200 Subject: [PATCH 058/100] 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 059/100] 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 e9232fc4e62a9146a809d1bbee2cb220fdf2b709 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 1 Sep 2022 13:23:34 +0000 Subject: [PATCH 060/100] 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 061/100] 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 59dccd6e4903f183c0319db8392017f0d9eb5ac0 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 26 Aug 2022 20:00:17 +0800 Subject: [PATCH 062/100] 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 f499381f5747938780532fe9b7f6edeef7591bb5 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 2 Sep 2022 18:45:37 +0800 Subject: [PATCH 063/100] 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 064/100] 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 065/100] 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 c9b512e33eed1a825e5b64d66cf7291eb7a004f2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 15:32:46 +0200 Subject: [PATCH 066/100] 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 3daf1f5c77190513761bedc986eae2db8a2ed207 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Sep 2022 16:57:51 +0200 Subject: [PATCH 067/100] 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 239847ad169ab36f8736c7be1598603693d099b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 17:48:36 +0200 Subject: [PATCH 068/100] 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 069/100] 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 19:18:44 +0200 Subject: [PATCH 070/100] 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 ec68ed8a9340873eb69df2c8100718125fe1b9da Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Sat, 3 Sep 2022 10:44:07 +0800 Subject: [PATCH 071/100] 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 712de530335ed915b259100848ab931c503d9eb2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 04:15:45 +0200 Subject: [PATCH 072/100] 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 2e85f9f0ade45d62b12c82eee93f410744e0611e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Aug 2022 22:26:10 +0200 Subject: [PATCH 073/100] 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 6c164905ca446f5a3bd69ca259f69decd33cd0ad Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Sep 2022 18:57:51 +0200 Subject: [PATCH 074/100] 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 075/100] 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 007680d93f9e9a0428828cd3b59273628bda82d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Sep 2022 02:13:51 +0200 Subject: [PATCH 076/100] 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 077/100] 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 078/100] 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 079/100] 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 080/100] 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 081/100] 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 082/100] 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 313747688e76825d33ead8190f0d285d7f4e2df2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Sep 2022 22:10:53 +0200 Subject: [PATCH 083/100] 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 084/100] 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 97de19d7ea4f89f615aa6f1158629fc695bcc718 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Sep 2022 12:12:22 +0200 Subject: [PATCH 085/100] 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 086/100] 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 330e3324b03ccba8afde778f76fb6e36d1045536 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 5 Sep 2022 12:46:26 +0000 Subject: [PATCH 087/100] 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 2724b6753706e38c7acac73ed9f83dce3fd03423 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Sep 2022 15:49:56 +0200 Subject: [PATCH 088/100] 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 089/100] 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 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 090/100] 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 091/100] 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 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 092/100] 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 093/100] 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 494dfff73c98ffd6c61d330b818f66f320ad368a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Sep 2022 02:04:18 +0200 Subject: [PATCH 094/100] 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 095/100] 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 6950016b8a83c990e7332decc8509fdf85132fbf Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 5 Sep 2022 19:06:16 +0800 Subject: [PATCH 096/100] 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 097/100] 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 7c4f42d014ba23a785f24f64e09710332532a049 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Sep 2022 13:58:26 +0200 Subject: [PATCH 098/100] 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 7032a1b267cb4d25fdc8325100b8443124a74fd5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 6 Sep 2022 11:14:55 -0400 Subject: [PATCH 099/100] 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 3073da9ba5c789890cd769fcbfbfdaa8606d7155 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 6 Sep 2022 18:39:06 -0400 Subject: [PATCH 100/100] 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.