From e21ab223c5ae7af17b6a1722b559dfcea17bf851 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 8 Mar 2022 16:11:21 +0800 Subject: [PATCH 001/233] finish dev --- programs/copier/ClusterCopierApp.cpp | 8 ++++++++ programs/extract-from-config/ExtractFromConfig.cpp | 14 ++++++++++++-- programs/server/Server.cpp | 6 ++++-- src/Interpreters/Context.cpp | 7 ++++--- 4 files changed, 28 insertions(+), 7 deletions(-) diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index d1ea59ed119..5f8bb84f47b 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -12,6 +12,11 @@ namespace fs = std::filesystem; namespace DB { +namespace ErrorCodes +{ + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; +} + /// ClusterCopierApp void ClusterCopierApp::initialize(Poco::Util::Application & self) @@ -192,6 +197,9 @@ void ClusterCopierApp::mainImpl() if (!task_file.empty()) copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); + if (config().has("zookeeper") || config().has("keeper")) + throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + copier->init(); copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); diff --git a/programs/extract-from-config/ExtractFromConfig.cpp b/programs/extract-from-config/ExtractFromConfig.cpp index 3fd665bcb26..e346ea83888 100644 --- a/programs/extract-from-config/ExtractFromConfig.cpp +++ b/programs/extract-from-config/ExtractFromConfig.cpp @@ -12,6 +12,10 @@ #include #include +namespace DB::ErrorCodes +{ + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; +} static void setupLogging(const std::string & log_level) { @@ -32,8 +36,14 @@ static std::string extractFromConfig( if (has_zk_includes && process_zk_includes) { DB::ConfigurationPtr bootstrap_configuration(new Poco::Util::XMLConfiguration(config_xml)); - zkutil::ZooKeeperPtr zookeeper = std::make_shared( - *bootstrap_configuration, "zookeeper", nullptr); + + if (bootstrap_configuration->has("zookeeper") && bootstrap_configuration->has("keeper")) + throw DB::Exception(DB::ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both zookeeper and keeper are specified"); + + zkutil::ZooKeeperPtr zookeeper; + zookeeper = std::make_shared( + *bootstrap_configuration, bootstrap_configuration->has("zookeeper") ? "zookeeper" : "keeper", nullptr); + zkutil::ZooKeeperNodeCache zk_node_cache([&] { return zookeeper; }); config_xml = processor.processConfig(&has_zk_includes, &zk_node_cache); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b856131d821..72b5eb6540a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -592,7 +592,9 @@ if (ThreadFuzzer::instance().isEffective()) ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10)); - bool has_zookeeper = config().has("zookeeper"); + if (config().has("zookeeper") && config().has("keeper")) + throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + bool has_zookeeper = config().has("zookeeper") || config().has("keeper"); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); zkutil::EventPtr main_config_zk_changed_event = std::make_shared(); @@ -955,7 +957,7 @@ if (ThreadFuzzer::instance().isEffective()) { /// We do not load ZooKeeper configuration on the first config loading /// because TestKeeper server is not started yet. - if (config->has("zookeeper")) + if (config->has("zookeeper") || config->has("keeper")) global_context->reloadZooKeeperIfChanged(config); global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b03875eed7a..756afed415a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1847,7 +1847,8 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const const auto & config = shared->zookeeper_config ? *shared->zookeeper_config : getConfigRef(); if (!shared->zookeeper) - shared->zookeeper = std::make_shared(config, "zookeeper", getZooKeeperLog()); + shared->zookeeper + = std::make_shared(config, config.has("zookeeper") ? "zookeeper" : "keeper", getZooKeeperLog()); else if (shared->zookeeper->expired()) shared->zookeeper = shared->zookeeper->startNewSession(); @@ -2058,7 +2059,7 @@ void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper_config = config; - reloadZooKeeperIfChangedImpl(config, "zookeeper", shared->zookeeper, getZooKeeperLog()); + reloadZooKeeperIfChangedImpl(config, config->has("zookeeper") ? "zookeeper" : "keeper", shared->zookeeper, getZooKeeperLog()); } void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config) @@ -2082,7 +2083,7 @@ void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & bool Context::hasZooKeeper() const { - return getConfigRef().has("zookeeper"); + return getConfigRef().has("zookeeper") || getConfigRef().has("keeper"); } bool Context::hasAuxiliaryZooKeeper(const String & name) const From d78493eeb2242531ead1b6fce8033cdd0220e6a4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 8 Mar 2022 16:41:16 +0800 Subject: [PATCH 002/233] commit again --- programs/copier/ClusterCopierApp.cpp | 2 +- programs/server/Server.cpp | 1 + src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- .../examples/get_abandonable_lock_in_all_partitions.cpp | 2 +- src/Storages/examples/get_current_inserts_in_replicated.cpp | 2 +- 6 files changed, 6 insertions(+), 5 deletions(-) diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 5f8bb84f47b..6e674473af3 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -197,7 +197,7 @@ void ClusterCopierApp::mainImpl() if (!task_file.empty()) copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); - if (config().has("zookeeper") || config().has("keeper")) + if (config().has("zookeeper") && config().has("keeper")) throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); copier->init(); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 72b5eb6540a..049d97f03f6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -594,6 +594,7 @@ if (ThreadFuzzer::instance().isEffective()) if (config().has("zookeeper") && config().has("keeper")) throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + bool has_zookeeper = config().has("zookeeper") || config().has("keeper"); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); diff --git a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp index cf819121234..e91296af61b 100644 --- a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp +++ b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp @@ -25,7 +25,7 @@ int main(int argc, char ** argv) DB::ConfigProcessor processor(argv[1], false, true); auto config = processor.loadConfig().configuration; - zkutil::ZooKeeper zk(*config, "zookeeper", nullptr); + zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : "keeper", nullptr); zkutil::EventPtr watch = std::make_shared(); /// NOTE: setting watches in multiple threads because doing it in a single thread is too slow. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 756afed415a..b7759e461ca 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1883,7 +1883,7 @@ bool Context::tryCheckClientConnectionToMyKeeperCluster() const try { /// If our server is part of main Keeper cluster - if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper")) + if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper") || checkZooKeeperConfigIsLocal(getConfigRef(), "keeper")) { LOG_DEBUG(shared->log, "Keeper server is participant of the main zookeeper cluster, will try to connect to it"); getZooKeeper(); diff --git a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp index e2145f5afb0..d947af71a60 100644 --- a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp +++ b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp @@ -26,7 +26,7 @@ try auto config = processor.loadConfig().configuration; String root_path = argv[2]; - zkutil::ZooKeeper zk(*config, "zookeeper", nullptr); + zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : "keeper", nullptr); String temp_path = root_path + "/temp"; String blocks_path = root_path + "/block_numbers"; diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index d7dedbcab9c..2355851bd89 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -29,7 +29,7 @@ try auto config = processor.loadConfig().configuration; String zookeeper_path = argv[2]; - auto zookeeper = std::make_shared(*config, "zookeeper", nullptr); + auto zookeeper = std::make_shared(*config, config->has("zookeeper") ? "zookeeper" : "keeper", nullptr); std::unordered_map> current_inserts; From 3b957367e160bb986d590fe1ccf98ca4951a4833 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 8 Mar 2022 17:03:24 +0800 Subject: [PATCH 003/233] add tests --- .../test_zookeeper_config/configs/zookeeper_config_root_a.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml b/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml index 6c413378524..d3c62862002 100644 --- a/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml +++ b/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml @@ -1,5 +1,5 @@ - + zoo1 2181 @@ -14,5 +14,5 @@ 3000 /root_a - + From e2bb06dbb5eab4db2326862cc6b4b9ee27d9ac5a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 8 Mar 2022 17:14:55 +0800 Subject: [PATCH 004/233] add tests --- tests/integration/helpers/zookeeper_secure_config.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/zookeeper_secure_config.xml b/tests/integration/helpers/zookeeper_secure_config.xml index b3b20eb3429..7f4e6f8a12b 100644 --- a/tests/integration/helpers/zookeeper_secure_config.xml +++ b/tests/integration/helpers/zookeeper_secure_config.xml @@ -1,5 +1,5 @@ - + zoo1 2281 @@ -13,5 +13,5 @@ 2281 3000 - + From 22a938fff02bb8592947c5f0ce604d5c96c76ec2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 8 Mar 2022 19:16:09 +0800 Subject: [PATCH 005/233] solve issue: https://github.com/ClickHouse/ClickHouse/issues/34767 --- programs/server/Server.cpp | 4 +- src/Common/ZooKeeper/ZooKeeper.cpp | 56 ++++++++++++++++--- .../examples/zk_many_watches_reconnect.cpp | 2 +- src/Interpreters/Context.cpp | 21 +++++-- ...get_abandonable_lock_in_all_partitions.cpp | 2 +- .../get_current_inserts_in_replicated.cpp | 7 ++- 6 files changed, 74 insertions(+), 18 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 049d97f03f6..1ed0d27c80d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -595,7 +595,7 @@ if (ThreadFuzzer::instance().isEffective()) if (config().has("zookeeper") && config().has("keeper")) throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); - bool has_zookeeper = config().has("zookeeper") || config().has("keeper"); + bool has_zookeeper = config().has("zookeeper") || config().has("keeper") || config().has("keeper_server"); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); zkutil::EventPtr main_config_zk_changed_event = std::make_shared(); @@ -958,7 +958,7 @@ if (ThreadFuzzer::instance().isEffective()) { /// We do not load ZooKeeper configuration on the first config loading /// because TestKeeper server is not started yet. - if (config->has("zookeeper") || config->has("keeper")) + if (config->has("zookeeper") || config->has("keeper") || config->has("keeper_server")) global_context->reloadZooKeeperIfChanged(config); global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b1574341c40..19647166b2a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -175,14 +175,32 @@ ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int3 struct ZooKeeperArgs { +public: 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"; + + if (endsWith(config_name, "keeper_server.raft_configuration")) + initFromSectionKeeperServer(config, config_name); + else + initFromSectionKeeper(config, config_name); + + 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(); + } + } + +private: + void initFromSectionKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_name, keys); for (const auto & key : keys) { if (startsWith(key, "node")) @@ -216,16 +234,38 @@ struct ZooKeeperArgs else throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); } + } - if (!chroot.empty()) + void initFromSectionKeeperServer(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_name, keys); + + bool secure = false; + for (const auto & key : keys) { - 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(); + if (key == "server") + { + hosts.push_back( + // (config.getBool(config_name + "." + key + ".secure", false) ? "secure://" : "") + + config.getString(config_name + "." + key + ".hostname") + ":" + + config.getString(config_name + "." + key + ".port", "9234") + ); + } + else if (key == "secure") + { + secure = config.getBool(config_name + "." + key, false); + } + } + + if (secure && !hosts.empty()) + { + for (auto & host : hosts) + host = "secure://" + host; } } +public: Strings hosts; std::string identity; int session_timeout_ms; diff --git a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp index e91296af61b..b68035b4c4c 100644 --- a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp +++ b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp @@ -25,7 +25,7 @@ int main(int argc, char ** argv) DB::ConfigProcessor processor(argv[1], false, true); auto config = processor.loadConfig().configuration; - zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : "keeper", nullptr); + zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : config->has("keeper") ? "keeper" : "keeper_server", nullptr); zkutil::EventPtr watch = std::make_shared(); /// NOTE: setting watches in multiple threads because doing it in a single thread is too slow. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b7759e461ca..a8efe834034 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1847,8 +1847,12 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const const auto & config = shared->zookeeper_config ? *shared->zookeeper_config : getConfigRef(); if (!shared->zookeeper) - shared->zookeeper - = std::make_shared(config, config.has("zookeeper") ? "zookeeper" : "keeper", getZooKeeperLog()); + shared->zookeeper = std::make_shared( + config, + config.has("zookeeper") ? "zookeeper" + : config.has("keeper") ? "keeper" + : "keeper_server", + getZooKeeperLog()); else if (shared->zookeeper->expired()) shared->zookeeper = shared->zookeeper->startNewSession(); @@ -1883,7 +1887,8 @@ bool Context::tryCheckClientConnectionToMyKeeperCluster() const try { /// If our server is part of main Keeper cluster - if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper") || checkZooKeeperConfigIsLocal(getConfigRef(), "keeper")) + if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper") || checkZooKeeperConfigIsLocal(getConfigRef(), "keeper") + || (!getConfigRef().has("zookeeper") && !getConfigRef().has("keeper") && getConfigRef().has("keeper_server"))) { LOG_DEBUG(shared->log, "Keeper server is participant of the main zookeeper cluster, will try to connect to it"); getZooKeeper(); @@ -2059,7 +2064,13 @@ void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper_config = config; - reloadZooKeeperIfChangedImpl(config, config->has("zookeeper") ? "zookeeper" : "keeper", shared->zookeeper, getZooKeeperLog()); + reloadZooKeeperIfChangedImpl( + config, + config->has("zookeeper") ? "zookeeper" + : config->has("keeper") ? "keeper" + : "keeper_server", + shared->zookeeper, + getZooKeeperLog()); } void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config) @@ -2083,7 +2094,7 @@ void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & bool Context::hasZooKeeper() const { - return getConfigRef().has("zookeeper") || getConfigRef().has("keeper"); + return getConfigRef().has("zookeeper") || getConfigRef().has("keeper") || getConfigRef().has("keeper_server"); } bool Context::hasAuxiliaryZooKeeper(const String & name) const diff --git a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp index d947af71a60..ee068d1bf26 100644 --- a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp +++ b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp @@ -26,7 +26,7 @@ try auto config = processor.loadConfig().configuration; String root_path = argv[2]; - zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : "keeper", nullptr); + zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : config->has("keeper") ? "keeper" : "keeper_server", nullptr); String temp_path = root_path + "/temp"; String blocks_path = root_path + "/block_numbers"; diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index 2355851bd89..c4ffa08ce27 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -29,7 +29,12 @@ try auto config = processor.loadConfig().configuration; String zookeeper_path = argv[2]; - auto zookeeper = std::make_shared(*config, config->has("zookeeper") ? "zookeeper" : "keeper", nullptr); + auto zookeeper = std::make_shared( + *config, + config->has("zookeeper") ? "zookeeper" + : config->has("keeper") ? "keeper" + : "keeper_server", + nullptr); std::unordered_map> current_inserts; From 08c3b361080ca2af99f6df1e143b5c9d8441b324 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 9 Mar 2022 12:25:30 +0800 Subject: [PATCH 006/233] commit again --- src/Common/ZooKeeper/ZooKeeper.cpp | 54 ++++++++++++++++++++---------- 1 file changed, 36 insertions(+), 18 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 19647166b2a..c4a562ae6b7 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -182,7 +182,7 @@ public: operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; implementation = "zookeeper"; - if (endsWith(config_name, "keeper_server.raft_configuration")) + if (endsWith(config_name, "keeper_server")) initFromSectionKeeperServer(config, config_name); else initFromSectionKeeper(config, config_name); @@ -238,30 +238,48 @@ private: void initFromSectionKeeperServer(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) { - Poco::Util::AbstractConfiguration::Keys keys; + Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_name, keys); - + bool secure = false; + String tcp_port; + String tcp_port_secure; for (const auto & key : keys) { - if (key == "server") + if (key == "tcp_port_secure") + { + secure = true; + tcp_port_secure = config.getString(config_name + "." + key); + } + else if (key == "tcp_port") + { + tcp_port = config.getString(config_name + "." + key); + } + else if (key == "coordination_settings") + { + if (config.has(config_name + "." + key + ".operation_timeout_ms")) + operation_timeout_ms = config.getInt(config_name + "." + key + ".operation_timeout_ms"); + if (config.has(config_name + "." + key + ".session_timeout_ms")) + session_timeout_ms = config.getInt(config_name + "." + key + ".session_timeout_ms"); + } + + /// TODO: consider digest + } + + if (secure && tcp_port_secure.empty()) + throw KeeperException("No tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); + if (!secure && tcp_port.empty()) + throw KeeperException("No tcp_port in config file", Coordination::Error::ZBADARGUMENTS); + + config.keys(config_name + ".raft_configuration", keys); + for (const auto & key : keys) + { + if (startsWith(key, "server")) { hosts.push_back( - // (config.getBool(config_name + "." + key + ".secure", false) ? "secure://" : "") + - config.getString(config_name + "." + key + ".hostname") + ":" - + config.getString(config_name + "." + key + ".port", "9234") - ); + (secure ? "secure://" : "") + config.getString(config_name + ".raft_configuration." + key + ".hostname") + ":" + + (secure ? tcp_port_secure : tcp_port)); } - else if (key == "secure") - { - secure = config.getBool(config_name + "." + key, false); - } - } - - if (secure && !hosts.empty()) - { - for (auto & host : hosts) - host = "secure://" + host; } } From 4e3098bbc21f13a827af6a4132341bfd3df1d2fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 28 Dec 2022 15:06:57 +0800 Subject: [PATCH 007/233] Update programs/extract-from-config/ExtractFromConfig.cpp Co-authored-by: Antonio Andelic --- programs/extract-from-config/ExtractFromConfig.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/extract-from-config/ExtractFromConfig.cpp b/programs/extract-from-config/ExtractFromConfig.cpp index 2d50ba240a7..75b0d311fdb 100644 --- a/programs/extract-from-config/ExtractFromConfig.cpp +++ b/programs/extract-from-config/ExtractFromConfig.cpp @@ -96,7 +96,7 @@ static std::vector extractFromConfig( DB::ConfigurationPtr bootstrap_configuration(new Poco::Util::XMLConfiguration(config_xml)); if (bootstrap_configuration->has("zookeeper") && bootstrap_configuration->has("keeper")) - throw DB::Exception(DB::ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both zookeeper and keeper are specified"); + throw DB::Exception(DB::ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both ZooKeeper and Keeper are specified"); zkutil::ZooKeeperPtr zookeeper; zookeeper = std::make_shared( From 24052916121e479233d4c3661c9182a109849dff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 28 Dec 2022 15:07:06 +0800 Subject: [PATCH 008/233] Update programs/copier/ClusterCopierApp.cpp Co-authored-by: Antonio Andelic --- programs/copier/ClusterCopierApp.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 024cab2a32d..e8d62224c48 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -198,7 +198,7 @@ void ClusterCopierApp::mainImpl() copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); if (config().has("zookeeper") && config().has("keeper")) - throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + throw Exception("Both ZooKeeper and Keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); copier->init(); copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); From 56b6378c121fd4ee17e8465ee21f47f502575235 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 28 Dec 2022 15:08:50 +0800 Subject: [PATCH 009/233] Update programs/server/Server.cpp Co-authored-by: Antonio Andelic --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 759e652562c..69e56abed16 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -767,7 +767,7 @@ try ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10)); if (config().has("zookeeper") && config().has("keeper")) - throw Exception("Both zookeeper and keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + throw Exception("Both ZooKeeper and Keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); bool has_zookeeper = config().has("zookeeper") || config().has("keeper") || config().has("keeper_server"); From 866c8d0fd0ced14a71b59625e521fcb49799e947 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Dec 2022 16:04:34 +0800 Subject: [PATCH 010/233] change as request --- src/Common/ZooKeeper/ZooKeeper.cpp | 13 +++++++++++++ src/Common/ZooKeeper/ZooKeeper.h | 2 ++ .../examples/zk_many_watches_reconnect.cpp | 2 +- src/Interpreters/Context.cpp | 15 ++------------- .../get_abandonable_lock_in_all_partitions.cpp | 2 +- .../get_current_inserts_in_replicated.cpp | 7 +------ 6 files changed, 20 insertions(+), 21 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index ab5d918e1f0..6bb7b465a6d 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; + extern const int NO_ELEMENTS_IN_CONFIG; } } @@ -1333,4 +1334,16 @@ String getSequentialNodeName(const String & prefix, UInt64 number) return name; } +String getZookeeperConfigName(const Poco::Util::AbstractConfiguration & config) +{ + if (config.has("zookeeper")) + return "zookeeper"; + else if (config.has("keeper")) + return "keeper"; + else if (config.has("keeper_server")) + return "keeper_server"; + else + throw DB::Exception("There is no Zookeeper configuration in server config", DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG); +} + } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 9de8241cfbe..67ae166452f 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -645,4 +645,6 @@ String extractZooKeeperPath(const String & path, bool check_starts_with_slash, P String getSequentialNodeName(const String & prefix, UInt64 number); +String getZookeeperConfigName(const Poco::Util::AbstractConfiguration & config); + } diff --git a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp index b68035b4c4c..aad8913ca8b 100644 --- a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp +++ b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp @@ -25,7 +25,7 @@ int main(int argc, char ** argv) DB::ConfigProcessor processor(argv[1], false, true); auto config = processor.loadConfig().configuration; - zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : config->has("keeper") ? "keeper" : "keeper_server", nullptr); + zkutil::ZooKeeper zk(*config, zkutil::getZookeeperConfigName(*config), nullptr); zkutil::EventPtr watch = std::make_shared(); /// NOTE: setting watches in multiple threads because doing it in a single thread is too slow. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5fb2715c6ce..2713ee5b50e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2233,12 +2233,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const const auto & config = shared->zookeeper_config ? *shared->zookeeper_config : getConfigRef(); if (!shared->zookeeper) - shared->zookeeper = std::make_shared( - config, - config.has("zookeeper") ? "zookeeper" - : config.has("keeper") ? "keeper" - : "keeper_server", - getZooKeeperLog()); + shared->zookeeper = std::make_shared(config, zkutil::getZookeeperConfigName(config), getZooKeeperLog()); else if (shared->zookeeper->expired()) { Stopwatch watch; @@ -2478,13 +2473,7 @@ void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper_config = config; - reloadZooKeeperIfChangedImpl( - config, - config->has("zookeeper") ? "zookeeper" - : config->has("keeper") ? "keeper" - : "keeper_server", - shared->zookeeper, - getZooKeeperLog()); + reloadZooKeeperIfChangedImpl(config, zkutil::getZookeeperConfigName(*config), shared->zookeeper, getZooKeeperLog()); } void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config) diff --git a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp index ee068d1bf26..e1faa67eb45 100644 --- a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp +++ b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp @@ -26,7 +26,7 @@ try auto config = processor.loadConfig().configuration; String root_path = argv[2]; - zkutil::ZooKeeper zk(*config, config->has("zookeeper") ? "zookeeper" : config->has("keeper") ? "keeper" : "keeper_server", nullptr); + zkutil::ZooKeeper zk(*config, zkutil::getZookeeperConfigName(*config), nullptr); String temp_path = root_path + "/temp"; String blocks_path = root_path + "/block_numbers"; diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index 36584b8593a..9ba75b81440 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -29,12 +29,7 @@ try auto config = processor.loadConfig().configuration; String zookeeper_path = argv[2]; - auto zookeeper = std::make_shared( - *config, - config->has("zookeeper") ? "zookeeper" - : config->has("keeper") ? "keeper" - : "keeper_server", - nullptr); + auto zookeeper = std::make_shared(*config, zkutil::getZookeeperConfigName(*config), nullptr); std::unordered_map> current_inserts; From 28f726f82532f85214d71d4de4358cf4dad39239 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Dec 2022 16:58:21 +0800 Subject: [PATCH 011/233] add zookeeper integration tests --- .../__init__.py | 0 .../configs/keeper_config.xml | 17 ++++++ .../configs/remote_servers.xml | 18 +++++++ .../configs/zookeeper_config.xml | 17 ++++++ .../test_alternative_keeper_config/test.py | 54 +++++++++++++++++++ 5 files changed, 106 insertions(+) create mode 100644 tests/integration/test_alternative_keeper_config/__init__.py create mode 100644 tests/integration/test_alternative_keeper_config/configs/keeper_config.xml create mode 100644 tests/integration/test_alternative_keeper_config/configs/remote_servers.xml create mode 100644 tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml create mode 100644 tests/integration/test_alternative_keeper_config/test.py diff --git a/tests/integration/test_alternative_keeper_config/__init__.py b/tests/integration/test_alternative_keeper_config/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml b/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml new file mode 100644 index 00000000000..bd783b83853 --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml @@ -0,0 +1,17 @@ + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 3000 + + diff --git a/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml b/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml new file mode 100644 index 00000000000..e77cc5c65e6 --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + + node1 + 9000 + + + + node2 + 9000 + + + + + + diff --git a/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml b/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml new file mode 100644 index 00000000000..7a0d7c1de92 --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml @@ -0,0 +1,17 @@ + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 3000 + + diff --git a/tests/integration/test_alternative_keeper_config/test.py b/tests/integration/test_alternative_keeper_config/test.py new file mode 100644 index 00000000000..86877e677ba --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/test.py @@ -0,0 +1,54 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from tests.integration.helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/keeper_config.xml"], + macros={"replica": "node1"}, +) + +node2 = cluster.add_instance( + "node2", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config.xml"], + macros={"replica": "node2"}, +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_create_insert(started_cluster): + node1.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'test_cluster' NO DELAY") + node1.query( + """ + CREATE TABLE tbl ON CLUSTER 'test_cluster' ( + id Int64, + str String + ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') + ORDER BY id + """ + ) + + node1.query("INSERT INTO tbl VALUES (1, 'str1')") + node2.query("INSERT INTO tbl VALUES (1, 'str1')") # Test deduplication + node2.query("INSERT INTO tbl VALUES (2, 'str2')") + + expected = [[1, "str1"], [2, "str2"]] + assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) + assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) + assert node1.query("CHECK TABLE tbl") == "1\n" + assert node2.query("CHECK TABLE tbl") == "1\n" From dc34393f668919684a2a1c20aec18aac8364e271 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Dec 2022 18:27:36 +0800 Subject: [PATCH 012/233] add another test --- .../configs_keeper_server/enable_keeper1.xml | 33 ++++++++++ .../configs_keeper_server/enable_keeper2.xml | 33 ++++++++++ .../configs_keeper_server/remote_servers.xml | 18 ++++++ .../configs_keeper_server/use_keeper.xml | 12 ++++ .../test_keeper_server.py | 61 +++++++++++++++++++ 5 files changed, 157 insertions(+) create mode 100644 tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml create mode 100644 tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml create mode 100644 tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml create mode 100644 tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml create mode 100644 tests/integration/test_alternative_keeper_config/test_keeper_server.py diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml b/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml new file mode 100644 index 00000000000..7c2e283e89f --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml @@ -0,0 +1,33 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + 75 + trace + + + + + 1 + node1 + 9234 + true + 3 + + + 2 + node2 + 9234 + true + true + 2 + + + + diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml b/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml new file mode 100644 index 00000000000..618e6a04aec --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml @@ -0,0 +1,33 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + 75 + trace + + + + + 1 + node1 + 9234 + true + 3 + + + 2 + node2 + 9234 + true + true + 2 + + + + diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml b/tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml new file mode 100644 index 00000000000..e77cc5c65e6 --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + + node1 + 9000 + + + + node2 + 9000 + + + + + + diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml b/tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml new file mode 100644 index 00000000000..b250f06cf81 --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml @@ -0,0 +1,12 @@ + + + + node1 + 9181 + + + node2 + 9181 + + + diff --git a/tests/integration/test_alternative_keeper_config/test_keeper_server.py b/tests/integration/test_alternative_keeper_config/test_keeper_server.py new file mode 100644 index 00000000000..7311b32e0af --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/test_keeper_server.py @@ -0,0 +1,61 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from tests.integration.helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + with_zookeeper=True, + main_configs=[ + "configs_keeper_server/remote_servers.xml", + "configs_keeper_server/enable_keeper1.xml", + "configs_keeper_server/use_keeper.xml", + ], + macros={"replica": "node1"}, +) + +node2 = cluster.add_instance( + "node2", + with_zookeeper=True, + main_configs=[ + "configs_keeper_server/remote_servers.xml", + "configs_keeper_server/enable_keeper2.xml", + ], + macros={"replica": "node2"}, +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_create_insert(started_cluster): + node1.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'test_cluster' NO DELAY") + node1.query( + """ + CREATE TABLE tbl ON CLUSTER 'test_cluster' ( + id Int64, + str String + ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') + ORDER BY id + """ + ) + + node1.query("INSERT INTO tbl VALUES (1, 'str1')") + node2.query("INSERT INTO tbl VALUES (1, 'str1')") # Test deduplication + node2.query("INSERT INTO tbl VALUES (2, 'str2')") + + expected = [[1, "str1"], [2, "str2"]] + assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) + assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) + assert node1.query("CHECK TABLE tbl") == "1\n" + assert node2.query("CHECK TABLE tbl") == "1\n" From 84f33a3114a9feb2421d27aa100f6f528534a317 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Dec 2022 14:56:09 +0800 Subject: [PATCH 013/233] commit again --- tests/integration/test_alternative_keeper_config/test.py | 4 ++-- .../test_alternative_keeper_config/test_keeper_server.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_alternative_keeper_config/test.py b/tests/integration/test_alternative_keeper_config/test.py index 86877e677ba..f2afc9c77ca 100644 --- a/tests/integration/test_alternative_keeper_config/test.py +++ b/tests/integration/test_alternative_keeper_config/test.py @@ -1,6 +1,6 @@ -import time +#!/usr/bin/env python3 + import pytest -import logging from helpers.cluster import ClickHouseCluster from tests.integration.helpers.test_tools import TSV diff --git a/tests/integration/test_alternative_keeper_config/test_keeper_server.py b/tests/integration/test_alternative_keeper_config/test_keeper_server.py index 7311b32e0af..711be4c7136 100644 --- a/tests/integration/test_alternative_keeper_config/test_keeper_server.py +++ b/tests/integration/test_alternative_keeper_config/test_keeper_server.py @@ -1,6 +1,6 @@ -import time +#!/usr/bin/env python3 + import pytest -import logging from helpers.cluster import ClickHouseCluster from tests.integration.helpers.test_tools import TSV From d5dcf9377599589a41d010c8e523a3f11fb82d32 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Dec 2022 16:37:20 +0800 Subject: [PATCH 014/233] fix import erorr --- tests/integration/test_alternative_keeper_config/test.py | 2 +- .../test_alternative_keeper_config/test_keeper_server.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_alternative_keeper_config/test.py b/tests/integration/test_alternative_keeper_config/test.py index f2afc9c77ca..8784cb9be56 100644 --- a/tests/integration/test_alternative_keeper_config/test.py +++ b/tests/integration/test_alternative_keeper_config/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -from tests.integration.helpers.test_tools import TSV +from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_alternative_keeper_config/test_keeper_server.py b/tests/integration/test_alternative_keeper_config/test_keeper_server.py index 711be4c7136..9c61e076671 100644 --- a/tests/integration/test_alternative_keeper_config/test_keeper_server.py +++ b/tests/integration/test_alternative_keeper_config/test_keeper_server.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -from tests.integration.helpers.test_tools import TSV +from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) From f9e81ca7dea94e019062655f08a70aaa152260a5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Feb 2023 23:30:16 +0300 Subject: [PATCH 015/233] secure in named collections - initial --- .../ClickHouseDictionarySource.cpp | 14 +- .../ExternalDataSourceConfiguration.cpp | 149 +++++++++++++----- .../ExternalDataSourceConfiguration.h | 3 +- 3 files changed, 126 insertions(+), 40 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index b962a6ab529..45ba865f2ff 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -230,6 +231,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) std::string table = config.getString(settings_config_prefix + ".table", ""); UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); }; + bool clickhouse_secure = config.getBool(settings_config_prefix + ".secure", false); auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context, has_config_key) @@ -245,6 +247,16 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) db = configuration.database; table = configuration.table; port = configuration.port; + clickhouse_secure = configuration.secure; + + // const auto & storage_specific_args = named_collection->specific_args; + // for (const auto & [arg_name, arg_value] : storage_specific_args) + // { + // if (arg_name == "secure") + // { + // clickhouse_secure = checkAndGetLiteralArgument(arg_value, "secure"); + // } + // } } ClickHouseDictionarySource::Configuration configuration{ @@ -261,7 +273,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), .port = port, .is_local = isLocalAddress({host, port}, default_port), - .secure = config.getBool(settings_config_prefix + ".secure", false)}; + .secure = clickhouse_secure}; // config.getBool(settings_config_prefix + ".secure", false)}; ContextMutablePtr context; diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index d7c3fe44f38..0f7964f8fcc 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -35,12 +35,11 @@ namespace ErrorCodes IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) -static const std::unordered_set dictionary_allowed_keys = { - "host", "port", "user", "password", "quota_key", "db", - "database", "table", "schema", "replica", - "update_field", "update_lag", "invalidate_query", "query", - "where", "name", "secure", "uri", "collection"}; - +// static const std::unordered_set dictionary_allowed_keys = { +// "host", "port", "user", "password", "quota_key", "db", +// "database", "table", "schema", "replica", +// "update_field", "update_lag", "invalidate_query", "query", +// "where", "name", "secure", "uri", "collection"}; template SettingsChanges getSettingsChangesFromConfig( @@ -83,6 +82,7 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration { host = conf.host; port = conf.port; + secure = conf.secure; username = conf.username; password = conf.password; quota_key = conf.quota_key; @@ -93,6 +93,54 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration addresses_expr = conf.addresses_expr; } +namespace +{ +void initExternalDataSourceConfiguration(ExternalDataSourceConfiguration & configuration) +{ + configuration = ExternalDataSourceConfiguration(); + configuration.username = ""; +} + +void readNamedCollection(const Poco::Util::AbstractConfiguration & config, + std::string_view collection_prefix, + ExternalDataSourceConfiguration & configuration) +{ + auto get_path = [collection_prefix](std::string_view fname) + { + return fmt::format("{}.{}", collection_prefix, fname); + }; + + configuration.host = config.getString(get_path("host"), configuration.host); + configuration.port = config.getInt(get_path("port"), configuration.port); + configuration.secure = config.getBool(get_path("secure"), configuration.secure); + configuration.username = config.getString(get_path("user"), configuration.username); + configuration.password = config.getString(get_path("password"), configuration.password); + configuration.quota_key = config.getString(get_path("quota_key"), configuration.quota_key); + configuration.database = config.getString(get_path("db"), config.getString(get_path("database"), configuration.database)); + configuration.table = config.getString(get_path("table"), config.getString(get_path("collection"), configuration.table)); + configuration.schema = config.getString(get_path("schema"), configuration.schema); + configuration.addresses_expr = config.getString(get_path("addresses_expr"), configuration.addresses_expr); +} + +using ConfigWithPrefix = std::pair; + +/// Logical priority is from left to right. +/// If first element of config_with_prefix_vect does not have a particular field, +/// second element is used, etc. +/// Technically values are overwritten from right to left. +/// If no luck, default values come into play. +void readNamedCollection(const std::vector & config_with_prefix_vect, + ExternalDataSourceConfiguration & configuration) +{ + initExternalDataSourceConfiguration(configuration); + + for (auto it = std::crbegin(config_with_prefix_vect); it != std::crend(config_with_prefix_vect); ++it) + { + readNamedCollection((*it).first, (*it).second, configuration); + } +} +} + template std::optional getExternalDataSourceConfiguration( @@ -121,15 +169,7 @@ std::optional getExternalDataSourceConfiguration( SettingsChanges config_settings = getSettingsChangesFromConfig(storage_settings, config, collection_prefix); - configuration.host = config.getString(collection_prefix + ".host", ""); - configuration.port = config.getInt(collection_prefix + ".port", 0); - configuration.username = config.getString(collection_prefix + ".user", ""); - configuration.password = config.getString(collection_prefix + ".password", ""); - configuration.quota_key = config.getString(collection_prefix + ".quota_key", ""); - configuration.database = config.getString(collection_prefix + ".database", ""); - configuration.table = config.getString(collection_prefix + ".table", config.getString(collection_prefix + ".collection", "")); - configuration.schema = config.getString(collection_prefix + ".schema", ""); - configuration.addresses_expr = config.getString(collection_prefix + ".addresses_expr", ""); + readNamedCollection({{config, collection_prefix}}, configuration); if (!configuration.addresses_expr.empty() && !configuration.host.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have `addresses_expr` and `host`, `port` in configuration at the same time"); @@ -169,6 +209,8 @@ std::optional getExternalDataSourceConfiguration( configuration.host = arg_value.safeGet(); else if (arg_name == "port") configuration.port = arg_value.safeGet(); + else if (arg_name == "secure") + configuration.secure = arg_value.safeGet(); else if (arg_name == "user") configuration.username = arg_value.safeGet(); else if (arg_name == "password") @@ -223,6 +265,7 @@ std::optional getExternalDataSourceConfiguration( { validateConfigKeys(dict_config, dict_config_prefix, has_config_key); ExternalDataSourceConfiguration configuration; + StorageSpecificArgs non_common_args; auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); if (!collection_name.empty()) @@ -238,15 +281,28 @@ std::optional getExternalDataSourceConfiguration( if (!config.has(collection_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); - configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(collection_prefix + ".host", "")); - configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0)); - configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", "")); - configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", "")); - configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", config.getString(collection_prefix + ".quota_key", "")); - configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database", - config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", "")))); - configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", "")); - configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", "")); + + // configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(collection_prefix + ".host", "")); + // configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0)); + // configuration.secure = dict_config.getInt(dict_config_prefix + ".secure", config.getUInt(collection_prefix + ".secure", 0)); + // configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", "")); + // configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", "")); + // configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", config.getString(collection_prefix + ".quota_key", "")); + // configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database", + // config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", "")))); + // configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", "")); + // configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", "")); + + + readNamedCollection({{dict_config, dict_config_prefix}, {config, collection_prefix}}, configuration); + + + // if (dict_config.has(dict_config_prefix + ".secure") || config.has(collection_prefix + ".secure")) + // { + // non_common_args.emplace_back(std::make_pair("secure", + // dict_config.getString(dict_config_prefix + ".secure", config.getString(collection_prefix + ".secure", "")))); + // } + if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.table.empty()) { @@ -254,7 +310,7 @@ std::optional getExternalDataSourceConfiguration( "Named collection of connection parameters is missing some " "of the parameters and dictionary parameters are not added"); } - return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = {}, .settings_changes = config_settings }; + return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = non_common_args, .settings_changes = config_settings }; } return std::nullopt; } @@ -333,14 +389,18 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( } else { - common_configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); - common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); - common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); - common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); - common_configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", ""); - common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", "")); - common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); - common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); + // common_configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); + // common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); + // common_configuration.secure = dict_config.getBool(dict_config_prefix + ".secure", 0); + // common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); + // common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); + // common_configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", ""); + // common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", "")); + // common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); + // common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); + + readNamedCollection({{dict_config, dict_config_prefix}}, common_configuration); + } ExternalDataSourcesByPriority configuration @@ -365,11 +425,17 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( validateConfigKeys(dict_config, replica_name, has_config_key); size_t priority = dict_config.getInt(replica_name + ".priority", 0); - replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); - replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port); - replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username); - replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password); - replica_configuration.quota_key = dict_config.getString(replica_name + ".quota_key", common_configuration.quota_key); + // replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); + // replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port); + // replica_configuration.secure = dict_config.getUInt(replica_name + ".secure", common_configuration.secure); + // replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username); + // replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password); + // replica_configuration.quota_key = dict_config.getString(replica_name + ".quota_key", common_configuration.quota_key); + + + readNamedCollection({{dict_config, replica_name}, + {dict_config, dict_config_prefix}}, replica_configuration); + if (replica_configuration.host.empty() || replica_configuration.port == 0 || replica_configuration.username.empty() || replica_configuration.password.empty()) @@ -378,6 +444,13 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( "Named collection of connection parameters is missing some " "of the parameters and no other dictionary parameters are added"); } + if (replica_configuration.database != common_configuration.database + || replica_configuration.table != common_configuration.table + || replica_configuration.schema != common_configuration.schema) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Named collection of connection parameters is not consistent"); + } configuration.replicas_configurations[priority].emplace_back(replica_configuration); } diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 65ef7d28b1b..6f116295bde 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -18,6 +18,7 @@ struct ExternalDataSourceConfiguration { String host; UInt16 port = 0; + bool secure = false; String username = "default"; String password; String quota_key; @@ -63,7 +64,7 @@ struct ExternalDataSourceInfo * configuration values, i.e. ENGINE = PostgreSQL(postgresql_configuration, database = 'postgres_database'); * * Any key-value engine argument except common (`host`, `port`, `username`, `password`, `database`) - * is returned in EngineArgs struct. + * is returned in StorageSpecificArgs struct. */ template std::optional getExternalDataSourceConfiguration( From 3b72b3f13b36bde864995ce492b0892f4f2d88b0 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 10 Feb 2023 13:42:11 +0300 Subject: [PATCH 016/233] secure in named collection - switched to specific_args, tests added --- .../ClickHouseDictionarySource.cpp | 36 +++-- .../ExternalDataSourceConfiguration.cpp | 54 ++------ .../ExternalDataSourceConfiguration.h | 1 - .../test_dictionaries_ddl/configs/client.xml | 10 ++ .../configs/named_coll.xml | 21 +++ .../test_dictionaries_ddl/configs/server.crt | 18 +++ .../test_dictionaries_ddl/configs/server.key | 28 ++++ .../configs/ssl_conf.xml | 16 +++ .../integration/test_dictionaries_ddl/test.py | 124 +++++++++++++++++- 9 files changed, 249 insertions(+), 59 deletions(-) create mode 100644 tests/integration/test_dictionaries_ddl/configs/client.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/named_coll.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/server.crt create mode 100644 tests/integration/test_dictionaries_ddl/configs/server.key create mode 100644 tests/integration/test_dictionaries_ddl/configs/ssl_conf.xml diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 45ba865f2ff..f3c5672a8b1 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -207,6 +207,18 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re } } +void logConfig(const Poco::Util::AbstractConfiguration & config, std::string start_from = "dictionary", size_t level = 0) +{ + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(start_from, keys); + for (const auto & key : keys) + { + LOG_TRACE(&Poco::Logger::get("ClickHouseDictionarySource"), + "logConfig key: {}{}", std::string(level, ' '), key); + logConfig(config, start_from + "." + key, level + 1); + } +} + void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, @@ -217,6 +229,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const std::string & default_database [[maybe_unused]], bool created_from_ddl) -> DictionarySourcePtr { + logConfig(config); + bool secure = config.getBool(config_prefix + ".secure", false); UInt16 default_port = getPortFromContext(global_context, secure); @@ -231,7 +245,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) std::string table = config.getString(settings_config_prefix + ".table", ""); UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); }; - bool clickhouse_secure = config.getBool(settings_config_prefix + ".secure", false); + bool secure_from_named = false; auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context, has_config_key) @@ -247,16 +261,16 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) db = configuration.database; table = configuration.table; port = configuration.port; - clickhouse_secure = configuration.secure; - // const auto & storage_specific_args = named_collection->specific_args; - // for (const auto & [arg_name, arg_value] : storage_specific_args) - // { - // if (arg_name == "secure") - // { - // clickhouse_secure = checkAndGetLiteralArgument(arg_value, "secure"); - // } - // } + const auto & storage_specific_args = named_collection->specific_args; + for (const auto & [arg_name, arg_value] : storage_specific_args) + { + if (arg_name == "secure") + { + secure_from_named = checkAndGetLiteralArgument(arg_value, "secure"); + LOG_TRACE(&Poco::Logger::get("ClickHouseDictionarySource"), "clickhouse_secure {}", secure_from_named); + } + } } ClickHouseDictionarySource::Configuration configuration{ @@ -273,7 +287,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), .port = port, .is_local = isLocalAddress({host, port}, default_port), - .secure = clickhouse_secure}; // config.getBool(settings_config_prefix + ".secure", false)}; + .secure = config.getBool(settings_config_prefix + ".secure", secure_from_named)}; ContextMutablePtr context; diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 0f7964f8fcc..20f8920ff5a 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -35,12 +35,6 @@ namespace ErrorCodes IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) -// static const std::unordered_set dictionary_allowed_keys = { -// "host", "port", "user", "password", "quota_key", "db", -// "database", "table", "schema", "replica", -// "update_field", "update_lag", "invalidate_query", "query", -// "where", "name", "secure", "uri", "collection"}; - template SettingsChanges getSettingsChangesFromConfig( const BaseSettings & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) @@ -82,7 +76,6 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration { host = conf.host; port = conf.port; - secure = conf.secure; username = conf.username; password = conf.password; quota_key = conf.quota_key; @@ -112,7 +105,6 @@ void readNamedCollection(const Poco::Util::AbstractConfiguration & config, configuration.host = config.getString(get_path("host"), configuration.host); configuration.port = config.getInt(get_path("port"), configuration.port); - configuration.secure = config.getBool(get_path("secure"), configuration.secure); configuration.username = config.getString(get_path("user"), configuration.username); configuration.password = config.getString(get_path("password"), configuration.password); configuration.quota_key = config.getString(get_path("quota_key"), configuration.quota_key); @@ -209,8 +201,8 @@ std::optional getExternalDataSourceConfiguration( configuration.host = arg_value.safeGet(); else if (arg_name == "port") configuration.port = arg_value.safeGet(); - else if (arg_name == "secure") - configuration.secure = arg_value.safeGet(); + // else if (arg_name == "secure") + // configuration.secure = arg_value.safeGet(); else if (arg_name == "user") configuration.username = arg_value.safeGet(); else if (arg_name == "password") @@ -281,27 +273,15 @@ std::optional getExternalDataSourceConfiguration( if (!config.has(collection_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); - - // configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(collection_prefix + ".host", "")); - // configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0)); - // configuration.secure = dict_config.getInt(dict_config_prefix + ".secure", config.getUInt(collection_prefix + ".secure", 0)); - // configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", "")); - // configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", "")); - // configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", config.getString(collection_prefix + ".quota_key", "")); - // configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database", - // config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", "")))); - // configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", "")); - // configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", "")); - - readNamedCollection({{dict_config, dict_config_prefix}, {config, collection_prefix}}, configuration); - // if (dict_config.has(dict_config_prefix + ".secure") || config.has(collection_prefix + ".secure")) - // { - // non_common_args.emplace_back(std::make_pair("secure", - // dict_config.getString(dict_config_prefix + ".secure", config.getString(collection_prefix + ".secure", "")))); - // } + if (dict_config.has(dict_config_prefix + ".secure") || config.has(collection_prefix + ".secure")) + { + uint64_t secure = dict_config.getBool(dict_config_prefix + ".secure", config.getBool(collection_prefix + ".secure", false)); + + non_common_args.emplace_back(std::make_pair("secure", std::make_shared(secure))); + } if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.table.empty()) @@ -389,18 +369,7 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( } else { - // common_configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); - // common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); - // common_configuration.secure = dict_config.getBool(dict_config_prefix + ".secure", 0); - // common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); - // common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); - // common_configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", ""); - // common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", "")); - // common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); - // common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); - readNamedCollection({{dict_config, dict_config_prefix}}, common_configuration); - } ExternalDataSourcesByPriority configuration @@ -425,13 +394,6 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( validateConfigKeys(dict_config, replica_name, has_config_key); size_t priority = dict_config.getInt(replica_name + ".priority", 0); - // replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); - // replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port); - // replica_configuration.secure = dict_config.getUInt(replica_name + ".secure", common_configuration.secure); - // replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username); - // replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password); - // replica_configuration.quota_key = dict_config.getString(replica_name + ".quota_key", common_configuration.quota_key); - readNamedCollection({{dict_config, replica_name}, {dict_config, dict_config_prefix}}, replica_configuration); diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 6f116295bde..3a841d45884 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -18,7 +18,6 @@ struct ExternalDataSourceConfiguration { String host; UInt16 port = 0; - bool secure = false; String username = "default"; String password; String quota_key; diff --git a/tests/integration/test_dictionaries_ddl/configs/client.xml b/tests/integration/test_dictionaries_ddl/configs/client.xml new file mode 100644 index 00000000000..15d83a7b1ab --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/client.xml @@ -0,0 +1,10 @@ + + + + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_dictionaries_ddl/configs/named_coll.xml b/tests/integration/test_dictionaries_ddl/configs/named_coll.xml new file mode 100644 index 00000000000..b14113a301d --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/named_coll.xml @@ -0,0 +1,21 @@ + + + + localhost + default + 9440 + 0 + + + localhost + default + 9440 + 1 + + + localhost + default + 9440 + + + diff --git a/tests/integration/test_dictionaries_ddl/configs/server.crt b/tests/integration/test_dictionaries_ddl/configs/server.crt new file mode 100644 index 00000000000..6f4deca038f --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/server.crt @@ -0,0 +1,18 @@ +-----BEGIN CERTIFICATE----- +MIIC+zCCAeOgAwIBAgIJAIhI9ozZJ+TWMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAeFw0xOTA0MjIwNDMyNTJaFw0yMDA0MjEwNDMyNTJaMBQx +EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC +ggEBAK+wVUEdqF2uXvN0MJBgnAHyXi6JTi4p/F6igsrCjSNjJWzHH0vQmK8ujfcF +CkifW88i+W5eHctuEtQqNHK+t9x9YiZtXrj6m/XkOXs20mYgENSmbbbHbriTPnZB +zZrq6UqMlwIHNNAa+I3NMORQxVRaI0ybXnGVO5elr70xHpk03xL0JWKHpEqYp4db +2aBQgF6y3Ww4khxjIYqpUYXWXGFnVIRU7FKVEAM1xyKqvQzXjQ5sVM/wyHknveEF +3b/X4ggN+KNl5KOc0cWDh1/XaatJAPaUUPqZcq76tynLbP64Xm3dxHcj+gtRkO67 +ef6MSg6l63m3XQP6Qb+MIkd06OsCAwEAAaNQME4wHQYDVR0OBBYEFDmODTO8QLDN +ykR3x0LIOnjNhrKhMB8GA1UdIwQYMBaAFDmODTO8QLDNykR3x0LIOnjNhrKhMAwG +A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAAwaiJc7uqEpnH3aukbftDwX +m8GfEnj1HVdgg+9GGNq+9rvUYBF6gdPmjRCX9dO0cclLFx8jc2org0rTSq9WoOhX +E6qL4Eqrmc5SE3Y9jZM0h6GRD4oXK014FmtZ3T6ddZU3dQLj3BS2r1XrvmubTvGN +ZuTJNY8nx8Hh6H5XINmsEjUF9E5hog+PwCE03xt2adIdYL+gsbxASeNYyeUFpZv5 +zcXR3VoakBWnAaOVgCHq2qh96QAnL7ZKzFkGf/MdwV10KU3dmb+ICbQUUdf9Gc17 +aaDCIRws312F433FdXBkGs2UkB7ZZme9dfn6O1QbeTNvex2VLMqYx/CTkfFbOQA= +-----END CERTIFICATE----- diff --git a/tests/integration/test_dictionaries_ddl/configs/server.key b/tests/integration/test_dictionaries_ddl/configs/server.key new file mode 100644 index 00000000000..6eddb3295db --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCvsFVBHahdrl7z +dDCQYJwB8l4uiU4uKfxeooLKwo0jYyVsxx9L0JivLo33BQpIn1vPIvluXh3LbhLU +KjRyvrfcfWImbV64+pv15Dl7NtJmIBDUpm22x264kz52Qc2a6ulKjJcCBzTQGviN +zTDkUMVUWiNMm15xlTuXpa+9MR6ZNN8S9CVih6RKmKeHW9mgUIBest1sOJIcYyGK +qVGF1lxhZ1SEVOxSlRADNcciqr0M140ObFTP8Mh5J73hBd2/1+IIDfijZeSjnNHF +g4df12mrSQD2lFD6mXKu+rcpy2z+uF5t3cR3I/oLUZDuu3n+jEoOpet5t10D+kG/ +jCJHdOjrAgMBAAECggEARF66zrxb6RkSmmt8+rKeA6PuQu3sHsr4C1vyyjUr97l9 +tvdGlpp20LWtSZQMjHZ3pARYTTsTHTeY3DgQcRcHNicVKx8k3ZepWeeW9vw+pL+V +zSt3RsoVrH6gsCSrfr4sS3aqzX9AbjwQvh48CJ3mLQ1m70kHV+xbZIh1+4pB/hyP +1wKyUE18ZkOptXvO/TtoHzLQCecpkXtWzmry1Eh2isvXA+NMrAtLibGsyM1mtm7i +5ozevzHabvvCDBEe+KgZdONgVhhhvm2eOd+/s4w3rw4ETud4fI/ZAJyWXhiIKFnA +VJbElWruSAoVBW7p2bsF5PbmVzvo8vXL+VylxYD+AQKBgQDhLoRKTVhNkn/QjKxq +sdOh+QZra0LzjVpAmkQzu7wZMSHEz9qePQciDQQrYKrmRF1vNcIRCVUTqWYheJ/1 +lKRrCGa0ab6k96zkWMqLHD5u+UeJV7r1dJIx08ME9kNJ+x/XtB8klRIji16NiQUS +qc6p8z0M2AnbJzsRfWZRH8FeYwKBgQDHu8dzdtVGI7MtxfPOE/bfajiopDg8BdTC +pdug2T8XofRHRq7Q+0vYjTAZFT/slib91Pk6VvvPdo9VBZiL4omv4dAq6mOOdX/c +U14mJe1X5GCrr8ExZ8BfNJ3t/6sV1fcxyJwAw7iBguqxA2JqdM/wFk10K8XqvzVn +CD6O9yGt2QKBgFX1BMi8N538809vs41S7l9hCQNOQZNo/O+2M5yv6ECRkbtoQKKw +1x03bMUGNJaLuELweXE5Z8GGo5bZTe5X3F+DKHlr+DtO1C+ieUaa9HY2MAmMdLCn +2/qrREGLo+oEs4YKmuzC/taUp/ZNPKOAMISNdluFyFVg51pozPrgrVbTAoGBAKkE +LBl3O67o0t0vH8sJdeVFG8EJhlS0koBMnfgVHqC++dm+5HwPyvTrNQJkyv1HaqNt +r6FArkG3ED9gRuBIyT6+lctbIPgSUip9mbQqcBfqOCvQxGksZMur2ODncz09HLtS +CUFUXjOqNzOnq4ZuZu/Bz7U4vXiSaXxQq6+LTUKxAoGAFZU/qrI06XxnrE9A1X0W +l7DSkpZaDcu11NrZ473yONih/xOZNh4SSBpX8a7F6Pmh9BdtGqphML8NFPvQKcfP +b9H2iid2tc292uyrUEb5uTMmv61zoTwtitqLzO0+tS6PT3fXobX+eyeEWKzPBljL +HFtxG5CCXpkdnWRmaJnhTzA= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_dictionaries_ddl/configs/ssl_conf.xml b/tests/integration/test_dictionaries_ddl/configs/ssl_conf.xml new file mode 100644 index 00000000000..e1b06aeb44b --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/ssl_conf.xml @@ -0,0 +1,16 @@ + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + none + true + true + sslv2,sslv3 + true + + + 9440 + diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index cb70deef72b..d728dead6a8 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -15,6 +15,13 @@ node1 = cluster.add_instance( "node1", with_mysql=True, dictionaries=["configs/dictionaries/simple_dictionary.xml"], + main_configs=[ + "configs/ssl_conf.xml", + "configs/client.xml", + "configs/named_coll.xml", + "configs/server.crt", + "configs/server.key", + ], user_configs=["configs/user_admin.xml", "configs/user_default.xml"], ) node2 = cluster.add_instance( @@ -108,6 +115,7 @@ def started_cluster(): ), ], ) + def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): mysql_conn = create_mysql_conn( "root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port @@ -207,7 +215,6 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): ) == str(hex(977))[2:] + "\n" clickhouse.query(f"drop dictionary default.{name}") - def test_restricted_database(started_cluster): for node in [node1, node2]: node.query("CREATE DATABASE IF NOT EXISTS restricted_db", user="admin") @@ -463,3 +470,118 @@ def test_clickhouse_remote(started_cluster): node3.query( "select dictGetUInt8('test.clickhouse_remote', 'SomeValue1', toUInt64(17))" ) == "17\n" + +# https://github.com/ClickHouse/ClickHouse/issues/38450 +# suggests placing 'secure' in named_collections +def test_secure(started_cluster): + node1.query("DROP TABLE IF EXISTS test.foo_dict") + node1.query("CREATE TABLE test.foo_dict(`id` UInt64, `value` String) ENGINE = Log") + node1.query("INSERT INTO test.foo_dict values (1, 'value1')") + + # No named collection, secure is set in DDL + node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") + node1.query( + """ + CREATE DICTIONARY test.clickhouse_secure( + id UInt64, + value String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE( + HOST 'localhost' + PORT 9440 USER 'default' + TABLE 'foo_dict' DB 'test' + SECURE 1 + )) + LIFETIME(MIN 1 MAX 10) + """ + ) + value = node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + assert(value == "value1\n") + + # Secure set in named collection + node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") + node1.query( + """ + CREATE DICTIONARY test.clickhouse_secure( + id UInt64, + value String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE( + NAME 'nc_secure_1' + TABLE 'foo_dict' DB 'test' + )) + LIFETIME(MIN 1 MAX 10) + """ + ) + value = node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + assert(value == "value1\n") + + # Secure is not set + node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") + node1.query( + """ + CREATE DICTIONARY test.clickhouse_secure( + id UInt64, + value String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE( + NAME 'nc_no_secure' + TABLE 'foo_dict' DB 'test' + )) + LIFETIME(MIN 1 MAX 10) + """ + ) + with pytest.raises(QueryRuntimeException) as excinfo: + node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + assert "Unexpected packet from server localhost:9440" in str(excinfo.value) + + # Secure is set to 0 in named collection + node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") + node1.query( + """ + CREATE DICTIONARY test.clickhouse_secure( + id UInt64, + value String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE( + NAME 'nc_secure_0' + TABLE 'foo_dict' DB 'test' + )) + LIFETIME(MIN 1 MAX 10) + """ + ) + with pytest.raises(QueryRuntimeException) as excinfo: + node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + assert "Unexpected packet from server localhost:9440" in str(excinfo.value) + + # Secure is set to 0 in named collection and in 1 in DDL + node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") + node1.query( + """ + CREATE DICTIONARY test.clickhouse_secure( + id UInt64, + value String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE( + NAME 'nc_secure_0' + TABLE 'foo_dict' DB 'test' + SECURE 1 + )) + LIFETIME(MIN 1 MAX 10) + """ + ) + value = node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + assert(value == "value1\n") + + node1.query("DROP DICTIONARY test.clickhouse_secure") + node1.query("DROP TABLE test.foo_dict") From 38ea27489c018771c3773449d77cd0e469c92693 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 13 Feb 2023 01:04:38 +0300 Subject: [PATCH 017/233] secure in named collections - small cleanup --- docs/en/operations/named-collections.md | 2 ++ src/Dictionaries/ClickHouseDictionarySource.cpp | 15 --------------- src/Storages/ExternalDataSourceConfiguration.cpp | 2 -- 3 files changed, 2 insertions(+), 17 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index f3cfa4a5372..0189cc0a8e2 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -244,10 +244,12 @@ Example of configuration: system foo secret + 1 ``` +`secure` is not needed for connection because of `remoteSecure`, but it can be used for dictionaries. ### Example of using named collections with the `remote`/`remoteSecure` functions diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index f3c5672a8b1..6c1304ea027 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -207,18 +207,6 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re } } -void logConfig(const Poco::Util::AbstractConfiguration & config, std::string start_from = "dictionary", size_t level = 0) -{ - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(start_from, keys); - for (const auto & key : keys) - { - LOG_TRACE(&Poco::Logger::get("ClickHouseDictionarySource"), - "logConfig key: {}{}", std::string(level, ' '), key); - logConfig(config, start_from + "." + key, level + 1); - } -} - void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, @@ -229,8 +217,6 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const std::string & default_database [[maybe_unused]], bool created_from_ddl) -> DictionarySourcePtr { - logConfig(config); - bool secure = config.getBool(config_prefix + ".secure", false); UInt16 default_port = getPortFromContext(global_context, secure); @@ -268,7 +254,6 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) if (arg_name == "secure") { secure_from_named = checkAndGetLiteralArgument(arg_value, "secure"); - LOG_TRACE(&Poco::Logger::get("ClickHouseDictionarySource"), "clickhouse_secure {}", secure_from_named); } } } diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 20f8920ff5a..a5b5a8ff72c 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -201,8 +201,6 @@ std::optional getExternalDataSourceConfiguration( configuration.host = arg_value.safeGet(); else if (arg_name == "port") configuration.port = arg_value.safeGet(); - // else if (arg_name == "secure") - // configuration.secure = arg_value.safeGet(); else if (arg_name == "user") configuration.username = arg_value.safeGet(); else if (arg_name == "password") From 00bcd272a89b1a8b8410b95a9094aae7ec6ff9a8 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 13 Feb 2023 10:54:28 +0300 Subject: [PATCH 018/233] secure in named collections - make style check happy --- src/Storages/ExternalDataSourceConfiguration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index a5b5a8ff72c..9f7b05979e1 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -278,7 +278,7 @@ std::optional getExternalDataSourceConfiguration( { uint64_t secure = dict_config.getBool(dict_config_prefix + ".secure", config.getBool(collection_prefix + ".secure", false)); - non_common_args.emplace_back(std::make_pair("secure", std::make_shared(secure))); + non_common_args.emplace_back(std::make_pair("secure", std::make_shared(secure))); } From f937dc3500ae7164019fd9b9eed448af472d085d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 13 Feb 2023 14:10:56 +0300 Subject: [PATCH 019/233] secure in named collections - test.py reformatted by black --- .../integration/test_dictionaries_ddl/test.py | 45 +++++++++++-------- 1 file changed, 26 insertions(+), 19 deletions(-) diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index d728dead6a8..29da9d7af7b 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -115,7 +115,6 @@ def started_cluster(): ), ], ) - def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): mysql_conn = create_mysql_conn( "root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port @@ -215,6 +214,7 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): ) == str(hex(977))[2:] + "\n" clickhouse.query(f"drop dictionary default.{name}") + def test_restricted_database(started_cluster): for node in [node1, node2]: node.query("CREATE DATABASE IF NOT EXISTS restricted_db", user="admin") @@ -471,6 +471,7 @@ def test_clickhouse_remote(started_cluster): "select dictGetUInt8('test.clickhouse_remote', 'SomeValue1', toUInt64(17))" ) == "17\n" + # https://github.com/ClickHouse/ClickHouse/issues/38450 # suggests placing 'secure' in named_collections def test_secure(started_cluster): @@ -481,7 +482,7 @@ def test_secure(started_cluster): # No named collection, secure is set in DDL node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") node1.query( - """ + """ CREATE DICTIONARY test.clickhouse_secure( id UInt64, value String @@ -496,14 +497,16 @@ def test_secure(started_cluster): )) LIFETIME(MIN 1 MAX 10) """ - ) - value = node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; - assert(value == "value1\n") + ) + value = node1.query( + "SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))" + ) + assert value == "value1\n" # Secure set in named collection node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") node1.query( - """ + """ CREATE DICTIONARY test.clickhouse_secure( id UInt64, value String @@ -516,14 +519,16 @@ def test_secure(started_cluster): )) LIFETIME(MIN 1 MAX 10) """ - ) - value = node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; - assert(value == "value1\n") + ) + value = node1.query( + "SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))" + ) + assert value == "value1\n" # Secure is not set node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") node1.query( - """ + """ CREATE DICTIONARY test.clickhouse_secure( id UInt64, value String @@ -536,15 +541,15 @@ def test_secure(started_cluster): )) LIFETIME(MIN 1 MAX 10) """ - ) + ) with pytest.raises(QueryRuntimeException) as excinfo: - node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") assert "Unexpected packet from server localhost:9440" in str(excinfo.value) # Secure is set to 0 in named collection node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") node1.query( - """ + """ CREATE DICTIONARY test.clickhouse_secure( id UInt64, value String @@ -557,15 +562,15 @@ def test_secure(started_cluster): )) LIFETIME(MIN 1 MAX 10) """ - ) + ) with pytest.raises(QueryRuntimeException) as excinfo: - node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; + node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") assert "Unexpected packet from server localhost:9440" in str(excinfo.value) # Secure is set to 0 in named collection and in 1 in DDL node1.query("DROP DICTIONARY IF EXISTS test.clickhouse_secure") node1.query( - """ + """ CREATE DICTIONARY test.clickhouse_secure( id UInt64, value String @@ -579,9 +584,11 @@ def test_secure(started_cluster): )) LIFETIME(MIN 1 MAX 10) """ - ) - value = node1.query("SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))") ; - assert(value == "value1\n") + ) + value = node1.query( + "SELECT dictGet('test.clickhouse_secure', 'value', toUInt64(1))" + ) + assert value == "value1\n" node1.query("DROP DICTIONARY test.clickhouse_secure") node1.query("DROP TABLE test.foo_dict") From 85e0a57974b4000807def54ce1f88648646cb4ec Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 1 Mar 2023 15:37:20 +0000 Subject: [PATCH 020/233] Add function ULIDStringToDateTime --- src/Functions/FunctionsCodingULID.cpp | 160 ++++++++++++++++++ .../0_stateless/02668_ulid_decoding.reference | 1 + .../0_stateless/02668_ulid_decoding.sql | 3 + 3 files changed, 164 insertions(+) create mode 100644 src/Functions/FunctionsCodingULID.cpp create mode 100644 tests/queries/0_stateless/02668_ulid_decoding.reference create mode 100644 tests/queries/0_stateless/02668_ulid_decoding.sql diff --git a/src/Functions/FunctionsCodingULID.cpp b/src/Functions/FunctionsCodingULID.cpp new file mode 100644 index 00000000000..cc4210e1480 --- /dev/null +++ b/src/Functions/FunctionsCodingULID.cpp @@ -0,0 +1,160 @@ +#include "config.h" + +#if USE_ULID + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + +class FunctionULIDStringToDateTime : public IFunction +{ +public: + static constexpr size_t ULID_LENGTH = 26; + static constexpr UInt32 DATETIME_SCALE = 3; + + static constexpr auto name = "ULIDStringToDateTime"; + + static FunctionPtr create(ContextPtr /*context*/) + { + return std::make_shared(); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: 1 expected.", getName()); + + const auto * arg_fixed_string = checkAndGetDataType(arguments[0].type.get()); + const auto * arg_string = checkAndGetDataType(arguments[0].type.get()); + + if (!arg_string && !(arg_fixed_string && arg_fixed_string->getN() == ULID_LENGTH)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}. Must be String or FixedString(26).", + arguments[0].type->getName(), + getName()); + + return std::make_shared(DATETIME_SCALE, ""); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnDateTime64::create(input_rows_count, DATETIME_SCALE); + auto & vec_res = col_res->getData(); + + const ColumnPtr column = arguments[0].column; + + const auto * column_fixed_string = checkAndGetColumn(column.get()); + const auto * column_string = checkAndGetColumn(column.get()); + + if (column_fixed_string) + { + if (column_fixed_string->getN() != ULID_LENGTH) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}, expected String or FixedString({})", + arguments[0].name, getName(), ULID_LENGTH + ); + + const auto & vec_src = column_fixed_string->getChars(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + DateTime64 time = decode(vec_src.data() + i * ULID_LENGTH); + vec_res[i] = time; + } + } + else if (column_string) + { + const auto & vec_src = column_string->getChars(); + const auto & offsets_src = column_string->getOffsets(); + + size_t src_offset = 0; + + for (size_t i = 0; i < input_rows_count; ++i) + { + DateTime64 time = 0; + + size_t string_size = offsets_src[i] - src_offset; + if (string_size == ULID_LENGTH + 1) + time = decode(vec_src.data() + offsets_src[i]); + + src_offset += string_size; + vec_res[i] = time; + } + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}, expected String or FixedString({})", + arguments[0].name, getName(), ULID_LENGTH + ); + + return col_res; + } + + static DateTime64 decode(const UInt8 * data) + { + unsigned char buffer[16]; + int ret = ulid_decode(buffer, reinterpret_cast(data)); + if (ret != 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot decode ULID"); + + /// Timestamp in milliseconds is the first 48 bits of the decoded ULID + Int64 ms = 0; + memcpy(reinterpret_cast(&ms) + 2, buffer, 6); + +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ + std::reverse(reinterpret_cast(&ms), reinterpret_cast(&ms) + sizeof(Int64)); +#endif + + return DecimalUtils::decimalFromComponents(ms / intExp10(DATETIME_SCALE), ms % intExp10(DATETIME_SCALE), DATETIME_SCALE); + } +}; + + +REGISTER_FUNCTION(ULIDStringToDateTime) +{ + factory.registerFunction( + { + R"( +Decodes ULID and returns its timestammp as DateTime64(3). +This function takes as an argument ULID of type String or FixedString(26). +)", + Documentation::Examples{{"ulid", "SELECT ULIDStringToDateTime(generateULID())"}}, + Documentation::Categories{"ULID"} + }, + FunctionFactory::CaseSensitive); +} + +} + +#endif diff --git a/tests/queries/0_stateless/02668_ulid_decoding.reference b/tests/queries/0_stateless/02668_ulid_decoding.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02668_ulid_decoding.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02668_ulid_decoding.sql b/tests/queries/0_stateless/02668_ulid_decoding.sql new file mode 100644 index 00000000000..042c8e52b36 --- /dev/null +++ b/tests/queries/0_stateless/02668_ulid_decoding.sql @@ -0,0 +1,3 @@ +-- Tags: no-fasttest + +SELECT dateDiff('s', ULIDStringToDateTime(generateULID()), now()) = 0; From 5f5e15f8c121bffd3ae511f4f038dc3b8bd65dd7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 7 Mar 2023 18:28:10 +0000 Subject: [PATCH 021/233] Update sort description to respect aliases in header --- src/Processors/QueryPlan/ExpressionStep.cpp | 60 +++++++++++++++++++ src/Processors/QueryPlan/QueryPlan.cpp | 19 ++++++ ..._input_stream_properties_explain.reference | 4 +- ...ting_by_input_stream_properties_explain.sh | 4 -- 4 files changed, 81 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index dcfa6e5a891..3557e9aa685 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -87,10 +87,70 @@ void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Expression", expression->toTree()); } +namespace +{ + const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name) + { + /// find alias in output + const ActionsDAG::Node * output_alias = nullptr; + for (const auto * node : actions->getOutputs()) + { + if (node->result_name == output_name) + { + output_alias = node; + break; + } + } + if (!output_alias) + { + // logDebug("getOriginalNodeForOutputAlias: no output alias found", output_name); + return nullptr; + } + + /// find original(non alias) node it refers to + const ActionsDAG::Node * node = output_alias; + while (node && node->type == ActionsDAG::ActionType::ALIAS) + { + chassert(!node->children.empty()); + node = node->children.front(); + } + if (node && node->type != ActionsDAG::ActionType::INPUT) + return nullptr; + + return node; + } + +} + void ExpressionStep::updateOutputStream() { output_stream = createOutputStream( input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, *actions_dag), getDataStreamTraits()); + + const ActionsDAGPtr & actions = actions_dag; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "ActionsDAG dump:\n{}", actions->dumpDAG()); + + const auto & input_sort_description = getInputStreams().front().sort_description; + for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) + { + const auto & desc = input_sort_description[i]; + String alias; + const auto & origin_column = desc.column_name; + for (const auto & column : output_stream->header) + { + const auto * original_node = getOriginalNodeForOutputAlias(actions, column.name); + if (original_node && original_node->result_name == origin_column) + { + alias = column.name; + break; + } + } + + if (alias.empty()) + return; + + output_stream->sort_description[i].column_name = alias; + } } } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 8b666bba7da..601952e6944 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include @@ -452,6 +453,22 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio } } +void updateRootInputStream(QueryPlan::Node & root) +{ + auto* to = root.step.get(); + const auto * from = root.children.front()->step.get(); + + auto * to_update = dynamic_cast(to); + if (!to_update) + return; + const auto * update_from = dynamic_cast(from); + if (!update_from) + return; + + if (update_from->hasOutputStream()) + to_update->updateInputStream(update_from->getOutputStream()); +} + void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { /// optimization need to be applied before "mergeExpressions" optimization @@ -462,6 +479,8 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); + + updateRootInputStream(*root); } void QueryPlan::explainEstimate(MutableColumns & columns) 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 9d78707429e..0b34e878cee 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 @@ -74,10 +74,10 @@ Sorting (Stream): a ASC Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;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 (Global): x ASC, y ASC +Sorting (Global): a ASC, b ASC Sorting (Sorting for ORDER BY) Sorting (Global): x ASC, y ASC -Sorting (Stream): a ASC, b ASC +Sorting (Stream): x ASC, y ASC Sorting (Stream): 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;set max_threads=3;set query_plan_remove_redundant_sorting=0;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 diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index 0678ff63e3f..e4cab987d35 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -62,10 +62,6 @@ explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting= echo "-- aliases break sorting order" explain_sortmode "$MAKE_OUTPUT_STABLE;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" -# FIXME: we still do full sort here, - it's because, for most inner subqueury, sorting description contains original column names but header contains only aliases on those columns: -#| Header: x Int32 │ -#│ y Int32 │ -#│ Sort Mode: Chunk: a ASC, b ASC │ echo "-- aliases DONT break sorting order" explain_sortmode "$MAKE_OUTPUT_STABLE;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)" From e88912009a901ddf04757038d326241de4d915d8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 7 Mar 2023 20:49:55 +0000 Subject: [PATCH 022/233] Update data streams for entire plan --- src/Processors/QueryPlan/ExpressionStep.cpp | 3 + src/Processors/QueryPlan/QueryPlan.cpp | 34 +++--- src/Processors/QueryPlan/QueryPlanVisitor.h | 111 ++++++++++++++++++++ 3 files changed, 136 insertions(+), 12 deletions(-) create mode 100644 src/Processors/QueryPlan/QueryPlanVisitor.h diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 3557e9aa685..1425f281d22 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -127,6 +127,9 @@ void ExpressionStep::updateOutputStream() output_stream = createOutputStream( input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, *actions_dag), getDataStreamTraits()); + if (!getDataStreamTraits().preserves_sorting) + return; + const ActionsDAGPtr & actions = actions_dag; LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "ActionsDAG dump:\n{}", actions->dumpDAG()); diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 601952e6944..3d5bd0db6d0 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include @@ -453,20 +454,29 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio } } -void updateRootInputStream(QueryPlan::Node & root) +static void updateDataStreams(QueryPlan::Node & root) { - auto* to = root.step.get(); - const auto * from = root.children.front()->step.get(); + class UpdateDataStreams : public QueryPlanVisitor + { + public: + explicit UpdateDataStreams(QueryPlan::Node * root_) : QueryPlanVisitor(root_) { } - auto * to_update = dynamic_cast(to); - if (!to_update) - return; - const auto * update_from = dynamic_cast(from); - if (!update_from) - return; + static bool visitTopDownImpl(QueryPlan::Node * /*current_node*/, QueryPlan::Node * /*parent_node*/) { return true; } - if (update_from->hasOutputStream()) - to_update->updateInputStream(update_from->getOutputStream()); + static void visitBottomUpImpl(QueryPlan::Node * current_node, QueryPlan::Node * parent_node) + { + if (!parent_node || parent_node->children.size() != 1) + return; + + if (!current_node->step->hasOutputStream()) + return; + + if (auto * parent_transform_step = dynamic_cast(parent_node->step.get()); parent_transform_step) + parent_transform_step->updateInputStream(current_node->step->getOutputStream()); + } + }; + + UpdateDataStreams(&root).visit(); } void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) @@ -480,7 +490,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); - updateRootInputStream(*root); + updateDataStreams(*root); } void QueryPlan::explainEstimate(MutableColumns & columns) diff --git a/src/Processors/QueryPlan/QueryPlanVisitor.h b/src/Processors/QueryPlan/QueryPlanVisitor.h new file mode 100644 index 00000000000..0f265216649 --- /dev/null +++ b/src/Processors/QueryPlan/QueryPlanVisitor.h @@ -0,0 +1,111 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +template +class QueryPlanVisitor +{ +protected: + struct FrameWithParent + { + QueryPlan::Node * node = nullptr; + QueryPlan::Node * parent_node = nullptr; + size_t next_child = 0; + }; + + using StackWithParent = std::vector; + + QueryPlan::Node * root = nullptr; + StackWithParent stack; + +public: + explicit QueryPlanVisitor(QueryPlan::Node * root_) : root(root_) { } + + void visit() + { + stack.push_back({.node = root}); + + while (!stack.empty()) + { + auto & frame = stack.back(); + + QueryPlan::Node * current_node = frame.node; + QueryPlan::Node * parent_node = frame.parent_node; + + logStep("back", current_node); + + /// top-down visit + if (0 == frame.next_child) + { + logStep("top-down", current_node); + if (!visitTopDown(current_node, parent_node)) + continue; + } + /// Traverse all children + if (frame.next_child < frame.node->children.size()) + { + auto next_frame = FrameWithParent{.node = current_node->children[frame.next_child], .parent_node = current_node}; + ++frame.next_child; + logStep("push", next_frame.node); + stack.push_back(next_frame); + continue; + } + + /// bottom-up visit + logStep("bottom-up", current_node); + visitBottomUp(current_node, parent_node); + + logStep("pop", current_node); + stack.pop_back(); + } + } + + bool visitTopDown(QueryPlan::Node * current_node, QueryPlan::Node * parent_node) + { + return getDerived().visitTopDownImpl(current_node, parent_node); + } + void visitBottomUp(QueryPlan::Node * current_node, QueryPlan::Node * parent_node) + { + getDerived().visitBottomUpImpl(current_node, parent_node); + } + +private: + Derived & getDerived() { return *static_cast(this); } + + const Derived & getDerived() const { return *static_cast(this); } + + std::unordered_map address2name; + std::unordered_map name_gen; + + std::string getStepId(const IQueryPlanStep* step) + { + const auto step_name = step->getName(); + auto it = address2name.find(step); + if (it != address2name.end()) + return it->second; + + const auto seq_num = name_gen[step_name]++; + return address2name.insert({step, fmt::format("{}{}", step_name, seq_num)}).first->second; + } + +protected: + void logStep(const char * prefix, const QueryPlan::Node * node) + { + if constexpr (debug_logging) + { + const IQueryPlanStep * current_step = node->step.get(); + LOG_DEBUG( + &Poco::Logger::get("QueryPlanVisitor"), + "{}: {}: {}", + prefix, + getStepId(current_step), + reinterpret_cast(current_step)); + } + } +}; + +} From 6f2851ef702268f3813251d7dfa32bae805707c9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 7 Mar 2023 22:16:22 +0000 Subject: [PATCH 023/233] Update FilterStep --- src/Interpreters/ActionsDAG.cpp | 28 +++++ src/Interpreters/ActionsDAG.h | 2 + src/Processors/QueryPlan/ExpressionStep.cpp | 35 ------ src/Processors/QueryPlan/FilterStep.cpp | 28 +++++ .../Optimizations/removeRedundantDistinct.cpp | 31 ------ .../Optimizations/removeRedundantSorting.cpp | 103 +----------------- 6 files changed, 59 insertions(+), 168 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index daba4c1608d..afa8071adfb 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2288,4 +2288,32 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( return result_dag; } +const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name) +{ + /// find alias in output + const ActionsDAG::Node * output_alias = nullptr; + for (const auto * node : actions->getOutputs()) + { + if (node->result_name == output_name) + { + output_alias = node; + break; + } + } + if (!output_alias) + return nullptr; + + /// find original(non alias) node it refers to + const ActionsDAG::Node * node = output_alias; + while (node && node->type == ActionsDAG::ActionType::ALIAS) + { + chassert(!node->children.empty()); + node = node->children.front(); + } + if (node && node->type != ActionsDAG::ActionType::INPUT) + return nullptr; + + return node; +} + } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 0182db8e027..b20d4959bc9 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -367,6 +367,8 @@ private: static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); }; +const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name); + /// This is an ugly way to bypass impossibility to forward declare ActionDAG::Node. struct ActionDAGNodes { diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 1425f281d22..85b466cd934 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -87,41 +87,6 @@ void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Expression", expression->toTree()); } -namespace -{ - const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name) - { - /// find alias in output - const ActionsDAG::Node * output_alias = nullptr; - for (const auto * node : actions->getOutputs()) - { - if (node->result_name == output_name) - { - output_alias = node; - break; - } - } - if (!output_alias) - { - // logDebug("getOriginalNodeForOutputAlias: no output alias found", output_name); - return nullptr; - } - - /// find original(non alias) node it refers to - const ActionsDAG::Node * node = output_alias; - while (node && node->type == ActionsDAG::ActionType::ALIAS) - { - chassert(!node->children.empty()); - node = node->children.front(); - } - if (node && node->type != ActionsDAG::ActionType::INPUT) - return nullptr; - - return node; - } - -} - void ExpressionStep::updateOutputStream() { output_stream = createOutputStream( diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 4699a7c1908..6f26939ff4e 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -120,6 +120,34 @@ void FilterStep::updateOutputStream() input_streams.front(), FilterTransform::transformHeader(input_streams.front().header, actions_dag.get(), filter_column_name, remove_filter_column), getDataStreamTraits()); + + if (!getDataStreamTraits().preserves_sorting) + return; + + const ActionsDAGPtr & actions = actions_dag; + // LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "ActionsDAG dump:\n{}", actions->dumpDAG()); + + const auto & input_sort_description = getInputStreams().front().sort_description; + for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) + { + const auto & desc = input_sort_description[i]; + String alias; + const auto & origin_column = desc.column_name; + for (const auto & column : output_stream->header) + { + const auto * original_node = getOriginalNodeForOutputAlias(actions, column.name); + if (original_node && original_node->result_name == origin_column) + { + alias = column.name; + break; + } + } + + if (alias.empty()) + return; + + output_stream->sort_description[i].column_name = alias; + } } } diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index 02725dc3122..6685e46d780 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -63,37 +63,6 @@ namespace return non_const_columns; } - const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name) - { - /// find alias in output - const ActionsDAG::Node * output_alias = nullptr; - for (const auto * node : actions->getOutputs()) - { - if (node->result_name == output_name) - { - output_alias = node; - break; - } - } - if (!output_alias) - { - logDebug("getOriginalNodeForOutputAlias: no output alias found", output_name); - return nullptr; - } - - /// find original(non alias) node it refers to - const ActionsDAG::Node * node = output_alias; - while (node && node->type == ActionsDAG::ActionType::ALIAS) - { - chassert(!node->children.empty()); - node = node->children.front(); - } - if (node && node->type != ActionsDAG::ActionType::INPUT) - return nullptr; - - return node; - } - bool compareAggregationKeysWithDistinctColumns( const Names & aggregation_keys, const DistinctColumns & distinct_columns, const ActionsDAGPtr & path_actions) { diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index 41e30dee83e..188d7f6f117 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -18,108 +19,6 @@ namespace DB::QueryPlanOptimizations { -template -class QueryPlanVisitor -{ -protected: - struct FrameWithParent - { - QueryPlan::Node * node = nullptr; - QueryPlan::Node * parent_node = nullptr; - size_t next_child = 0; - }; - - using StackWithParent = std::vector; - - QueryPlan::Node * root = nullptr; - StackWithParent stack; - -public: - explicit QueryPlanVisitor(QueryPlan::Node * root_) : root(root_) { } - - void visit() - { - stack.push_back({.node = root}); - - while (!stack.empty()) - { - auto & frame = stack.back(); - - QueryPlan::Node * current_node = frame.node; - QueryPlan::Node * parent_node = frame.parent_node; - - logStep("back", current_node); - - /// top-down visit - if (0 == frame.next_child) - { - logStep("top-down", current_node); - if (!visitTopDown(current_node, parent_node)) - continue; - } - /// Traverse all children - if (frame.next_child < frame.node->children.size()) - { - auto next_frame = FrameWithParent{.node = current_node->children[frame.next_child], .parent_node = current_node}; - ++frame.next_child; - logStep("push", next_frame.node); - stack.push_back(next_frame); - continue; - } - - /// bottom-up visit - logStep("bottom-up", current_node); - visitBottomUp(current_node, parent_node); - - logStep("pop", current_node); - stack.pop_back(); - } - } - - bool visitTopDown(QueryPlan::Node * current_node, QueryPlan::Node * parent_node) - { - return getDerived().visitTopDownImpl(current_node, parent_node); - } - void visitBottomUp(QueryPlan::Node * current_node, QueryPlan::Node * parent_node) - { - getDerived().visitBottomUpImpl(current_node, parent_node); - } - -private: - Derived & getDerived() { return *static_cast(this); } - - const Derived & getDerived() const { return *static_cast(this); } - - std::unordered_map address2name; - std::unordered_map name_gen; - - std::string getStepId(const IQueryPlanStep* step) - { - const auto step_name = step->getName(); - auto it = address2name.find(step); - if (it != address2name.end()) - return it->second; - - const auto seq_num = name_gen[step_name]++; - return address2name.insert({step, fmt::format("{}{}", step_name, seq_num)}).first->second; - } - -protected: - void logStep(const char * prefix, const QueryPlan::Node * node) - { - if constexpr (debug_logging) - { - const IQueryPlanStep * current_step = node->step.get(); - LOG_DEBUG( - &Poco::Logger::get("QueryPlanVisitor"), - "{}: {}: {}", - prefix, - getStepId(current_step), - reinterpret_cast(current_step)); - } - } -}; - constexpr bool debug_logging_enabled = false; class RemoveRedundantSorting : public QueryPlanVisitor From 79d4d029b87ebb9dd3cd61e3774ef16dbbc133a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 7 Mar 2023 22:40:11 +0000 Subject: [PATCH 024/233] Fix Distinct properties --- src/Processors/QueryPlan/DistinctStep.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 323ef0bbdab..f903fc973ad 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -26,13 +26,14 @@ static bool checkColumnsAlreadyDistinct(const Names & columns, const NameSet & d static ITransformingStep::Traits getTraits(bool pre_distinct, bool already_distinct_columns) { + const bool preserves_number_of_streams = pre_distinct || already_distinct_columns; return ITransformingStep::Traits { { .preserves_distinct_columns = already_distinct_columns, /// Will be calculated separately otherwise .returns_single_stream = !pre_distinct && !already_distinct_columns, - .preserves_number_of_streams = pre_distinct || already_distinct_columns, - .preserves_sorting = true, /// Sorting is preserved indeed because of implementation. + .preserves_number_of_streams = preserves_number_of_streams, + .preserves_sorting = preserves_number_of_streams, }, { .preserves_number_of_rows = false, From e1b309a904e4d89c968549b4a8c1716bab53f0df Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 8 Mar 2023 06:01:33 +0000 Subject: [PATCH 025/233] Better docs, add timezone --- .../sql-reference/functions/ulid-functions.md | 33 +++++++++++++ src/Functions/FunctionsCodingULID.cpp | 46 ++++++++++++++----- 2 files changed, 67 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/ulid-functions.md b/docs/en/sql-reference/functions/ulid-functions.md index 94167945f76..2bff5da1c1a 100644 --- a/docs/en/sql-reference/functions/ulid-functions.md +++ b/docs/en/sql-reference/functions/ulid-functions.md @@ -48,6 +48,39 @@ SELECT generateULID(1), generateULID(2) └────────────────────────────┴────────────────────────────┘ ``` +## ULIDStringToDateTime + +This function extracts the timestamp from a ULID. + +**Syntax** + +``` sql +ULIDStringToDateTime(ulid[, timezone]) +``` + +**Arguments** + +- `ulid` — Input UUID. [String](/docs/en/sql-reference/data-types/string.md) or [FixedString(26)](/docs/en/sql-reference/data-types/fixedstring.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- Timestamp with milliseconds precision. + +Type: [DateTime64(3)](/docs/en/sql-reference/data-types/datetime64.md). + +**Usage example** + +``` sql +SELECT ULIDStringToDateTime('01GNB2S2FGN2P93QPXDNB4EN2R') +``` + +``` text +┌─ULIDStringToDateTime('01GNB2S2FGN2P93QPXDNB4EN2R')─┐ +│ 2022-12-28 00:40:37.616 │ +└────────────────────────────────────────────────────┘ +``` + ## See Also - [UUID](../../sql-reference/functions/uuid-functions.md) diff --git a/src/Functions/FunctionsCodingULID.cpp b/src/Functions/FunctionsCodingULID.cpp index cc4210e1480..07be6f57cac 100644 --- a/src/Functions/FunctionsCodingULID.cpp +++ b/src/Functions/FunctionsCodingULID.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -41,14 +42,18 @@ public: String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: 1 expected.", getName()); + if (arguments.size() < 1 || arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Wrong number of arguments for function {}: should be 1 or 2", + getName()); const auto * arg_fixed_string = checkAndGetDataType(arguments[0].type.get()); const auto * arg_string = checkAndGetDataType(arguments[0].type.get()); @@ -60,7 +65,18 @@ public: arguments[0].type->getName(), getName()); - return std::make_shared(DATETIME_SCALE, ""); + String timezone; + if (arguments.size() == 2) + { + timezone = extractTimeZoneNameFromColumn(*arguments[1].column); + + if (timezone.empty()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} supports a 2nd argument (optional) that must be a valid time zone", + getName()); + } + + return std::make_shared(DATETIME_SCALE, timezone); } bool useDefaultImplementationForConstants() const override { return true; } @@ -105,7 +121,7 @@ public: size_t string_size = offsets_src[i] - src_offset; if (string_size == ULID_LENGTH + 1) - time = decode(vec_src.data() + offsets_src[i]); + time = decode(vec_src.data() + src_offset); src_offset += string_size; vec_res[i] = time; @@ -126,15 +142,18 @@ public: unsigned char buffer[16]; int ret = ulid_decode(buffer, reinterpret_cast(data)); if (ret != 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot decode ULID"); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot parse ULID {}", + std::string_view(reinterpret_cast(data), ULID_LENGTH) + ); /// Timestamp in milliseconds is the first 48 bits of the decoded ULID Int64 ms = 0; memcpy(reinterpret_cast(&ms) + 2, buffer, 6); -#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - std::reverse(reinterpret_cast(&ms), reinterpret_cast(&ms) + sizeof(Int64)); -#endif + if constexpr (std::endian::native == std::endian::little) + std::reverse(reinterpret_cast(&ms), reinterpret_cast(&ms) + sizeof(Int64)); return DecimalUtils::decimalFromComponents(ms / intExp10(DATETIME_SCALE), ms % intExp10(DATETIME_SCALE), DATETIME_SCALE); } @@ -146,10 +165,13 @@ REGISTER_FUNCTION(ULIDStringToDateTime) factory.registerFunction( { R"( -Decodes ULID and returns its timestammp as DateTime64(3). -This function takes as an argument ULID of type String or FixedString(26). +This function extracts the timestamp from a ULID and returns it as a DateTime64(3) typed value. +The function expects the ULID to be provided as the first argument, which can be either a String or a FixedString(26) data type. +An optional second argument can be passed to specify a timezone for the timestamp. )", - Documentation::Examples{{"ulid", "SELECT ULIDStringToDateTime(generateULID())"}}, + Documentation::Examples{ + {"ulid", "SELECT ULIDStringToDateTime(generateULID())"}, + {"timezone", "SELECT ULIDStringToDateTime(generateULID(), 'Asia/Istanbul')"}}, Documentation::Categories{"ULID"} }, FunctionFactory::CaseSensitive); From 68858635647c6cb86003db3971bf6e94d417a073 Mon Sep 17 00:00:00 2001 From: Maksym Sobolyev Date: Wed, 8 Mar 2023 16:18:13 -0800 Subject: [PATCH 026/233] Use "SELECT FROM ONLY xyz", not "SELECT FROM xyz" while replicating postgresql tables, to properly handle inherited tables. Currently, it would fetch same data twice - once from the child tables and then from the parent table. --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f9bfe1d174a..29314523860 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -354,7 +354,7 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. auto quoted_name = doubleQuoteWithSchema(table_name); - query_str = fmt::format("SELECT * FROM {}", quoted_name); + query_str = fmt::format("SELECT * FROM ONLY {}", quoted_name); LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); auto table_structure = fetchTableStructure(*tx, table_name); From 5bc21538e516f01c634f51330639d8721b41c9af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 9 Mar 2023 10:31:55 +0000 Subject: [PATCH 027/233] Enable use_environment_credentials by default --- src/Backups/BackupIO_S3.cpp | 2 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/Storages/StorageS3.cpp | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 2f315e8d488..2be5484262a 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -65,7 +65,7 @@ namespace settings.auth_settings.server_side_encryption_customer_key_base64, std::move(headers), settings.auth_settings.use_environment_credentials.value_or( - context->getConfigRef().getBool("s3.use_environment_credentials", false)), + context->getConfigRef().getBool("s3.use_environment_credentials", true)), settings.auth_settings.use_insecure_imds_request.value_or( context->getConfigRef().getBool("s3.use_insecure_imds_request", false))); } diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 7b47324a890..108578f03a7 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -102,7 +102,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo credentials.GetAWSSecretKey(), auth_settings.server_side_encryption_customer_key_base64, std::move(headers), - auth_settings.use_environment_credentials.value_or(false), + auth_settings.use_environment_credentials.value_or(true), auth_settings.use_insecure_imds_request.value_or(false)); auto new_client = std::make_shared(std::move(new_uri), std::move(auth_settings), std::move(client)); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index e0e4735f519..0fd04fdcc69 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -151,7 +151,7 @@ std::unique_ptr getClient( config.getString(config_prefix + ".secret_access_key", ""), config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), {}, - config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)), + config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", true)), config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false))); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ed290c38c1f..e29c5e17cc1 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1265,7 +1265,7 @@ void StorageS3::updateConfiguration(ContextPtr ctx, StorageS3::Configuration & u credentials.GetAWSSecretKey(), upd.auth_settings.server_side_encryption_customer_key_base64, std::move(headers), - upd.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), + upd.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", true)), upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); } @@ -1281,7 +1281,7 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); - configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 0); + configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); configuration.format = collection.getOrDefault("format", "auto"); configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); From aa56dddc56b25c4eac828b45a3769d2054030133 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 13 Mar 2023 08:08:31 +0000 Subject: [PATCH 028/233] Add CNF --- src/Analyzer/Passes/CNF.cpp | 131 ++++++++++++++++++++++++++++++++++++ src/Analyzer/Passes/CNF.h | 46 +++++++++++++ 2 files changed, 177 insertions(+) create mode 100644 src/Analyzer/Passes/CNF.cpp create mode 100644 src/Analyzer/Passes/CNF.h diff --git a/src/Analyzer/Passes/CNF.cpp b/src/Analyzer/Passes/CNF.cpp new file mode 100644 index 00000000000..9d69024f43d --- /dev/null +++ b/src/Analyzer/Passes/CNF.cpp @@ -0,0 +1,131 @@ +#include + +#include +#include + +#include +#include + +#include + +#include + +namespace DB::Analyzer +{ + +namespace +{ + +bool isLogicalFunction(const FunctionNode & function_node) +{ + const std::string_view name = function_node.getFunctionName(); + return name == "and" || name == "or" || name == "not"; +} + +class SplitMultiLogicVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + + explicit SplitMultiLogicVisitor(ContextPtr context) + : Base(std::move(context)) + {} + + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr &) + { + auto * function_node = parent->as(); + + if (!function_node) + return false; + + return isLogicalFunction(*function_node); + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + if (!function_node || !isLogicalFunction(*function_node)) + { + ++atom_count; + return; + } + + const auto & name = function_node->getFunctionName(); + + if (name == "and" || name == "or") + { + auto function_resolver = FunctionFactory::instance().get(name, getContext()); + + const auto & arguments = function_node->getArguments().getNodes(); + if (arguments.size() > 2) + { + QueryTreeNodePtr current = arguments[0]; + for (size_t i = 1; i < arguments.size(); ++i) + { + QueryTreeNodes new_arguments; + new_arguments.reserve(2); + new_arguments.push_back(std::move(current)); + new_arguments.push_back(arguments[i]); + auto new_function_node = std::make_shared(); + new_function_node->getArguments().getNodes() = std::move(new_arguments); + new_function_node->resolveAsFunction(function_resolver); + current = std::move(new_function_node); + } + + auto & new_function_node = current->as(); + function_node->getArguments().getNodes() = std::move(new_function_node.getArguments().getNodes()); + function_node->resolveAsFunction(function_resolver); + } + } + else + { + assert(name == "not"); + } + } + + size_t atom_count = 0; +}; + +} + +bool CNF::AtomicFormula::operator==(const AtomicFormula & rhs) const +{ + return negative == rhs.negative && node_with_hash == rhs.node_with_hash; +} + +std::string CNF::dump() const +{ + WriteBufferFromOwnString res; + bool first = true; + for (const auto & group : statements) + { + if (!first) + res << " AND "; + first = false; + res << "("; + bool first_in_group = true; + for (const auto & atom : group) + { + if (!first_in_group) + res << " OR "; + first_in_group = false; + if (atom.negative) + res << " NOT "; + res << atom.node_with_hash.node->formatASTForErrorMessage(); + } + res << ")"; + } + + return res.str(); +} + +std::optional CNF::tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t) +{ + auto node_cloned = node->clone(); + SplitMultiLogicVisitor split_visitor(std::move(context)); + split_visitor.visit(node_cloned); +// size_t num_atoms = countAtoms(node); + return std::nullopt; +} + +} diff --git a/src/Analyzer/Passes/CNF.h b/src/Analyzer/Passes/CNF.h new file mode 100644 index 00000000000..cdf28eedd9e --- /dev/null +++ b/src/Analyzer/Passes/CNF.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include + +#include + +#include + + +namespace DB::Analyzer +{ + +class CNF +{ +public: + struct AtomicFormula + { + bool negative = false; + QueryTreeNodePtrWithHash node_with_hash; + + bool operator==(const AtomicFormula & rhs) const; + }; + + using OrGroup = std::unordered_set; + using AndGroup = std::unordered_set; + + std::string dump() const; + + static constexpr size_t DEFAULT_MAX_GROWTH_MULTIPLIER = 20; + + static std::optional tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); +private: + AndGroup statements; +}; + +} + +template <> +struct std::hash +{ + size_t operator()(const DB::Analyzer::CNF::AtomicFormula & atomic_formula) const + { + return std::hash()(atomic_formula.node_with_hash); + } +}; From e0e06ceeeff72f8e6b875c6be748e3f3ab21f0b2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 14 Mar 2023 12:58:12 +0000 Subject: [PATCH 029/233] Fix: 02343_group_by_use_nulls --- src/Processors/QueryPlan/RollupStep.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 3305f24602f..baf766e142d 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -53,7 +53,9 @@ void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ void RollupStep::updateOutputStream() { output_stream = createOutputStream( - input_streams.front(), appendGroupingSetColumn(params.getHeader(input_streams.front().header, final)), getDataStreamTraits()); + input_streams.front(), + generateOutputHeader(params.getHeader(input_streams.front().header, final), params.keys, use_nulls), + getDataStreamTraits()); /// Aggregation keys are distinct for (const auto & key : params.keys) From 80d248aa06c36e56fb8b019fa90ea6a4542f62f7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 13 Mar 2023 15:52:35 +0000 Subject: [PATCH 030/233] Finish first part of CNF --- src/Analyzer/Passes/CNF.cpp | 415 ++++++++++++++++-- src/Analyzer/Passes/CNF.h | 49 ++- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 68 +++ src/Analyzer/Passes/ConvertQueryToCNFPass.h | 18 + src/Analyzer/QueryTreePassManager.cpp | 4 +- 5 files changed, 506 insertions(+), 48 deletions(-) create mode 100644 src/Analyzer/Passes/ConvertQueryToCNFPass.cpp create mode 100644 src/Analyzer/Passes/ConvertQueryToCNFPass.h diff --git a/src/Analyzer/Passes/CNF.cpp b/src/Analyzer/Passes/CNF.cpp index 9d69024f43d..23d132540f2 100644 --- a/src/Analyzer/Passes/CNF.cpp +++ b/src/Analyzer/Passes/CNF.cpp @@ -9,6 +9,7 @@ #include #include +#include "Interpreters/ActionsDAG.h" namespace DB::Analyzer { @@ -22,55 +23,60 @@ bool isLogicalFunction(const FunctionNode & function_node) return name == "and" || name == "or" || name == "not"; } -class SplitMultiLogicVisitor : public InDepthQueryTreeVisitorWithContext +template +QueryTreeNodePtr createFunctionNode(const FunctionOverloadResolverPtr & function_resolver, Args &&... args) +{ + auto function_node = std::make_shared(function_resolver->getName()); + auto & new_arguments = function_node->getArguments().getNodes(); + new_arguments.reserve(sizeof...(args)); + (new_arguments.push_back(std::forward(args)), ...); + function_node->resolveAsFunction(function_resolver); + return function_node; +} + +size_t countAtoms(const QueryTreeNodePtr & node) +{ + checkStackSize(); + + const auto * function_node = node->as(); + if (!function_node || !isLogicalFunction(*function_node)) + return 1; + + size_t atom_count = 0; + const auto & arguments = function_node->getArguments().getNodes(); + for (const auto & argument : arguments) + atom_count += countAtoms(argument); + + return atom_count; +} + +class SplitMultiLogicVisitor { public: - using Base = InDepthQueryTreeVisitorWithContext; - explicit SplitMultiLogicVisitor(ContextPtr context) - : Base(std::move(context)) + : current_context(std::move(context)) {} - static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr &) + void visit(QueryTreeNodePtr & node) { - auto * function_node = parent->as(); + checkStackSize(); - if (!function_node) - return false; - - return isLogicalFunction(*function_node); - } - - void visitImpl(QueryTreeNodePtr & node) - { auto * function_node = node->as(); if (!function_node || !isLogicalFunction(*function_node)) - { - ++atom_count; return; - } const auto & name = function_node->getFunctionName(); if (name == "and" || name == "or") { - auto function_resolver = FunctionFactory::instance().get(name, getContext()); + auto function_resolver = FunctionFactory::instance().get(name, current_context); const auto & arguments = function_node->getArguments().getNodes(); if (arguments.size() > 2) { QueryTreeNodePtr current = arguments[0]; for (size_t i = 1; i < arguments.size(); ++i) - { - QueryTreeNodes new_arguments; - new_arguments.reserve(2); - new_arguments.push_back(std::move(current)); - new_arguments.push_back(arguments[i]); - auto new_function_node = std::make_shared(); - new_function_node->getArguments().getNodes() = std::move(new_arguments); - new_function_node->resolveAsFunction(function_resolver); - current = std::move(new_function_node); - } + current = createFunctionNode(function_resolver, std::move(current), arguments[i]); auto & new_function_node = current->as(); function_node->getArguments().getNodes() = std::move(new_function_node.getArguments().getNodes()); @@ -81,9 +87,185 @@ public: { assert(name == "not"); } + + auto & arguments = function_node->getArguments().getNodes(); + for (auto & argument : arguments) + visit(argument); } - size_t atom_count = 0; +private: + ContextPtr current_context; +}; + +class PushNotVisitor +{ +public: + explicit PushNotVisitor(ContextPtr context) + : current_context(std::move(context)) + {} + + void visit(QueryTreeNodePtr & node, bool add_negation) + { + checkStackSize(); + + auto * function_node = node->as(); + + if (!function_node || !isLogicalFunction(*function_node)) + { + if (add_negation) + node = createFunctionNode(FunctionFactory::instance().get("not", current_context), std::move(node)); + return; + } + + std::string_view function_name = function_node->getFunctionName(); + if (function_name == "and" || function_name == "or") + { + if (add_negation) + { + auto function_resolver = FunctionFactory::instance().get(function_name == "and" ? "or" : "and", current_context); + function_node->resolveAsFunction(function_resolver); + } + + auto & arguments = function_node->getArguments().getNodes(); + for (auto & argument : arguments) + visit(argument, add_negation); + return; + } + + assert(function_name == "not"); + auto & arguments = function_node->getArguments().getNodes(); + assert(arguments.size() == 1); + node = arguments[0]; + visit(node, !add_negation); + } + +private: + ContextPtr current_context; +}; + +class PushOrVisitor +{ +public: + PushOrVisitor(ContextPtr context, size_t max_atoms_, size_t num_atoms_) + : max_atoms(max_atoms_) + , num_atoms(num_atoms_) + , and_resolver(FunctionFactory::instance().get("and", context)) + , or_resolver(FunctionFactory::instance().get("or", context)) + {} + + bool visit(QueryTreeNodePtr & node) + { + if (max_atoms && num_atoms > max_atoms) + return false; + + checkStackSize(); + + auto * function_node = node->as(); + + if (!function_node) + return true; + + std::string_view name = function_node->getFunctionName(); + + if (name == "or" || name == "and") + { + auto & arguments = function_node->getArguments().getNodes(); + for (auto & argument : arguments) + visit(argument); + } + + if (name == "or") + { + auto & arguments = function_node->getArguments().getNodes(); + assert(arguments.size() == 2); + + size_t and_node_id = arguments.size(); + + for (size_t i = 0; i < arguments.size(); ++i) + { + auto & argument = arguments[i]; + if (auto * argument_function_node = argument->as(); + argument_function_node && argument_function_node->getFunctionName() == "and") + and_node_id = i; + } + + if (and_node_id == arguments.size()) + return true; + + auto & other_node = arguments[1 - and_node_id]; + auto & and_function_arguments = arguments[and_node_id]->as().getArguments().getNodes(); + + auto lhs = createFunctionNode(or_resolver, other_node->clone(), std::move(and_function_arguments[0])); + num_atoms += countAtoms(other_node); + + auto rhs = createFunctionNode(or_resolver, std::move(other_node), std::move(and_function_arguments[1])); + node = createFunctionNode(and_resolver, std::move(lhs), std::move(rhs)); + + visit(node); + } + + return true; + } + +private: + size_t max_atoms; + size_t num_atoms; + + const FunctionOverloadResolverPtr and_resolver; + const FunctionOverloadResolverPtr or_resolver; +}; + +class CollectGroupsVisitor +{ +public: + void visit(QueryTreeNodePtr & node) + { + CNF::OrGroup or_group; + visitImpl(node, or_group); + if (!or_group.empty()) + and_group.insert(std::move(or_group)); + } + + CNF::AndGroup and_group; + +private: + void visitImpl(QueryTreeNodePtr & node, CNF::OrGroup & or_group) + { + checkStackSize(); + + auto * function_node = node->as(); + if (!function_node || !isLogicalFunction(*function_node)) + { + or_group.insert(CNF::AtomicFormula{false, std::move(node)}); + return; + } + + std::string_view name = function_node->getFunctionName(); + + if (name == "and") + { + auto & arguments = function_node->getArguments().getNodes(); + for (auto & argument : arguments) + { + CNF::OrGroup argument_or_group; + visitImpl(argument, argument_or_group); + if (!argument_or_group.empty()) + and_group.insert(std::move(argument_or_group)); + } + } + else if (name == "or") + { + auto & arguments = function_node->getArguments().getNodes(); + for (auto & argument : arguments) + visitImpl(argument, or_group); + } + else + { + assert(name == "not"); + auto & arguments = function_node->getArguments().getNodes(); + or_group.insert(CNF::AtomicFormula{true, std::move(arguments[0])}); + } + } }; } @@ -93,6 +275,13 @@ bool CNF::AtomicFormula::operator==(const AtomicFormula & rhs) const return negative == rhs.negative && node_with_hash == rhs.node_with_hash; } +bool CNF::AtomicFormula::operator<(const AtomicFormula & rhs) const +{ + return node_with_hash.hash == rhs.node_with_hash.hash + ? negative < rhs.negative + : node_with_hash.hash < rhs.node_with_hash.hash; +} + std::string CNF::dump() const { WriteBufferFromOwnString res; @@ -119,13 +308,171 @@ std::string CNF::dump() const return res.str(); } -std::optional CNF::tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t) +CNF & CNF::transformGroups(std::function fn) { + AndGroup result; + + for (const auto & group : statements) + { + auto new_group = fn(group); + if (!new_group.empty()) + result.insert(std::move(new_group)); + } + + statements = std::move(result); + return *this; +} + +CNF & CNF::transformAtoms(std::function fn) +{ + transformGroups([fn](const OrGroup & group) + { + OrGroup result; + for (const auto & atom : group) + { + auto new_atom = fn(atom); + if (new_atom.node_with_hash.node) + result.insert(std::move(new_atom)); + } + + return result; + }); + + return *this; +} + +CNF & CNF::pushNotIntoFunctions(const ContextPtr & context) +{ + transformAtoms([&](const AtomicFormula & atom) + { + if (!atom.negative) + return atom; + + static const std::unordered_map inverse_relations = { + {"equals", "notEquals"}, + {"less", "greaterOrEquals"}, + {"lessOrEquals", "greater"}, + {"in", "notIn"}, + {"like", "notLike"}, + {"empty", "notEmpty"}, + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + {"notIn", "in"}, + {"notLike", "like"}, + {"notEmpty", "empty"}, + }; + + auto * function_node = atom.node_with_hash.node->as(); + if (!function_node) + return atom; + + if (auto it = inverse_relations.find(function_node->getFunctionName()); it != inverse_relations.end()) + { + auto inverse_function_resolver = FunctionFactory::instance().get(it->second, context); + function_node->resolveAsFunction(inverse_function_resolver); + return AtomicFormula{!atom.negative, atom.node_with_hash.node}; + } + + return atom; + }); + + std::cout << "gorup: " << statements.size() << std::endl; + + return *this; +} + +CNF::CNF(AndGroup statements_) + : statements(std::move(statements_)) +{} + +std::optional CNF::tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier) +{ + auto * function_node = node->as(); + + if (!function_node || !isLogicalFunction(*function_node)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert nodes that are not logical functions to CNF"); + auto node_cloned = node->clone(); - SplitMultiLogicVisitor split_visitor(std::move(context)); - split_visitor.visit(node_cloned); -// size_t num_atoms = countAtoms(node); - return std::nullopt; + + size_t atom_count = countAtoms(node_cloned); + size_t max_atoms = max_growth_multiplier ? std::max(MAX_ATOMS_WITHOUT_CHECK, atom_count * max_growth_multiplier) : 0; + + { + SplitMultiLogicVisitor visitor(context); + visitor.visit(node_cloned); + } + + { + PushNotVisitor visitor(context); + visitor.visit(node_cloned, false); + } + + if (PushOrVisitor visitor(context, max_atoms, atom_count); + !visitor.visit(node_cloned)) + return std::nullopt; + + CollectGroupsVisitor collect_visitor; + collect_visitor.visit(node_cloned); + + if (collect_visitor.and_group.empty()) + return std::nullopt; + + return CNF{std::move(collect_visitor.and_group)}; +} + +QueryTreeNodePtr CNF::toQueryTree(ContextPtr context) const +{ + if (statements.empty()) + return nullptr; + + QueryTreeNodes and_arguments; + and_arguments.reserve(statements.size()); + + auto not_resolver = FunctionFactory::instance().get("not", context); + auto or_resolver = FunctionFactory::instance().get("or", context); + auto and_resolver = FunctionFactory::instance().get("and", context); + + const auto function_node_from_atom = [&](const auto & atom) -> QueryTreeNodePtr + { + auto cloned_node = atom.node_with_hash.node->clone(); + if (atom.negative) + return createFunctionNode(not_resolver, std::move(cloned_node)); + + return std::move(cloned_node); + }; + + for (const auto & or_group : statements) + { + if (or_group.size() == 1) + { + const auto & atom = *or_group.begin(); + and_arguments.push_back(function_node_from_atom(atom)); + } + else + { + QueryTreeNodes or_arguments; + or_arguments.reserve(or_group.size()); + + for (const auto & atom : or_group) + or_arguments.push_back(function_node_from_atom(atom)); + + auto or_function = std::make_shared("or"); + or_function->getArguments().getNodes() = std::move(or_arguments); + or_function->resolveAsFunction(or_resolver); + + and_arguments.push_back(std::move(or_function)); + } + } + + if (and_arguments.size() == 1) + return std::move(and_arguments[0]); + + auto and_function = std::make_shared("and"); + and_function->getArguments().getNodes() = std::move(and_arguments); + and_function->resolveAsFunction(and_resolver); + + return and_function; } } diff --git a/src/Analyzer/Passes/CNF.h b/src/Analyzer/Passes/CNF.h index cdf28eedd9e..9736a356d63 100644 --- a/src/Analyzer/Passes/CNF.h +++ b/src/Analyzer/Passes/CNF.h @@ -3,11 +3,12 @@ #include #include +#include + #include #include - namespace DB::Analyzer { @@ -20,27 +21,49 @@ public: QueryTreeNodePtrWithHash node_with_hash; bool operator==(const AtomicFormula & rhs) const; + bool operator<(const AtomicFormula & rhs) const; }; - using OrGroup = std::unordered_set; - using AndGroup = std::unordered_set; + struct SetAtomicFormulaHash + { + size_t operator()(const std::set & or_group) const + { + SipHash hash; + for (const auto & atomic_formula : or_group) + { + SipHash atomic_formula_hash; + atomic_formula_hash.update(atomic_formula.negative); + atomic_formula_hash.update(atomic_formula.node_with_hash.hash); + + hash.update(atomic_formula_hash.get64()); + } + + return hash.get64(); + } + }; + + // Different hash is generated for different order, so we use std::set + using OrGroup = std::set; + using AndGroup = std::unordered_set; std::string dump() const; static constexpr size_t DEFAULT_MAX_GROWTH_MULTIPLIER = 20; + static constexpr size_t MAX_ATOMS_WITHOUT_CHECK = 200; + + CNF & transformAtoms(std::function fn); + + /// Convert "NOT fn" to a single node representing inverse of "fn" + CNF & pushNotIntoFunctions(const ContextPtr & context); static std::optional tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); + + QueryTreeNodePtr toQueryTree(ContextPtr context) const; private: + explicit CNF(AndGroup statements_); + + CNF & transformGroups(std::function fn); AndGroup statements; }; - -} -template <> -struct std::hash -{ - size_t operator()(const DB::Analyzer::CNF::AtomicFormula & atomic_formula) const - { - return std::hash()(atomic_formula.node_with_hash); - } -}; +} diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp new file mode 100644 index 00000000000..1c7e9a5753d --- /dev/null +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -0,0 +1,68 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ + +bool isLogicalFunction(const FunctionNode & function_node) +{ + const std::string_view name = function_node.getFunctionName(); + return name == "and" || name == "or" || name == "not"; +} + +class ConvertQueryToCNFVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &) + { + if (!getSettings().convert_query_to_cnf) + return false; + + auto * function_node = parent->as(); + return !function_node || !isLogicalFunction(*function_node); + } + + void visitImpl(QueryTreeNodePtr & node) + { + if (!getSettings().convert_query_to_cnf) + return; + + auto * function_node = node->as(); + + if (!function_node || !isLogicalFunction(*function_node)) + return; + + const auto & context = getContext(); + auto cnf_form = Analyzer::CNF::tryBuildCNF(node, context); + if (!cnf_form) + return; + + cnf_form->pushNotIntoFunctions(context); + std::cout << "CNF " << cnf_form->dump() << std::endl; + auto new_node = cnf_form->toQueryTree(context); + if (!new_node) + return; + + node = std::move(new_node); + } +}; + +} + +void ConvertQueryToCnfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + ConvertQueryToCNFVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} + + +} diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.h b/src/Analyzer/Passes/ConvertQueryToCNFPass.h new file mode 100644 index 00000000000..fd2544ec228 --- /dev/null +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +namespace DB +{ + +class ConvertQueryToCnfPass final : public IQueryTreePass +{ +public: + String getName() override { return "ConvertQueryToCnfPass"; } + + String getDescription() override { return "Convery query to CNF"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 9c0f2381c31..a6ac79c8702 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -41,7 +41,7 @@ #include #include #include - +#include namespace DB { @@ -235,6 +235,8 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); From 0ecce85c4879ee15cbd5362f1ec03844b35865ca Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 14 Mar 2023 16:37:14 +0000 Subject: [PATCH 031/233] Crutch for 02404_memory_bound_merging --- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 5 +++++ src/Processors/QueryPlan/MergingAggregatedStep.h | 3 +++ 2 files changed, 8 insertions(+) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index e4fc332a1fd..369254cc1b2 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -75,6 +75,9 @@ MergingAggregatedStep::MergingAggregatedStep( void MergingAggregatedStep::applyOrder(SortDescription sort_description, DataStream::SortScope sort_scope) { + is_order_overwritten = true; + overwritten_sort_scope = sort_scope; + auto & input_stream = input_streams.front(); input_stream.sort_scope = sort_scope; input_stream.sort_description = sort_description; @@ -157,6 +160,8 @@ void MergingAggregatedStep::describeActions(JSONBuilder::JSONMap & map) const void MergingAggregatedStep::updateOutputStream() { output_stream = createOutputStream(input_streams.front(), params.getHeader(input_streams.front().header, final), getDataStreamTraits()); + if (is_order_overwritten) /// overwrite order again + applyOrder(group_by_sort_description, overwritten_sort_scope); /// Aggregation keys are distinct for (const auto & key : params.keys) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 2dea289ca89..e798dabc3ba 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -50,6 +50,9 @@ private: const size_t memory_bound_merging_max_block_bytes; SortDescription group_by_sort_description; + bool is_order_overwritten = false; + DataStream::SortScope overwritten_sort_scope = DataStream::SortScope::None; + /// These settings are used to determine if we should resize pipeline to 1 at the end. const bool should_produce_results_in_order_of_bucket_number; const bool memory_bound_merging_of_aggregation_results_enabled; From 0f6e4d3934cda52145c7b7754c4e985b4e6b4947 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 15 Mar 2023 11:41:29 +0000 Subject: [PATCH 032/233] Start implementing constraint optimizer --- src/Analyzer/Passes/CNF.cpp | 190 +++++-- src/Analyzer/Passes/CNF.h | 26 +- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 440 ++++++++++++++- src/Analyzer/Passes/ConvertQueryToCNFPass.h | 8 +- src/Analyzer/QueryTreePassManager.cpp | 2 +- .../AddIndexConstraintsOptimizer.cpp | 42 +- src/Interpreters/ComparisonGraph.cpp | 529 +++++++++++------- src/Interpreters/ComparisonGraph.h | 97 ++-- .../SubstituteColumnOptimizer.cpp | 6 +- src/Interpreters/TreeCNFConverter.cpp | 70 +-- src/Interpreters/TreeCNFConverter.h | 68 +++ .../WhereConstraintsOptimizer.cpp | 12 +- src/Storages/ConstraintsDescription.cpp | 98 +++- src/Storages/ConstraintsDescription.h | 31 +- ...ergeTreeIndexHypothesisMergedCondition.cpp | 12 +- .../MergeTreeIndexHypothesisMergedCondition.h | 6 +- 16 files changed, 1227 insertions(+), 410 deletions(-) diff --git a/src/Analyzer/Passes/CNF.cpp b/src/Analyzer/Passes/CNF.cpp index 23d132540f2..3d95200a948 100644 --- a/src/Analyzer/Passes/CNF.cpp +++ b/src/Analyzer/Passes/CNF.cpp @@ -2,6 +2,9 @@ #include #include +#include + +#include #include #include @@ -9,9 +12,17 @@ #include #include -#include "Interpreters/ActionsDAG.h" -namespace DB::Analyzer +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_MANY_TEMPORARY_COLUMNS; + extern const int LOGICAL_ERROR; +} + +namespace Analyzer { namespace @@ -268,6 +279,22 @@ private: } }; +std::optional tryInvertFunction( + const CNF::AtomicFormula & atom, const ContextPtr & context, const std::unordered_map & inverse_relations) +{ + auto * function_node = atom.node_with_hash.node->as(); + if (!function_node) + return std::nullopt; + + if (auto it = inverse_relations.find(function_node->getFunctionName()); it != inverse_relations.end()) + { + auto inverse_function_resolver = FunctionFactory::instance().get(it->second, context); + function_node->resolveAsFunction(inverse_function_resolver); + return CNF::AtomicFormula{!atom.negative, atom.node_with_hash.node}; + } + + return std::nullopt; +} } bool CNF::AtomicFormula::operator==(const AtomicFormula & rhs) const @@ -345,43 +372,123 @@ CNF & CNF::pushNotIntoFunctions(const ContextPtr & context) { transformAtoms([&](const AtomicFormula & atom) { - if (!atom.negative) - return atom; - - static const std::unordered_map inverse_relations = { - {"equals", "notEquals"}, - {"less", "greaterOrEquals"}, - {"lessOrEquals", "greater"}, - {"in", "notIn"}, - {"like", "notLike"}, - {"empty", "notEmpty"}, - {"notEquals", "equals"}, - {"greaterOrEquals", "less"}, - {"greater", "lessOrEquals"}, - {"notIn", "in"}, - {"notLike", "like"}, - {"notEmpty", "empty"}, - }; - - auto * function_node = atom.node_with_hash.node->as(); - if (!function_node) - return atom; - - if (auto it = inverse_relations.find(function_node->getFunctionName()); it != inverse_relations.end()) - { - auto inverse_function_resolver = FunctionFactory::instance().get(it->second, context); - function_node->resolveAsFunction(inverse_function_resolver); - return AtomicFormula{!atom.negative, atom.node_with_hash.node}; - } - - return atom; + return pushNotIntoFunction(atom, context); }); - std::cout << "gorup: " << statements.size() << std::endl; + return *this; +} + +CNF::AtomicFormula CNF::pushNotIntoFunction(const AtomicFormula & atom, const ContextPtr & context) +{ + if (!atom.negative) + return atom; + + static const std::unordered_map inverse_relations = { + {"equals", "notEquals"}, + {"less", "greaterOrEquals"}, + {"lessOrEquals", "greater"}, + {"in", "notIn"}, + {"like", "notLike"}, + {"empty", "notEmpty"}, + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + {"notIn", "in"}, + {"notLike", "like"}, + {"notEmpty", "empty"}, + }; + + if (auto inverted_atom = tryInvertFunction(atom, context, inverse_relations); + inverted_atom.has_value()) + return std::move(*inverted_atom); + + return atom; +} + +CNF & CNF::pullNotOutFunctions(const ContextPtr & context) +{ + transformAtoms([&](const AtomicFormula & atom) + { + static const std::unordered_map inverse_relations = { + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + {"notIn", "in"}, + {"notLike", "like"}, + {"notEmpty", "empty"}, + }; + + if (auto inverted_atom = tryInvertFunction(atom, context, inverse_relations); + inverted_atom.has_value()) + return std::move(*inverted_atom); + + return atom; + }); return *this; } +CNF & CNF::filterAlwaysTrueGroups(std::function predicate) +{ + AndGroup filtered; + for (const auto & or_group : statements) + { + if (predicate(or_group)) + filtered.insert(or_group); + } + + statements = std::move(filtered); + return *this; +} + +CNF & CNF::filterAlwaysFalseAtoms(std::function predicate) +{ + AndGroup filtered; + for (const auto & or_group : statements) + { + OrGroup filtered_group; + for (const auto & atom : or_group) + { + if (predicate(atom)) + filtered_group.insert(atom); + } + + if (!filtered_group.empty()) + filtered.insert(std::move(filtered_group)); + else + { + filtered.clear(); + filtered_group.insert(AtomicFormula{false, QueryTreeNodePtrWithHash{std::make_shared(static_cast(0))}}); + filtered.insert(std::move(filtered_group)); + break; + } + } + + statements = std::move(filtered); + return *this; +} + +CNF & CNF::reduce() +{ + while (true) + { + AndGroup new_statements = reduceOnceCNFStatements(statements); + if (statements == new_statements) + { + statements = filterCNFSubsets(statements); + return *this; + } + else + statements = new_statements; + } +} + +void CNF::appendGroup(const AndGroup & and_group) +{ + for (const auto & or_group : and_group) + statements.emplace(or_group); +} + CNF::CNF(AndGroup statements_) : statements(std::move(statements_)) {} @@ -421,10 +528,21 @@ std::optional CNF::tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr co return CNF{std::move(collect_visitor.and_group)}; } +CNF CNF::toCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier) +{ + auto cnf = tryBuildCNF(node, context, max_growth_multiplier); + if (!cnf) + throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, + "Cannot convert expression '{}' to CNF, because it produces to many clauses." + "Size of boolean formula in CNF can be exponential of size of source formula."); + + return *cnf; +} + QueryTreeNodePtr CNF::toQueryTree(ContextPtr context) const { - if (statements.empty()) - return nullptr; + if (statements.empty()) + return nullptr; QueryTreeNodes and_arguments; and_arguments.reserve(statements.size()); @@ -476,3 +594,5 @@ QueryTreeNodePtr CNF::toQueryTree(ContextPtr context) const } } + +} diff --git a/src/Analyzer/Passes/CNF.h b/src/Analyzer/Passes/CNF.h index 9736a356d63..238e32c4ce1 100644 --- a/src/Analyzer/Passes/CNF.h +++ b/src/Analyzer/Passes/CNF.h @@ -37,7 +37,7 @@ public: hash.update(atomic_formula_hash.get64()); } - + return hash.get64(); } }; @@ -52,17 +52,33 @@ public: static constexpr size_t MAX_ATOMS_WITHOUT_CHECK = 200; CNF & transformAtoms(std::function fn); + CNF & transformGroups(std::function fn); + + CNF & filterAlwaysTrueGroups(std::function predicate); + CNF & filterAlwaysFalseAtoms(std::function predicate); + + CNF & reduce(); + + void appendGroup(const AndGroup & and_group); /// Convert "NOT fn" to a single node representing inverse of "fn" CNF & pushNotIntoFunctions(const ContextPtr & context); + CNF & pullNotOutFunctions(const ContextPtr & context); - static std::optional tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); + static AtomicFormula pushNotIntoFunction(const AtomicFormula & atom, const ContextPtr & context); - QueryTreeNodePtr toQueryTree(ContextPtr context) const; -private: explicit CNF(AndGroup statements_); - CNF & transformGroups(std::function fn); + static std::optional tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); + static CNF toCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); + + QueryTreeNodePtr toQueryTree(ContextPtr context) const; + + const auto & getStatements() const + { + return statements; + } +private: AndGroup statements; }; diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 1c7e9a5753d..278cadf2638 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -2,7 +2,14 @@ #include #include +#include +#include +#include #include +#include + +#include +#include "Analyzer/HashUtils.h" namespace DB { @@ -16,53 +23,438 @@ bool isLogicalFunction(const FunctionNode & function_node) return name == "and" || name == "or" || name == "not"; } +std::optional tryConvertQueryToCNF(const QueryTreeNodePtr & node, const ContextPtr & context) +{ + auto * function_node = node->as(); + + if (!function_node || !isLogicalFunction(*function_node)) + return std::nullopt; + + auto cnf_form = Analyzer::CNF::tryBuildCNF(node, context); + if (!cnf_form) + return std::nullopt; + + cnf_form->pushNotIntoFunctions(context); + return cnf_form; +} + +enum class MatchState : uint8_t +{ + FULL_MATCH, /// a = b + PARTIAL_MATCH, /// a = not b + NONE, +}; + +MatchState match(const Analyzer::CNF::AtomicFormula & a, const Analyzer::CNF::AtomicFormula & b) +{ + using enum MatchState; + if (a.node_with_hash.hash != b.node_with_hash.hash) + return NONE; + + return a.negative == b.negative ? FULL_MATCH : PARTIAL_MATCH; +} + +bool checkIfGroupAlwaysTrueFullMatch(const Analyzer::CNF::OrGroup & group, const ConstraintsDescription & constraints_description, const ContextPtr & context) +{ + /// We have constraints in CNF. + /// CNF is always true => Each OR group in CNF is always true. + /// So, we try to check whether we have al least one OR group from CNF as subset in our group. + /// If we've found one then our group is always true too. + + const auto & query_tree_constraint = constraints_description.getQueryTreeData(context); + const auto & constraints_data = query_tree_constraint.getConstraintData(); + std::vector found(constraints_data.size()); + for (size_t i = 0; i < constraints_data.size(); ++i) + found[i] = constraints_data[i].size(); + + for (const auto & atom : group) + { + const auto constraint_atom_ids = query_tree_constraint.getAtomIds(atom.node_with_hash); + if (constraint_atom_ids) + { + const auto constraint_atoms = query_tree_constraint.getAtomsById(*constraint_atom_ids); + for (size_t i = 0; i < constraint_atoms.size(); ++i) + { + if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH) + { + if ((--found[(*constraint_atom_ids)[i].group_id]) == 0) + return true; + } + } + } + } + return false; +} + +bool checkIfGroupAlwaysTrueGraph(const Analyzer::CNF::OrGroup & group, const ComparisonGraph & graph) +{ + /// We try to find at least one atom that is always true by using comparison graph. + for (const auto & atom : group) + { + const auto * function_node = atom.node_with_hash.node->as(); + if (function_node) + { + const auto & arguments = function_node->getArguments().getNodes(); + if (arguments.size() == 2) + { + const auto expected = ComparisonGraph::atomToCompareResult(atom); + if (graph.isAlwaysCompare(expected, arguments[0], arguments[1])) + return true; + } + } + } + + return false; +} + +bool checkIfAtomAlwaysFalseFullMatch(const Analyzer::CNF::AtomicFormula & atom, const ConstraintsDescription & constraints_description, const ContextPtr & context) +{ + const auto & query_tree_constraint = constraints_description.getQueryTreeData(context); + const auto constraint_atom_ids = query_tree_constraint.getAtomIds(atom.node_with_hash); + if (constraint_atom_ids) + { + for (const auto & constraint_atom : query_tree_constraint.getAtomsById(*constraint_atom_ids)) + { + const auto match_result = match(constraint_atom, atom); + if (match_result == MatchState::PARTIAL_MATCH) + return true; + } + } + + return false; +} + +bool checkIfAtomAlwaysFalseGraph(const Analyzer::CNF::AtomicFormula & atom, const ComparisonGraph & graph) +{ + const auto * function_node = atom.node_with_hash.node->as(); + if (!function_node) + return false; + + const auto & arguments = function_node->getArguments().getNodes(); + if (arguments.size() != 2) + return false; + + /// TODO: special support for != + const auto expected = ComparisonGraph::atomToCompareResult(atom); + return !graph.isPossibleCompare(expected, arguments[0], arguments[1]); +} + +void replaceToConstants(QueryTreeNodePtr & term, const ComparisonGraph & graph) +{ + const auto equal_constant = graph.getEqualConst(term); + if (equal_constant) + { + term = (*equal_constant)->clone(); + return; + } + + for (auto & child : term->getChildren()) + replaceToConstants(child, graph); +} + +Analyzer::CNF::AtomicFormula replaceTermsToConstants(const Analyzer::CNF::AtomicFormula & atom, const ComparisonGraph & graph) +{ + auto node = atom.node_with_hash.node->clone(); + replaceToConstants(node, graph); + return {atom.negative, std::move(node)}; +} + +StorageMetadataPtr getStorageMetadata(const QueryTreeNodePtr & node) +{ + StorageSnapshotPtr storage_snapshot{nullptr}; + if (auto * table_node = node->as()) + storage_snapshot = table_node->getStorageSnapshot(); + else if (auto * table_function_node = node->as()) + storage_snapshot = table_function_node->getStorageSnapshot(); + + if (!storage_snapshot) + return nullptr; + + return storage_snapshot->metadata; +} + +bool onlyIndexColumns(const QueryTreeNodePtr & node, const std::unordered_set & primary_key_set) +{ + const auto * identifier_node = node->as(); + /// TODO: verify that full name is correct here + if (identifier_node && !primary_key_set.contains(identifier_node->getIdentifier().getFullName())) + return false; + + for (const auto & child : node->getChildren()) + { + if (!onlyIndexColumns(child, primary_key_set)) + return false; + } + + return true; +} + +bool onlyConstants(const QueryTreeNodePtr & node) +{ + if (node->as() != nullptr) + return false; + + for (const auto & child : node->getChildren()) + { + if (!onlyConstants(child)) + return false; + } + + return true; +} + +const std::unordered_map & getRelationMap() +{ + using enum ComparisonGraphCompareResult; + static const std::unordered_map relations = + { + {"equals", EQUAL}, + {"less", LESS}, + {"lessOrEquals", LESS_OR_EQUAL}, + {"greaterOrEquals", GREATER_OR_EQUAL}, + {"greater", GREATER}, + }; + return relations; +} + +const std::unordered_map & getReverseRelationMap() +{ + using enum ComparisonGraphCompareResult; + static const std::unordered_map relations = + { + {EQUAL, "equals"}, + {LESS, "less"}, + {LESS_OR_EQUAL, "lessOrEquals"}, + {GREATER_OR_EQUAL, "greaterOrEquals"}, + {GREATER, "greater"}, + }; + return relations; +} + +bool canBeSequence(const ComparisonGraphCompareResult left, const ComparisonGraphCompareResult right) +{ + using enum ComparisonGraphCompareResult; + if (left == UNKNOWN || right == UNKNOWN || left == NOT_EQUAL || right == NOT_EQUAL) + return false; + if ((left == GREATER || left == GREATER_OR_EQUAL) && (right == LESS || right == LESS_OR_EQUAL)) + return false; + if ((right == GREATER || right == GREATER_OR_EQUAL) && (left == LESS || left == LESS_OR_EQUAL)) + return false; + return true; +} + +ComparisonGraphCompareResult mostStrict(const ComparisonGraphCompareResult left, const ComparisonGraphCompareResult right) +{ + using enum ComparisonGraphCompareResult; + if (left == LESS || left == GREATER) + return left; + if (right == LESS || right == GREATER) + return right; + if (left == LESS_OR_EQUAL || left == GREATER_OR_EQUAL) + return left; + if (right == LESS_OR_EQUAL || right == GREATER_OR_EQUAL) + return right; + if (left == EQUAL) + return left; + if (right == EQUAL) + return right; + return UNKNOWN; +} + +/// Create OR-group for 'indexHint'. +/// Consider we have expression like A C, where C is constant. +/// Consider we have a constraint I A, where I depends only on columns from primary key. +/// Then if op1 and op2 forms a sequence of comparisons (e.g. A < C and I < A), +/// we can add to expression 'indexHint(I < A)' condition. +Analyzer::CNF::OrGroup createIndexHintGroup( + const Analyzer::CNF::OrGroup & group, + const ComparisonGraph & graph, + const QueryTreeNodes & primary_key_only_nodes, + const ContextPtr & context) +{ + Analyzer::CNF::OrGroup result; + for (const auto & atom : group) + { + const auto * function_node = atom.node_with_hash.node->as(); + if (!function_node || getRelationMap().contains(function_node->getFunctionName())) + continue; + + const auto & arguments = function_node->getArguments().getNodes(); + if (arguments.size() != 2) + continue; + + auto check_and_insert = [&](const size_t index, const ComparisonGraphCompareResult expected_result) + { + if (!onlyConstants(arguments[1 - index])) + return false; + + for (const auto & primary_key_node : primary_key_only_nodes) + { + ComparisonGraphCompareResult actual_result; + if (index == 0) + actual_result = graph.compare(primary_key_node, arguments[index]); + else + actual_result = graph.compare(arguments[index], primary_key_node); + + if (canBeSequence(expected_result, actual_result)) + { + auto helper_node = function_node->clone(); + auto & helper_function_node = helper_node->as(); + auto reverse_function_name = getReverseRelationMap().at(mostStrict(expected_result, actual_result)); + helper_function_node.resolveAsFunction(FunctionFactory::instance().get(reverse_function_name, context)); + result.insert(Analyzer::CNF::AtomicFormula{atom.negative, std::move(helper_node)}); + return true; + } + } + + return false; + }; + + auto expected = getRelationMap().at(function_node->getFunctionName()); + if (!check_and_insert(0, expected) && !check_and_insert(1, expected)) + return {}; + } + + return result; +} + +void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expressions, const ContextPtr & context) +{ + for (const auto & table_expression : table_expressions) + { + auto metadata = getStorageMetadata(table_expression); + if (!metadata) + continue; + + const auto primary_key = metadata->getColumnsRequiredForPrimaryKey(); + const std::unordered_set primary_key_set(primary_key.begin(), primary_key.end()); + const auto & query_tree_constraint = metadata->getConstraints().getQueryTreeData(context); + const auto & graph = query_tree_constraint.getGraph(); + + QueryTreeNodes primary_key_only_nodes; + for (const auto & vertex : graph.getVertices()) + { + for (const auto & node : vertex) + { + if (onlyIndexColumns(node, primary_key_set)) + primary_key_only_nodes.push_back(node); + } + } + + Analyzer::CNF::AndGroup and_group; + const auto & statements = cnf.getStatements(); + for (const auto & group : statements) + { + auto new_group = createIndexHintGroup(group, graph, primary_key_only_nodes, context); + if (!new_group.empty()) + and_group.emplace(std::move(new_group)); + } + + if (!and_group.empty()) + { + Analyzer::CNF::OrGroup new_group; + auto index_hint_node = std::make_shared("indexHint"); + index_hint_node->getArguments().getNodes().push_back(Analyzer::CNF{std::move(and_group)}.toQueryTree(context)); + index_hint_node->resolveAsFunction(FunctionFactory::instance().get("indexHint", context)); + new_group.insert({false, QueryTreeNodePtrWithHash{std::move(index_hint_node)}}); + + cnf.appendGroup({new_group}); + } + } +} + +void optimizeWithConstraints(Analyzer::CNF & cnf, const QueryTreeNodes & table_expressions, const ContextPtr & context) +{ + cnf.pullNotOutFunctions(context); + + for (const auto & table_expression : table_expressions) + { + auto metadata = getStorageMetadata(table_expression); + if (!metadata) + continue; + + const auto & constraints = metadata->getConstraints(); + const auto & query_tree_constraint = constraints.getQueryTreeData(context); + const auto & compare_graph = query_tree_constraint.getGraph(); + cnf.filterAlwaysTrueGroups([&](const auto & group) + { + /// remove always true groups from CNF + return !checkIfGroupAlwaysTrueFullMatch(group, constraints, context) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); + }) + .filterAlwaysFalseAtoms([&](const Analyzer::CNF::AtomicFormula & atom) + { + /// remove always false atoms from CNF + return !checkIfAtomAlwaysFalseFullMatch(atom, constraints, context) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); + }) + .transformAtoms([&](const auto & atom) + { + return replaceTermsToConstants(atom, compare_graph); + }) + .reduce(); + } + + cnf.pushNotIntoFunctions(context); + + const auto & settings = context->getSettingsRef(); + if (settings.optimize_append_index) + addIndexConstraint(cnf, table_expressions, context); +} + +void optimizeNode(QueryTreeNodePtr & node, const QueryTreeNodes & table_expressions, const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + + auto cnf = tryConvertQueryToCNF(node, context); + if (!cnf) + return; + + if (settings.optimize_using_constraints) + optimizeWithConstraints(*cnf, table_expressions, context); + + auto new_node = cnf->toQueryTree(context); + if (!new_node) + return; + + node = std::move(new_node); +} + class ConvertQueryToCNFVisitor : public InDepthQueryTreeVisitorWithContext { public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &) + static bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &) { - if (!getSettings().convert_query_to_cnf) - return false; - - auto * function_node = parent->as(); - return !function_node || !isLogicalFunction(*function_node); + return parent->as() == nullptr; } void visitImpl(QueryTreeNodePtr & node) { - if (!getSettings().convert_query_to_cnf) + auto * query_node = node->as(); + if (!query_node) return; - auto * function_node = node->as(); + auto table_expressions = extractTableExpressions(query_node->getJoinTree()); - if (!function_node || !isLogicalFunction(*function_node)) - return; + if (query_node->hasWhere()) + optimizeNode(query_node->getWhere(), table_expressions, getContext()); - const auto & context = getContext(); - auto cnf_form = Analyzer::CNF::tryBuildCNF(node, context); - if (!cnf_form) - return; - - cnf_form->pushNotIntoFunctions(context); - std::cout << "CNF " << cnf_form->dump() << std::endl; - auto new_node = cnf_form->toQueryTree(context); - if (!new_node) - return; - - node = std::move(new_node); + if (query_node->hasPrewhere()) + optimizeNode(query_node->getPrewhere(), table_expressions, getContext()); } }; } -void ConvertQueryToCnfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +void ConvertQueryToCNFPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { + const auto & settings = context->getSettingsRef(); + if (!settings.convert_query_to_cnf) + return; + ConvertQueryToCNFVisitor visitor(std::move(context)); visitor.visit(query_tree_node); } - } diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.h b/src/Analyzer/Passes/ConvertQueryToCNFPass.h index fd2544ec228..71fb28bdf85 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.h +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.h @@ -4,14 +4,14 @@ namespace DB { - -class ConvertQueryToCnfPass final : public IQueryTreePass + +class ConvertQueryToCNFPass final : public IQueryTreePass { public: String getName() override { return "ConvertQueryToCnfPass"; } - String getDescription() override { return "Convery query to CNF"; } - + String getDescription() override { return "Convert query to CNF"; } + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a6ac79c8702..bd9a2d4618c 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -235,7 +235,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp index b28e31d22f6..15adc737f6f 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.cpp +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -41,35 +41,35 @@ namespace return true; } - const std::unordered_map & getRelationMap() + const std::unordered_map & getRelationMap() { - const static std::unordered_map relations = + const static std::unordered_map relations = { - {"equals", ComparisonGraph::CompareResult::EQUAL}, - {"less", ComparisonGraph::CompareResult::LESS}, - {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, - {"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL}, - {"greater", ComparisonGraph::CompareResult::GREATER}, + {"equals", ComparisonGraphCompareResult::EQUAL}, + {"less", ComparisonGraphCompareResult::LESS}, + {"lessOrEquals", ComparisonGraphCompareResult::LESS_OR_EQUAL}, + {"greaterOrEquals", ComparisonGraphCompareResult::GREATER_OR_EQUAL}, + {"greater", ComparisonGraphCompareResult::GREATER}, }; return relations; } - const std::unordered_map & getReverseRelationMap() + const std::unordered_map & getReverseRelationMap() { - const static std::unordered_map relations = + const static std::unordered_map relations = { - {ComparisonGraph::CompareResult::EQUAL, "equals"}, - {ComparisonGraph::CompareResult::LESS, "less"}, - {ComparisonGraph::CompareResult::LESS_OR_EQUAL, "lessOrEquals"}, - {ComparisonGraph::CompareResult::GREATER_OR_EQUAL, "greaterOrEquals"}, - {ComparisonGraph::CompareResult::GREATER, "greater"}, + {ComparisonGraphCompareResult::EQUAL, "equals"}, + {ComparisonGraphCompareResult::LESS, "less"}, + {ComparisonGraphCompareResult::LESS_OR_EQUAL, "lessOrEquals"}, + {ComparisonGraphCompareResult::GREATER_OR_EQUAL, "greaterOrEquals"}, + {ComparisonGraphCompareResult::GREATER, "greater"}, }; return relations; } - bool canBeSequence(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right) + bool canBeSequence(const ComparisonGraphCompareResult left, const ComparisonGraphCompareResult right) { - using CR = ComparisonGraph::CompareResult; + using CR = ComparisonGraphCompareResult; if (left == CR::UNKNOWN || right == CR::UNKNOWN || left == CR::NOT_EQUAL || right == CR::NOT_EQUAL) return false; if ((left == CR::GREATER || left == CR::GREATER_OR_EQUAL) && (right == CR::LESS || right == CR::LESS_OR_EQUAL)) @@ -79,9 +79,9 @@ namespace return true; } - ComparisonGraph::CompareResult mostStrict(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right) + ComparisonGraphCompareResult mostStrict(const ComparisonGraphCompareResult left, const ComparisonGraphCompareResult right) { - using CR = ComparisonGraph::CompareResult; + using CR = ComparisonGraphCompareResult; if (left == CR::LESS || left == CR::GREATER) return left; if (right == CR::LESS || right == CR::GREATER) @@ -104,7 +104,7 @@ namespace /// we can add to expression 'indexHint(I < A)' condition. CNFQuery::OrGroup createIndexHintGroup( const CNFQuery::OrGroup & group, - const ComparisonGraph & graph, + const ComparisonGraph<> & graph, const ASTs & primary_key_only_asts) { CNFQuery::OrGroup result; @@ -113,14 +113,14 @@ namespace const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name)) { - auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result) + auto check_and_insert = [&](const size_t index, const ComparisonGraphCompareResult need_result) { if (!onlyConstants(func->arguments->children[1 - index])) return false; for (const auto & primary_key_ast : primary_key_only_asts) { - ComparisonGraph::CompareResult actual_result; + ComparisonGraphCompareResult actual_result; if (index == 0) actual_result = graph.compare(primary_key_ast, func->arguments->children[index]); else diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index aa44a03a0ce..4c4cdd85e2e 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -1,10 +1,17 @@ #include + #include #include #include #include + #include +#include +#include + +#include + namespace DB { @@ -17,7 +24,7 @@ namespace { /// Make function a > b or a >= b -ASTPtr normalizeAtom(const ASTPtr & atom) +ASTPtr normalizeAtom(const ASTPtr & atom, ContextPtr) { static const std::map inverse_relations = { @@ -29,26 +36,155 @@ ASTPtr normalizeAtom(const ASTPtr & atom) if (const auto * func = res->as()) { if (const auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) - { res = makeASTFunction(it->second, func->arguments->children[1]->clone(), func->arguments->children[0]->clone()); - } } return res; } +QueryTreeNodePtr normalizeAtom(const QueryTreeNodePtr & atom, const ContextPtr & context) +{ + static const std::map inverse_relations = + { + {"lessOrEquals", "greaterOrEquals"}, + {"less", "greater"}, + }; + + if (const auto * function_node = atom->as()) + { + if (const auto it = inverse_relations.find(function_node->getFunctionName()); it != inverse_relations.end()) + { + auto inverted_node = function_node->clone(); + auto * inverted_function_node = inverted_node->as(); + auto function_resolver = FunctionFactory::instance().get(it->second, context); + inverted_function_node->resolveAsFunction(function_resolver); + return inverted_node; + } + } + + return atom; +} + +const FunctionNode * tryGetFunctionNode(const QueryTreeNodePtr & node) +{ + return node->as(); +} + +const ASTFunction * tryGetFunctionNode(const ASTPtr & node) +{ + return node->as(); +} + +std::string functionName(const QueryTreeNodePtr & node) +{ + return node->as().getFunctionName(); +} + +std::string functionName(const ASTPtr & node) +{ + return node->as().name; +} + +const Field * tryGetConstantValue(const QueryTreeNodePtr & node) +{ + if (const auto * constant = node->as()) + return &constant->getValue(); + + return nullptr; +} + +const Field * tryGetConstantValue(const ASTPtr & node) +{ + if (const auto * constant = node->as()) + return &constant->value; + + return nullptr; +} + +template +const Field & getConstantValue(const Node & node) +{ + const auto * constant = tryGetConstantValue(node); + assert(constant); + return *constant; +} + +const auto & getNode(const Analyzer::CNF::AtomicFormula & atom) +{ + return atom.node_with_hash.node; +} + +const auto & getNode(const CNFQuery::AtomicFormula & atom) +{ + return atom.ast; +} + +std::string nodeToString(const ASTPtr & ast) +{ + return queryToString(ast); +} + +std::string nodeToString(const QueryTreeNodePtr & node) +{ + return queryToString(node->toAST()); +} + +const auto & getArguments(const ASTFunction * function) +{ + return function->arguments->children; +} + +const auto & getArguments(const FunctionNode * function) +{ + return function->getArguments().getNodes(); +} + bool less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, lhs, rhs); } bool greater(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, rhs, lhs); } bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals{}, lhs, rhs); } +ComparisonGraphCompareResult functionNameToCompareResult(const std::string & name) +{ + using enum ComparisonGraphCompareResult; + static const std::unordered_map relation_to_compare = + { + {"equals", EQUAL}, + {"notEquals", NOT_EQUAL}, + {"less", LESS}, + {"lessOrEquals", LESS_OR_EQUAL}, + {"greaterOrEquals", GREATER_OR_EQUAL}, + {"greater", GREATER}, + }; + + const auto it = relation_to_compare.find(name); + return it == std::end(relation_to_compare) ? UNKNOWN : it->second; } -ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas) +ComparisonGraphCompareResult inverseCompareResult(ComparisonGraphCompareResult result) +{ + using enum ComparisonGraphCompareResult; + static const std::unordered_map inverse_relations = + { + {NOT_EQUAL, EQUAL}, + {EQUAL, NOT_EQUAL}, + {GREATER_OR_EQUAL, LESS}, + {GREATER, LESS_OR_EQUAL}, + {LESS, GREATER_OR_EQUAL}, + {LESS_OR_EQUAL, GREATER}, + {UNKNOWN, UNKNOWN}, + }; + return inverse_relations.at(result); +} + +} + +template +ComparisonGraph::ComparisonGraph(const NodeContainer & atomic_formulas, ContextPtr context) { if (atomic_formulas.empty()) return; - static const std::unordered_map relation_to_enum = + static const std::unordered_map relation_to_enum = { {"equals", Edge::EQUAL}, {"greater", Edge::GREATER}, @@ -63,20 +199,23 @@ ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas) Graph g; for (const auto & atom_raw : atomic_formulas) { - const auto atom = normalizeAtom(atom_raw); + const auto atom = normalizeAtom(atom_raw, context); - auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::optional + auto get_index = [](const Node & node, Graph & nodes_graph) -> std::optional { - const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash()); - if (it != std::end(asts_graph.ast_hash_to_component)) + const auto it = nodes_graph.node_hash_to_component.find(Graph::getHash(node)); + if (it != std::end(nodes_graph.node_hash_to_component)) { if (!std::any_of( - std::cbegin(asts_graph.vertices[it->second].asts), - std::cend(asts_graph.vertices[it->second].asts), - [ast](const ASTPtr & constraint_ast) + std::cbegin(nodes_graph.vertices[it->second].nodes), + std::cend(nodes_graph.vertices[it->second].nodes), + [node](const Node & constraint_node) { - return constraint_ast->getTreeHash() == ast->getTreeHash() - && constraint_ast->getColumnName() == ast->getColumnName(); + if constexpr (with_ast) + return constraint_node->getTreeHash() == node->getTreeHash() + && constraint_node->getColumnName() == node->getColumnName(); + else + return constraint_node->getTreeHash() == node->getTreeHash(); })) { return {}; @@ -86,26 +225,30 @@ ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas) } else { - asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertices.size(); - asts_graph.vertices.push_back(EqualComponent{{ast}, std::nullopt}); - asts_graph.edges.emplace_back(); - return asts_graph.vertices.size() - 1; + nodes_graph.node_hash_to_component[Graph::getHash(node)] = nodes_graph.vertices.size(); + nodes_graph.vertices.push_back(EqualComponent{{node}, std::nullopt}); + nodes_graph.edges.emplace_back(); + return nodes_graph.vertices.size() - 1; } }; - const auto * func = atom->as(); - if (func && func->arguments->children.size() == 2) + const auto * function_node = tryGetFunctionNode(atom); + if (function_node) { - auto index_left = get_index(func->arguments->children[0], g); - auto index_right = get_index(func->arguments->children[1], g); - - if (index_left && index_right) + const auto & arguments = getArguments(function_node); + if (arguments.size() == 2) { - if (const auto it = relation_to_enum.find(func->name); it != std::end(relation_to_enum)) + auto index_left = get_index(arguments[0], g); + auto index_right = get_index(arguments[1], g); + + if (index_left && index_right) { - g.edges[*index_left].push_back(Edge{it->second, *index_right}); - if (it->second == Edge::EQUAL) - g.edges[*index_right].push_back(Edge{it->second, *index_left}); + if (const auto it = relation_to_enum.find(functionName(atom)); it != std::end(relation_to_enum)) + { + g.edges[*index_left].push_back(Edge{it->second, *index_right}); + if (it->second == Edge::EQUAL) + g.edges[*index_right].push_back(Edge{it->second, *index_left}); + } } } } @@ -119,9 +262,9 @@ ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas) /// All expressions from one equivalence class will be stored /// in the corresponding vertex of new graph. - graph = buildGraphFromAstsGraph(g); + graph = buildGraphFromNodesGraph(g); dists = buildDistsFromGraph(graph); - std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds(); + std::tie(node_const_lower_bound, node_const_upper_bound) = buildConstBounds(); /// Find expressions that are known to be unequal. static const std::unordered_set not_equals_functions = {"notEquals", "greater"}; @@ -130,36 +273,45 @@ ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas) /// TODO: Build a graph for unequal components. for (const auto & atom_raw : atomic_formulas) { - const auto atom = normalizeAtom(atom_raw); - const auto * func = atom->as(); + const auto atom = normalizeAtom(atom_raw, context); - if (func && not_equals_functions.contains(func->name)) + const auto * function_node = tryGetFunctionNode(atom); + if (function_node && not_equals_functions.contains(functionName(atom))) { - auto index_left = graph.ast_hash_to_component.at(func->arguments->children[0]->getTreeHash()); - auto index_right = graph.ast_hash_to_component.at(func->arguments->children[1]->getTreeHash()); - if (index_left == index_right) - throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, - "Found expression '{}', but its arguments considered equal according to constraints", - queryToString(atom)); + const auto & arguments = getArguments(function_node); + if (arguments.size() == 2) + { + auto index_left = graph.node_hash_to_component.at(Graph::getHash(arguments[0])); + auto index_right = graph.node_hash_to_component.at(Graph::getHash(arguments[1])); - not_equal.emplace(index_left, index_right); - not_equal.emplace(index_right, index_left); + if (index_left == index_right) + { + throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, + "Found expression '{}', but its arguments considered equal according to constraints", + nodeToString(atom)); + } + + not_equal.emplace(index_left, index_right); + not_equal.emplace(index_right, index_left); + } } } } -ComparisonGraph::CompareResult ComparisonGraph::pathToCompareResult(Path path, bool inverse) +template +ComparisonGraphCompareResult ComparisonGraph::pathToCompareResult(Path path, bool inverse) { switch (path) { - case Path::GREATER: return inverse ? CompareResult::LESS : CompareResult::GREATER; - case Path::GREATER_OR_EQUAL: return inverse ? CompareResult::LESS_OR_EQUAL : CompareResult::GREATER_OR_EQUAL; + case Path::GREATER: return inverse ? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::GREATER; + case Path::GREATER_OR_EQUAL: return inverse ? ComparisonGraphCompareResult::LESS_OR_EQUAL : ComparisonGraphCompareResult::GREATER_OR_EQUAL; } UNREACHABLE(); } -std::optional ComparisonGraph::findPath(size_t start, size_t finish) const +template +std::optional::Path> ComparisonGraph::findPath(size_t start, size_t finish) const { const auto it = dists.find(std::make_pair(start, finish)); if (it == std::end(dists)) @@ -170,18 +322,19 @@ std::optional ComparisonGraph::findPath(size_t start, siz return not_equal.contains({start, finish}) ? Path::GREATER : it->second; } -ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const +template +ComparisonGraphCompareResult ComparisonGraph::compare(const Node & left, const Node & right) const { size_t start = 0; size_t finish = 0; /// TODO: check full ast - const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash()); - const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); + const auto it_left = graph.node_hash_to_component.find(Graph::getHash(left)); + const auto it_right = graph.node_hash_to_component.find(Graph::getHash(right)); - if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component)) + if (it_left == std::end(graph.node_hash_to_component) || it_right == std::end(graph.node_hash_to_component)) { - CompareResult result = CompareResult::UNKNOWN; + auto result = ComparisonGraphCompareResult::UNKNOWN; { const auto left_bound = getConstLowerBound(left); const auto right_bound = getConstUpperBound(right); @@ -189,10 +342,10 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con if (left_bound && right_bound) { if (greater(left_bound->first, right_bound->first)) - result = CompareResult::GREATER; + result = ComparisonGraphCompareResult::GREATER; else if (equals(left_bound->first, right_bound->first)) result = left_bound->second || right_bound->second - ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL; + ? ComparisonGraphCompareResult::GREATER : ComparisonGraphCompareResult::GREATER_OR_EQUAL; } } { @@ -202,10 +355,10 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con if (left_bound && right_bound) { if (less(left_bound->first, right_bound->first)) - result = CompareResult::LESS; + result = ComparisonGraphCompareResult::LESS; else if (equals(left_bound->first, right_bound->first)) result = left_bound->second || right_bound->second - ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; + ? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::LESS_OR_EQUAL; } } @@ -218,7 +371,7 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con } if (start == finish) - return CompareResult::EQUAL; + return ComparisonGraphCompareResult::EQUAL; if (auto path = findPath(start, finish)) return pathToCompareResult(*path, /*inverse=*/ false); @@ -227,93 +380,102 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con return pathToCompareResult(*path, /*inverse=*/ true); if (not_equal.contains({start, finish})) - return CompareResult::NOT_EQUAL; + return ComparisonGraphCompareResult::NOT_EQUAL; - return CompareResult::UNKNOWN; + return ComparisonGraphCompareResult::UNKNOWN; } -bool ComparisonGraph::isPossibleCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const +template +bool ComparisonGraph::isPossibleCompare(ComparisonGraphCompareResult expected, const Node & left, const Node & right) const { const auto result = compare(left, right); - if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) + using enum ComparisonGraphCompareResult; + if (expected == UNKNOWN || result == UNKNOWN) return true; if (expected == result) return true; - static const std::set> possible_pairs = + static const std::set> possible_pairs = { - {CompareResult::EQUAL, CompareResult::LESS_OR_EQUAL}, - {CompareResult::EQUAL, CompareResult::GREATER_OR_EQUAL}, - {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, - {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, - {CompareResult::LESS_OR_EQUAL, CompareResult::NOT_EQUAL}, - {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, - {CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL}, - {CompareResult::GREATER_OR_EQUAL, CompareResult::NOT_EQUAL}, - {CompareResult::LESS, CompareResult::LESS}, - {CompareResult::LESS, CompareResult::LESS_OR_EQUAL}, - {CompareResult::LESS, CompareResult::NOT_EQUAL}, - {CompareResult::GREATER, CompareResult::GREATER}, - {CompareResult::GREATER, CompareResult::GREATER_OR_EQUAL}, - {CompareResult::GREATER, CompareResult::NOT_EQUAL}, - {CompareResult::NOT_EQUAL, CompareResult::LESS}, - {CompareResult::NOT_EQUAL, CompareResult::GREATER}, - {CompareResult::NOT_EQUAL, CompareResult::LESS_OR_EQUAL}, - {CompareResult::NOT_EQUAL, CompareResult::GREATER_OR_EQUAL}, + {EQUAL, LESS_OR_EQUAL}, + {EQUAL, GREATER_OR_EQUAL}, + {LESS_OR_EQUAL, LESS}, + {LESS_OR_EQUAL, EQUAL}, + {LESS_OR_EQUAL, NOT_EQUAL}, + {GREATER_OR_EQUAL, GREATER}, + {GREATER_OR_EQUAL, EQUAL}, + {GREATER_OR_EQUAL, NOT_EQUAL}, + {LESS, LESS}, + {LESS, LESS_OR_EQUAL}, + {LESS, NOT_EQUAL}, + {GREATER, GREATER}, + {GREATER, GREATER_OR_EQUAL}, + {GREATER, NOT_EQUAL}, + {NOT_EQUAL, LESS}, + {NOT_EQUAL, GREATER}, + {NOT_EQUAL, LESS_OR_EQUAL}, + {NOT_EQUAL, GREATER_OR_EQUAL}, }; return possible_pairs.contains({expected, result}); } -bool ComparisonGraph::isAlwaysCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const +template +bool ComparisonGraph::isAlwaysCompare(ComparisonGraphCompareResult expected, const Node & left, const Node & right) const { const auto result = compare(left, right); - if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) + using enum ComparisonGraphCompareResult; + if (expected == UNKNOWN || result == UNKNOWN) return false; if (expected == result) return true; - static const std::set> possible_pairs = + static const std::set> possible_pairs = { - {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, - {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, - {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, - {CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL}, - {CompareResult::NOT_EQUAL, CompareResult::GREATER}, - {CompareResult::NOT_EQUAL, CompareResult::LESS}, + {LESS_OR_EQUAL, LESS}, + {LESS_OR_EQUAL, EQUAL}, + {GREATER_OR_EQUAL, GREATER}, + {GREATER_OR_EQUAL, EQUAL}, + {NOT_EQUAL, GREATER}, + {NOT_EQUAL, LESS}, }; return possible_pairs.contains({expected, result}); } -ASTs ComparisonGraph::getEqual(const ASTPtr & ast) const +template +typename ComparisonGraph::NodeContainer ComparisonGraph::getEqual(const Node & node) const { - const auto res = getComponentId(ast); + const auto res = getComponentId(node); if (!res) return {}; else return getComponent(res.value()); } -std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) const +template +std::optional ComparisonGraph::getComponentId(const Node & node) const { - const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); - if (hash_it == std::end(graph.ast_hash_to_component)) + const auto hash_it = graph.node_hash_to_component.find(Graph::getHash(node)); + if (hash_it == std::end(graph.node_hash_to_component)) return {}; const size_t index = hash_it->second; if (std::any_of( - std::cbegin(graph.vertices[index].asts), - std::cend(graph.vertices[index].asts), - [ast](const ASTPtr & constraint_ast) + std::cbegin(graph.vertices[index].nodes), + std::cend(graph.vertices[index].nodes), + [node](const Node & constraint_node) { - return constraint_ast->getTreeHash() == ast->getTreeHash() && - constraint_ast->getColumnName() == ast->getColumnName(); + if constexpr (with_ast) + return constraint_node->getTreeHash() == node->getTreeHash() + && constraint_node->getColumnName() == node->getColumnName(); + else + return constraint_node->getTreeHash() == node->getTreeHash(); })) { return index; @@ -324,33 +486,38 @@ std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) const } } -bool ComparisonGraph::hasPath(size_t left, size_t right) const +template +bool ComparisonGraph::hasPath(size_t left, size_t right) const { return findPath(left, right) || findPath(right, left); } -ASTs ComparisonGraph::getComponent(size_t id) const +template +typename ComparisonGraph::NodeContainer ComparisonGraph::getComponent(size_t id) const { - return graph.vertices[id].asts; + return graph.vertices[id].nodes; } -bool ComparisonGraph::EqualComponent::hasConstant() const +template +bool ComparisonGraph::EqualComponent::hasConstant() const { return constant_index.has_value(); } -ASTPtr ComparisonGraph::EqualComponent::getConstant() const +template +Node ComparisonGraph::EqualComponent::getConstant() const { assert(constant_index); - return asts[*constant_index]; + return nodes[*constant_index]; } -void ComparisonGraph::EqualComponent::buildConstants() +template +void ComparisonGraph::EqualComponent::buildConstants() { constant_index.reset(); - for (size_t i = 0; i < asts.size(); ++i) + for (size_t i = 0; i < nodes.size(); ++i) { - if (asts[i]->as()) + if (tryGetConstantValue(nodes[i]) != nullptr) { constant_index = i; return; @@ -358,133 +525,111 @@ void ComparisonGraph::EqualComponent::buildConstants() } } -ComparisonGraph::CompareResult ComparisonGraph::atomToCompareResult(const CNFQuery::AtomicFormula & atom) +template +ComparisonGraphCompareResult ComparisonGraph::atomToCompareResult(const typename CNF::AtomicFormula & atom) { - if (const auto * func = atom.ast->as()) + const auto & node = getNode(atom); + if (tryGetFunctionNode(node) != nullptr) { - auto expected = functionNameToCompareResult(func->name); + auto expected = functionNameToCompareResult(functionName(node)); if (atom.negative) expected = inverseCompareResult(expected); return expected; } - return ComparisonGraph::CompareResult::UNKNOWN; + return ComparisonGraphCompareResult::UNKNOWN; } -ComparisonGraph::CompareResult ComparisonGraph::functionNameToCompareResult(const std::string & name) +template +std::optional ComparisonGraph::getEqualConst(const Node & node) const { - static const std::unordered_map relation_to_compare = - { - {"equals", CompareResult::EQUAL}, - {"notEquals", CompareResult::NOT_EQUAL}, - {"less", CompareResult::LESS}, - {"lessOrEquals", CompareResult::LESS_OR_EQUAL}, - {"greaterOrEquals", CompareResult::GREATER_OR_EQUAL}, - {"greater", CompareResult::GREATER}, - }; - - const auto it = relation_to_compare.find(name); - return it == std::end(relation_to_compare) ? CompareResult::UNKNOWN : it->second; -} - -ComparisonGraph::CompareResult ComparisonGraph::inverseCompareResult(CompareResult result) -{ - static const std::unordered_map inverse_relations = - { - {CompareResult::NOT_EQUAL, CompareResult::EQUAL}, - {CompareResult::EQUAL, CompareResult::NOT_EQUAL}, - {CompareResult::GREATER_OR_EQUAL, CompareResult::LESS}, - {CompareResult::GREATER, CompareResult::LESS_OR_EQUAL}, - {CompareResult::LESS, CompareResult::GREATER_OR_EQUAL}, - {CompareResult::LESS_OR_EQUAL, CompareResult::GREATER}, - {CompareResult::UNKNOWN, CompareResult::UNKNOWN}, - }; - return inverse_relations.at(result); -} - -std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const -{ - const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); - if (hash_it == std::end(graph.ast_hash_to_component)) + const auto hash_it = graph.node_hash_to_component.find(Graph::getHash(node)); + if (hash_it == std::end(graph.node_hash_to_component)) return std::nullopt; const size_t index = hash_it->second; return graph.vertices[index].hasConstant() - ? std::optional{graph.vertices[index].getConstant()} + ? std::optional{graph.vertices[index].getConstant()} : std::nullopt; } -std::optional> ComparisonGraph::getConstUpperBound(const ASTPtr & ast) const +template +std::optional> ComparisonGraph::getConstUpperBound(const Node & node) const { - if (const auto * literal = ast->as()) - return std::make_pair(literal->value, false); + if (const auto * constant = tryGetConstantValue(node)) + return std::make_pair(*constant, false); - const auto it = graph.ast_hash_to_component.find(ast->getTreeHash()); - if (it == std::end(graph.ast_hash_to_component)) + const auto it = graph.node_hash_to_component.find(Graph::getHash(node)); + if (it == std::end(graph.node_hash_to_component)) return std::nullopt; const size_t to = it->second; - const ssize_t from = ast_const_upper_bound[to]; + const ssize_t from = node_const_upper_bound[to]; if (from == -1) return std::nullopt; - return std::make_pair(graph.vertices[from].getConstant()->as()->value, dists.at({from, to}) == Path::GREATER); + return std::make_pair(getConstantValue(graph.vertices[from].getConstant()), dists.at({from, to}) == Path::GREATER); } -std::optional> ComparisonGraph::getConstLowerBound(const ASTPtr & ast) const +template +std::optional> ComparisonGraph::getConstLowerBound(const Node & node) const { - if (const auto * literal = ast->as()) - return std::make_pair(literal->value, false); + if (const auto * constant = tryGetConstantValue(node)) + return std::make_pair(*constant, false); - const auto it = graph.ast_hash_to_component.find(ast->getTreeHash()); - if (it == std::end(graph.ast_hash_to_component)) + const auto it = graph.node_hash_to_component.find(Graph::getHash(node)); + if (it == std::end(graph.node_hash_to_component)) return std::nullopt; const size_t from = it->second; - const ssize_t to = ast_const_lower_bound[from]; + const ssize_t to = node_const_lower_bound[from]; if (to == -1) return std::nullopt; - return std::make_pair(graph.vertices[to].getConstant()->as()->value, dists.at({from, to}) == Path::GREATER); + return std::make_pair(getConstantValue(graph.vertices[to].getConstant()), dists.at({from, to}) == Path::GREATER); } -void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) +template +void ComparisonGraph::dfsOrder(const Graph & nodes_graph, size_t v, std::vector & visited, std::vector & order) { visited[v] = true; - for (const auto & edge : asts_graph.edges[v]) + for (const auto & edge : nodes_graph.edges[v]) if (!visited[edge.to]) - dfsOrder(asts_graph, edge.to, visited, order); + dfsOrder(nodes_graph, edge.to, visited, order); order.push_back(v); } -ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) +template +typename ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & nodes_graph) { Graph g; - g.ast_hash_to_component = asts_graph.ast_hash_to_component; - g.vertices = asts_graph.vertices; + g.node_hash_to_component = nodes_graph.node_hash_to_component; + g.vertices = nodes_graph.vertices; g.edges.resize(g.vertices.size()); - for (size_t v = 0; v < asts_graph.vertices.size(); ++v) - for (const auto & edge : asts_graph.edges[v]) + for (size_t v = 0; v < nodes_graph.vertices.size(); ++v) + for (const auto & edge : nodes_graph.edges[v]) g.edges[edge.to].push_back(Edge{edge.type, v}); return g; } -std::vector ComparisonGraph::getVertices() const +template +std::vector::NodeContainer> ComparisonGraph::getVertices() const { - std::vector result; + std::vector result; for (const auto & vertex : graph.vertices) { result.emplace_back(); - for (const auto & ast : vertex.asts) - result.back().push_back(ast); + for (const auto & node : vertex.nodes) + result.back().push_back(node); } return result; } -void ComparisonGraph::dfsComponents( +template +void ComparisonGraph::dfsComponents( const Graph & reversed_graph, size_t v, OptionalIndices & components, size_t component) { @@ -494,11 +639,12 @@ void ComparisonGraph::dfsComponents( dfsComponents(reversed_graph, edge.to, components, component); } -ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & asts_graph) +template +typename ComparisonGraph::Graph ComparisonGraph::buildGraphFromNodesGraph(const Graph & nodes_graph) { /// Find strongly connected component by using 2 dfs traversals. /// https://en.wikipedia.org/wiki/Kosaraju%27s_algorithm - const auto n = asts_graph.vertices.size(); + const auto n = nodes_graph.vertices.size(); std::vector order; { @@ -506,14 +652,14 @@ ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & as for (size_t v = 0; v < n; ++v) { if (!visited[v]) - dfsOrder(asts_graph, v, visited, order); + dfsOrder(nodes_graph, v, visited, order); } } OptionalIndices components(n); size_t component = 0; { - const Graph reversed_graph = reverseGraph(asts_graph); + const Graph reversed_graph = reverseGraph(nodes_graph); for (auto it = order.rbegin(); it != order.rend(); ++it) { if (!components[*it]) @@ -527,14 +673,14 @@ ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & as Graph result; result.vertices.resize(component); result.edges.resize(component); - for (const auto & [hash, index] : asts_graph.ast_hash_to_component) + for (const auto & [hash, index] : nodes_graph.node_hash_to_component) { assert(components[index]); - result.ast_hash_to_component[hash] = *components[index]; - result.vertices[*components[index]].asts.insert( - std::end(result.vertices[*components[index]].asts), - std::begin(asts_graph.vertices[index].asts), - std::end(asts_graph.vertices[index].asts)); // asts_graph has only one ast per vertex + result.node_hash_to_component[hash] = *components[index]; + result.vertices[*components[index]].nodes.insert( + std::end(result.vertices[*components[index]].nodes), + std::begin(nodes_graph.vertices[index].nodes), + std::end(nodes_graph.vertices[index].nodes)); // asts_graph has only one ast per vertex } /// Calculate constants @@ -544,7 +690,7 @@ ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & as /// For each edge in initial graph, we add an edge between components in condensation graph. for (size_t v = 0; v < n; ++v) { - for (const auto & edge : asts_graph.edges[v]) + for (const auto & edge : nodes_graph.edges[v]) result.edges[*components[v]].push_back(Edge{edge.type, *components[edge.to]}); /// TODO: make edges unique (left most strict) @@ -557,11 +703,11 @@ ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & as { if (v != u && result.vertices[v].hasConstant() && result.vertices[u].hasConstant()) { - const auto * left = result.vertices[v].getConstant()->as(); - const auto * right = result.vertices[u].getConstant()->as(); + const auto & left = getConstantValue(result.vertices[v].getConstant()); + const auto & right = getConstantValue(result.vertices[u].getConstant()); /// Only GREATER. Equal constant fields = equal literals so it was already considered above. - if (greater(left->value, right->value)) + if (greater(left, right)) result.edges[v].push_back(Edge{Edge::GREATER, u}); } } @@ -570,7 +716,8 @@ ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & as return result; } -std::map, ComparisonGraph::Path> ComparisonGraph::buildDistsFromGraph(const Graph & g) +template +std::map, typename ComparisonGraph::Path> ComparisonGraph::buildDistsFromGraph(const Graph & g) { /// Min path : -1 means GREATER, 0 means GREATER_OR_EQUALS. /// We use Floyd–Warshall algorithm to find distances between all pairs of vertices. @@ -602,7 +749,8 @@ std::map, ComparisonGraph::Path> ComparisonGraph::buil return path; } -std::pair, std::vector> ComparisonGraph::buildConstBounds() const +template +std::pair, std::vector> ComparisonGraph::buildConstBounds() const { const size_t n = graph.vertices.size(); std::vector lower(n, -1); @@ -610,7 +758,7 @@ std::pair, std::vector> ComparisonGraph::buildCons auto get_value = [this](const size_t vertex) -> Field { - return graph.vertices[vertex].getConstant()->as()->value; + return getConstantValue(graph.vertices[vertex].getConstant()); }; for (const auto & [edge, path] : dists) @@ -637,4 +785,7 @@ std::pair, std::vector> ComparisonGraph::buildCons return {lower, upper}; } +template class ComparisonGraph; +template class ComparisonGraph; + } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 996526b60df..19b23917917 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -2,6 +2,12 @@ #include #include + +#include +#include "Analyzer/HashUtils.h" +#include "Analyzer/IQueryTreeNode.h" + +#include #include #include #include @@ -9,50 +15,56 @@ namespace DB { +enum class ComparisonGraphCompareResult : uint8_t +{ + LESS, + LESS_OR_EQUAL, + EQUAL, + GREATER_OR_EQUAL, + GREATER, + NOT_EQUAL, + UNKNOWN, +}; + +template +concept ComparisonGraphNodeType = std::same_as || std::same_as; + /* * Graph of relations between terms in constraints. * Allows to compare terms and get equal terms. */ +template class ComparisonGraph { public: + static constexpr bool with_ast = std::same_as; + using NodeContainer = std::conditional_t; + using CNF = std::conditional_t; + /// atomic_formulas are extracted from constraints. - explicit ComparisonGraph(const ASTs & atomic_formulas); + explicit ComparisonGraph(const NodeContainer & atomic_formulas, ContextPtr context = nullptr); - enum class CompareResult - { - LESS, - LESS_OR_EQUAL, - EQUAL, - GREATER_OR_EQUAL, - GREATER, - NOT_EQUAL, - UNKNOWN, - }; + static ComparisonGraphCompareResult atomToCompareResult(const typename CNF::AtomicFormula & atom); - static CompareResult atomToCompareResult(const CNFQuery::AtomicFormula & atom); - static CompareResult functionNameToCompareResult(const std::string & name); - static CompareResult inverseCompareResult(CompareResult result); - - CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; + ComparisonGraphCompareResult compare(const Node & left, const Node & right) const; /// It's possible that left right - bool isPossibleCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const; + bool isPossibleCompare(ComparisonGraphCompareResult expected, const Node & left, const Node & right) const; /// It's always true that left right - bool isAlwaysCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const; + bool isAlwaysCompare(ComparisonGraphCompareResult expected, const Node & left, const Node & right) const; - /// Returns all expressions from component to which @ast belongs if any. - ASTs getEqual(const ASTPtr & ast) const; + /// Returns all expressions from component to which @node belongs if any. + NodeContainer getEqual(const Node & node) const; - /// Returns constant expression from component to which @ast belongs if any. - std::optional getEqualConst(const ASTPtr & ast) const; + /// Returns constant expression from component to which @node belongs if any. + std::optional getEqualConst(const Node & node) const; - /// Finds component id to which @ast belongs if any. - std::optional getComponentId(const ASTPtr & ast) const; + /// Finds component id to which @node belongs if any. + std::optional getComponentId(const Node & node) const; /// Returns all expressions from component. - ASTs getComponent(size_t id) const; + NodeContainer getComponent(size_t id) const; size_t getNumOfComponents() const { return graph.vertices.size(); } @@ -61,22 +73,22 @@ public: /// Find constants lessOrEqual and greaterOrEqual. /// For int and double linear programming can be applied here. /// Returns: {constant, is strict less/greater} - std::optional> getConstUpperBound(const ASTPtr & ast) const; - std::optional> getConstLowerBound(const ASTPtr & ast) const; + std::optional> getConstUpperBound(const Node & node) const; + std::optional> getConstLowerBound(const Node & node) const; /// Returns all expression in graph. - std::vector getVertices() const; + std::vector getVertices() const; private: /// Strongly connected component struct EqualComponent { /// All these expressions are considered as equal. - ASTs asts; + NodeContainer nodes; std::optional constant_index; bool hasConstant() const; - ASTPtr getConstant() const; + Node getConstant() const; void buildConstants(); }; @@ -110,20 +122,29 @@ private: } }; - std::unordered_map ast_hash_to_component; + static auto getHash(const Node & node) + { + if constexpr (with_ast) + return node->getTreeHash(); + else + return QueryTreeNodePtrWithHash{node}; + } + + using NodeHashToComponentContainer = std::conditional_t, QueryTreeNodePtrWithHashMap>; + NodeHashToComponentContainer node_hash_to_component; std::vector vertices; std::vector> edges; }; /// Receives graph, in which each vertex corresponds to one expression. /// Then finds strongly connected components and builds graph on them. - static Graph buildGraphFromAstsGraph(const Graph & asts_graph); + static Graph buildGraphFromNodesGraph(const Graph & nodes_graph); - static Graph reverseGraph(const Graph & asts_graph); + static Graph reverseGraph(const Graph & nodes_graph); /// The first part of finding strongly connected components. /// Finds order of exit from vertices of dfs traversal of graph. - static void dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order); + static void dfsOrder(const Graph & nodes_graph, size_t v, std::vector & visited, std::vector & order); using OptionalIndices = std::vector>; @@ -139,13 +160,13 @@ private: GREATER_OR_EQUAL, }; - static CompareResult pathToCompareResult(Path path, bool inverse); + static ComparisonGraphCompareResult pathToCompareResult(Path path, bool inverse); std::optional findPath(size_t start, size_t finish) const; /// Calculate @dists. static std::map, Path> buildDistsFromGraph(const Graph & g); - /// Calculate @ast_const_lower_bound and @ast_const_lower_bound. + /// Calculate @nodeconst_lower_bound and @node_const_lower_bound. std::pair, std::vector> buildConstBounds() const; /// Direct acyclic graph in which each vertex corresponds @@ -165,11 +186,11 @@ private: /// Maximal constant value for each component that /// is lower bound for all expressions in component. - std::vector ast_const_lower_bound; + std::vector node_const_lower_bound; /// Minimal constant value for each component that /// is upper bound for all expressions in component. - std::vector ast_const_upper_bound; + std::vector node_const_upper_bound; }; } diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index d98491aaf9e..4cc9572749f 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -32,13 +32,13 @@ public: struct Data { - const ComparisonGraph & graph; + const ComparisonGraph<> & graph; std::set & components; std::unordered_map & old_name; std::unordered_map & component; UInt64 & current_id; - Data(const ComparisonGraph & graph_, + Data(const ComparisonGraph<> & graph_, std::set & components_, std::unordered_map & old_name_, std::unordered_map & component_, @@ -165,7 +165,7 @@ ColumnPrice calculatePrice( /// price of all columns on which ast depends. /// TODO: branch-and-bound void bruteforce( - const ComparisonGraph & graph, + const ComparisonGraph<> & graph, const std::vector & components, size_t current_component, const ColumnPriceByName & column_prices, diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index d036c6728fe..1613b09ee48 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -360,80 +360,14 @@ CNFQuery & CNFQuery::pushNotInFunctions() return *this; } -namespace -{ - CNFQuery::AndGroup reduceOnce(const CNFQuery::AndGroup & groups) - { - CNFQuery::AndGroup result; - for (const CNFQuery::OrGroup & group : groups) - { - CNFQuery::OrGroup copy(group); - bool inserted = false; - for (const CNFQuery::AtomicFormula & atom : group) - { - copy.erase(atom); - CNFQuery::AtomicFormula negative_atom(atom); - negative_atom.negative = !atom.negative; - copy.insert(negative_atom); - - if (groups.contains(copy)) - { - copy.erase(negative_atom); - result.insert(copy); - inserted = true; - break; - } - - copy.erase(negative_atom); - copy.insert(atom); - } - if (!inserted) - result.insert(group); - } - return result; - } - - bool isSubset(const CNFQuery::OrGroup & left, const CNFQuery::OrGroup & right) - { - if (left.size() > right.size()) - return false; - for (const auto & elem : left) - if (!right.contains(elem)) - return false; - return true; - } - - CNFQuery::AndGroup filterSubsets(const CNFQuery::AndGroup & groups) - { - CNFQuery::AndGroup result; - for (const CNFQuery::OrGroup & group : groups) - { - bool insert = true; - - for (const CNFQuery::OrGroup & other_group : groups) - { - if (isSubset(other_group, group) && group != other_group) - { - insert = false; - break; - } - } - - if (insert) - result.insert(group); - } - return result; - } -} - CNFQuery & CNFQuery::reduce() { while (true) { - AndGroup new_statements = reduceOnce(statements); + AndGroup new_statements = reduceOnceCNFStatements(statements); if (statements == new_statements) { - statements = filterSubsets(statements); + statements = filterCNFSubsets(statements); return *this; } else diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 70c8990f74a..7f2fee4e6fd 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -164,4 +164,72 @@ public: void pushNotIn(CNFQuery::AtomicFormula & atom); +template +TAndGroup reduceOnceCNFStatements(const TAndGroup & groups) +{ + TAndGroup result; + for (const auto & group : groups) + { + using GroupType = std::decay_t; + GroupType copy(group); + bool inserted = false; + for (const auto & atom : group) + { + copy.erase(atom); + using AtomType = std::decay_t; + AtomType negative_atom(atom); + negative_atom.negative = !atom.negative; + copy.insert(negative_atom); + + if (groups.contains(copy)) + { + copy.erase(negative_atom); + result.insert(copy); + inserted = true; + break; + } + + copy.erase(negative_atom); + copy.insert(atom); + } + if (!inserted) + result.insert(group); + } + return result; +} + +template +bool isCNFGroupSubset(const TOrGroup & left, const TOrGroup & right) +{ + if (left.size() > right.size()) + return false; + for (const auto & elem : left) + if (!right.contains(elem)) + return false; + return true; +} + +template +TAndGroup filterCNFSubsets(const TAndGroup & groups) +{ + TAndGroup result; + for (const auto & group : groups) + { + bool insert = true; + + for (const auto & other_group : groups) + { + if (isCNFGroupSubset(other_group, group) && group != other_group) + { + insert = false; + break; + } + } + + if (insert) + result.insert(group); + } + return result; +} + } diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 234b99167bb..e3934e8ea7f 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -74,7 +74,7 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const Cons return false; } -bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph & graph) +bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph<> & graph) { /// We try to find at least one atom that is always true by using comparison graph. for (const auto & atom : group) @@ -82,7 +82,7 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { - const auto expected = ComparisonGraph::atomToCompareResult(atom); + const auto expected = ComparisonGraph<>::atomToCompareResult(atom); if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1])) return true; } @@ -108,20 +108,20 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const return false; } -bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph) +bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<> & graph) { const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { /// TODO: special support for != - const auto expected = ComparisonGraph::atomToCompareResult(atom); + const auto expected = ComparisonGraph<>::atomToCompareResult(atom); return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]); } return false; } -void replaceToConstants(ASTPtr & term, const ComparisonGraph & graph) +void replaceToConstants(ASTPtr & term, const ComparisonGraph<> & graph) { const auto equal_constant = graph.getEqualConst(term); if (equal_constant) @@ -135,7 +135,7 @@ void replaceToConstants(ASTPtr & term, const ComparisonGraph & graph) } } -CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph) +CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<> & graph) { CNFQuery::AtomicFormula result; result.negative = atom.negative; diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 5207458af8c..b804a6e106a 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -11,6 +11,9 @@ #include +#include +#include + namespace DB { @@ -103,7 +106,7 @@ std::vector ConstraintsDescription::getAtomicConstraint return constraint_data; } -std::unique_ptr ConstraintsDescription::buildGraph() const +std::unique_ptr> ConstraintsDescription::buildGraph() const { static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; @@ -121,7 +124,7 @@ std::unique_ptr ConstraintsDescription::buildGraph() const } } - return std::make_unique(constraints_for_graph); + return std::make_unique>(constraints_for_graph); } ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextPtr context, @@ -143,7 +146,7 @@ ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextP return res; } -const ComparisonGraph & ConstraintsDescription::getGraph() const +const ComparisonGraph<> & ConstraintsDescription::getGraph() const { return *graph; } @@ -175,6 +178,93 @@ std::vector ConstraintsDescription::getAtomsById(const return result; } +const ConstraintsDescription::QueryTreeData & ConstraintsDescription::getQueryTreeData(const ContextPtr & context) const +{ + if (!query_tree_data) + { + QueryTreeData data; + std::vector atomic_constraints_data; + for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + { + auto query_tree = buildQueryTree(constraint->as()->expr->ptr(), context); + const auto cnf = Analyzer::CNF::toCNF(query_tree, context) + .pullNotOutFunctions(context); + for (const auto & group : cnf.getStatements()) + { + data.cnf_constraints.emplace_back(group.begin(), group.end()); + + if (group.size() == 1) + atomic_constraints_data.emplace_back(*group.begin()); + } + + data.constraints.push_back(std::move(query_tree)); + } + + for (size_t i = 0; i < data.cnf_constraints.size(); ++i) + for (size_t j = 0; j < data.cnf_constraints[i].size(); ++j) + data.query_node_to_atom_ids[data.cnf_constraints[i][j].node_with_hash].push_back({i, j}); + + /// build graph + if (constraints.empty()) + { + data.graph = std::make_unique>(QueryTreeNodes()); + } + else + { + static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; + + QueryTreeNodes constraints_for_graph; + for (const auto & atomic_formula : atomic_constraints_data) + { + Analyzer::CNF::AtomicFormula atom{atomic_formula.negative, atomic_formula.node_with_hash.node->clone()}; + atom = Analyzer::CNF::pushNotIntoFunction(atom, context); + + auto * function_node = atom.node_with_hash.node->as(); + if (function_node && relations.contains(function_node->getFunctionName())) + { + assert(!atom.negative); + constraints_for_graph.push_back(atom.node_with_hash.node); + } + } + + } + + query_tree_data.emplace(std::move(data)); + } + + return *query_tree_data; +} +const QueryTreeNodes & ConstraintsDescription::QueryTreeData::getConstraints() const +{ + return constraints; +} + +const std::vector> & ConstraintsDescription::QueryTreeData::getConstraintData() const +{ + return cnf_constraints; +} + +const ComparisonGraph & ConstraintsDescription::QueryTreeData::getGraph() const +{ + return *graph; +} + +std::optional ConstraintsDescription::QueryTreeData::getAtomIds(const QueryTreeNodePtrWithHash & node_with_hash) const +{ + auto it = query_node_to_atom_ids.find(node_with_hash); + if (it != query_node_to_atom_ids.end()) + return it->second; + return std::nullopt; +} + +std::vector ConstraintsDescription::QueryTreeData::getAtomsById(const AtomIds & ids) const +{ + std::vector result; + for (const auto & id : ids) + result.push_back(cnf_constraints[id.group_id][id.atom_id]); + return result; +} + ConstraintsDescription::ConstraintsDescription(const ASTs & constraints_) : constraints(constraints_) { @@ -218,7 +308,7 @@ void ConstraintsDescription::update() { cnf_constraints.clear(); ast_to_atom_ids.clear(); - graph = std::make_unique(ASTs()); + graph = std::make_unique>(ASTs()); return; } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index eb1eb95d33d..fcfb533ac8b 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -5,6 +5,8 @@ #include #include +#include + namespace DB { @@ -41,7 +43,7 @@ public: const std::vector> & getConstraintData() const; std::vector getAtomicConstraintData() const; - const ComparisonGraph & getGraph() const; + const ComparisonGraph<> & getGraph() const; ConstraintsExpressions getExpressions(ContextPtr context, const NamesAndTypesList & source_columns_) const; @@ -56,15 +58,38 @@ public: std::optional getAtomIds(const ASTPtr & ast) const; std::vector getAtomsById(const AtomIds & ids) const; + class QueryTreeData + { + public: + const QueryTreeNodes & getConstraints() const; + const std::vector> & getConstraintData() const; + std::optional getAtomIds(const QueryTreeNodePtrWithHash & node_with_hash) const; + std::vector getAtomsById(const AtomIds & ids) const; + const ComparisonGraph & getGraph() const; + private: + QueryTreeNodes constraints; + std::vector> cnf_constraints; + QueryTreeNodePtrWithHashMap query_node_to_atom_ids; + std::unique_ptr> graph; + + friend ConstraintsDescription; + }; + + const QueryTreeData & getQueryTreeData(const ContextPtr & context) const; + private: std::vector> buildConstraintData() const; - std::unique_ptr buildGraph() const; + std::unique_ptr> buildGraph() const; void update(); ASTs constraints; + std::vector> cnf_constraints; std::map ast_to_atom_ids; - std::unique_ptr graph; + + mutable std::optional query_tree_data; + + std::unique_ptr> graph; }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp index 1ab64fc84c7..2bb6857a855 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp @@ -108,7 +108,7 @@ bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const func->name = "greaterOrEquals"; } - const auto weak_graph = std::make_unique(active_atomic_formulas); + const auto weak_graph = std::make_unique>(active_atomic_formulas); bool useless = true; expression_cnf->iterateGroups( @@ -146,7 +146,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree values.push_back(granule->met); } - const ComparisonGraph * graph = nullptr; + const ComparisonGraph<> * graph = nullptr; { std::lock_guard lock(cache_mutex); @@ -170,7 +170,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { - const auto expected = ComparisonGraph::atomToCompareResult(atom); + const auto expected = ComparisonGraph<>::atomToCompareResult(atom); if (graph->isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) { /// If graph failed use matching. @@ -188,7 +188,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree return !always_false; } -std::unique_ptr MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector & values) const +std::unique_ptr> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector & values) const { ASTs active_atomic_formulas(atomic_constraints); for (size_t i = 0; i < values.size(); ++i) @@ -199,10 +199,10 @@ std::unique_ptr MergeTreeIndexhypothesisMergedCondition::buildG std::begin(index_to_compare_atomic_hypotheses[i]), std::end(index_to_compare_atomic_hypotheses[i])); } - return std::make_unique(active_atomic_formulas); + return std::make_unique>(active_atomic_formulas); } -const ComparisonGraph * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const +const ComparisonGraph<> * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const { auto [it, inserted] = graph_cache.try_emplace(values); if (inserted) diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h index 6153c214898..f08cfba6ca0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h @@ -20,8 +20,8 @@ public: private: void addConstraints(const ConstraintsDescription & constraints_description); - std::unique_ptr buildGraph(const std::vector & values) const; - const ComparisonGraph * getGraph(const std::vector & values) const; + std::unique_ptr> buildGraph(const std::vector & values) const; + const ComparisonGraph<> * getGraph(const std::vector & values) const; ASTPtr expression_ast; std::unique_ptr expression_cnf; @@ -29,7 +29,7 @@ private: /// Part analysis can be done in parallel. /// So, we have shared answer and graph cache. mutable std::mutex cache_mutex; - mutable std::unordered_map, std::unique_ptr> graph_cache; + mutable std::unordered_map, std::unique_ptr>> graph_cache; mutable std::unordered_map, bool> answer_cache; std::vector> index_to_compare_atomic_hypotheses; From c7787a6652ebee5c682afc6733d65e45d5cc5107 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Mar 2023 14:57:07 +0000 Subject: [PATCH 033/233] Fix column resolution --- src/Analyzer/Passes/CNF.cpp | 5 - src/Analyzer/Passes/CNF.h | 2 +- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 89 +++++++---------- src/Interpreters/ComparisonGraph.cpp | 8 +- src/Storages/ConstraintsDescription.cpp | 99 ++++++++++--------- src/Storages/ConstraintsDescription.h | 4 +- 6 files changed, 93 insertions(+), 114 deletions(-) diff --git a/src/Analyzer/Passes/CNF.cpp b/src/Analyzer/Passes/CNF.cpp index 3d95200a948..38d316a13f3 100644 --- a/src/Analyzer/Passes/CNF.cpp +++ b/src/Analyzer/Passes/CNF.cpp @@ -495,11 +495,6 @@ CNF::CNF(AndGroup statements_) std::optional CNF::tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier) { - auto * function_node = node->as(); - - if (!function_node || !isLogicalFunction(*function_node)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert nodes that are not logical functions to CNF"); - auto node_cloned = node->clone(); size_t atom_count = countAtoms(node_cloned); diff --git a/src/Analyzer/Passes/CNF.h b/src/Analyzer/Passes/CNF.h index 238e32c4ce1..8ddbf7efd85 100644 --- a/src/Analyzer/Passes/CNF.h +++ b/src/Analyzer/Passes/CNF.h @@ -72,7 +72,7 @@ public: static std::optional tryBuildCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); static CNF toCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_growth_multiplier = DEFAULT_MAX_GROWTH_MULTIPLIER); - QueryTreeNodePtr toQueryTree(ContextPtr context) const; + QueryTreeNodePtr toQueryTree(ContextPtr context) const; const auto & getStatements() const { diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 278cadf2638..27aea91d261 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -3,13 +3,13 @@ #include #include #include -#include +#include #include +#include #include #include #include -#include "Analyzer/HashUtils.h" namespace DB { @@ -17,19 +17,8 @@ namespace DB namespace { -bool isLogicalFunction(const FunctionNode & function_node) -{ - const std::string_view name = function_node.getFunctionName(); - return name == "and" || name == "or" || name == "not"; -} - std::optional tryConvertQueryToCNF(const QueryTreeNodePtr & node, const ContextPtr & context) { - auto * function_node = node->as(); - - if (!function_node || !isLogicalFunction(*function_node)) - return std::nullopt; - auto cnf_form = Analyzer::CNF::tryBuildCNF(node, context); if (!cnf_form) return std::nullopt; @@ -54,25 +43,24 @@ MatchState match(const Analyzer::CNF::AtomicFormula & a, const Analyzer::CNF::At return a.negative == b.negative ? FULL_MATCH : PARTIAL_MATCH; } -bool checkIfGroupAlwaysTrueFullMatch(const Analyzer::CNF::OrGroup & group, const ConstraintsDescription & constraints_description, const ContextPtr & context) +bool checkIfGroupAlwaysTrueFullMatch(const Analyzer::CNF::OrGroup & group, const ConstraintsDescription::QueryTreeData & query_tree_constraints) { /// We have constraints in CNF. /// CNF is always true => Each OR group in CNF is always true. /// So, we try to check whether we have al least one OR group from CNF as subset in our group. /// If we've found one then our group is always true too. - const auto & query_tree_constraint = constraints_description.getQueryTreeData(context); - const auto & constraints_data = query_tree_constraint.getConstraintData(); + const auto & constraints_data = query_tree_constraints.getConstraintData(); std::vector found(constraints_data.size()); for (size_t i = 0; i < constraints_data.size(); ++i) found[i] = constraints_data[i].size(); for (const auto & atom : group) { - const auto constraint_atom_ids = query_tree_constraint.getAtomIds(atom.node_with_hash); + const auto constraint_atom_ids = query_tree_constraints.getAtomIds(atom.node_with_hash); if (constraint_atom_ids) { - const auto constraint_atoms = query_tree_constraint.getAtomsById(*constraint_atom_ids); + const auto constraint_atoms = query_tree_constraints.getAtomsById(*constraint_atom_ids); for (size_t i = 0; i < constraint_atoms.size(); ++i) { if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH) @@ -107,13 +95,12 @@ bool checkIfGroupAlwaysTrueGraph(const Analyzer::CNF::OrGroup & group, const Com return false; } -bool checkIfAtomAlwaysFalseFullMatch(const Analyzer::CNF::AtomicFormula & atom, const ConstraintsDescription & constraints_description, const ContextPtr & context) +bool checkIfAtomAlwaysFalseFullMatch(const Analyzer::CNF::AtomicFormula & atom, const ConstraintsDescription::QueryTreeData & query_tree_constraints) { - const auto & query_tree_constraint = constraints_description.getQueryTreeData(context); - const auto constraint_atom_ids = query_tree_constraint.getAtomIds(atom.node_with_hash); + const auto constraint_atom_ids = query_tree_constraints.getAtomIds(atom.node_with_hash); if (constraint_atom_ids) { - for (const auto & constraint_atom : query_tree_constraint.getAtomsById(*constraint_atom_ids)) + for (const auto & constraint_atom : query_tree_constraints.getAtomsById(*constraint_atom_ids)) { const auto match_result = match(constraint_atom, atom); if (match_result == MatchState::PARTIAL_MATCH) @@ -149,7 +136,10 @@ void replaceToConstants(QueryTreeNodePtr & term, const ComparisonGraphgetChildren()) - replaceToConstants(child, graph); + { + if (child) + replaceToConstants(child, graph); + } } Analyzer::CNF::AtomicFormula replaceTermsToConstants(const Analyzer::CNF::AtomicFormula & atom, const ComparisonGraph & graph) @@ -159,31 +149,28 @@ Analyzer::CNF::AtomicFormula replaceTermsToConstants(const Analyzer::CNF::Atomic return {atom.negative, std::move(node)}; } -StorageMetadataPtr getStorageMetadata(const QueryTreeNodePtr & node) +StorageSnapshotPtr getStorageSnapshot(const QueryTreeNodePtr & node) { StorageSnapshotPtr storage_snapshot{nullptr}; if (auto * table_node = node->as()) - storage_snapshot = table_node->getStorageSnapshot(); + return table_node->getStorageSnapshot(); else if (auto * table_function_node = node->as()) - storage_snapshot = table_function_node->getStorageSnapshot(); + return table_function_node->getStorageSnapshot(); - if (!storage_snapshot) - return nullptr; - - return storage_snapshot->metadata; + return nullptr; } bool onlyIndexColumns(const QueryTreeNodePtr & node, const std::unordered_set & primary_key_set) { - const auto * identifier_node = node->as(); + const auto * column_node = node->as(); /// TODO: verify that full name is correct here - if (identifier_node && !primary_key_set.contains(identifier_node->getIdentifier().getFullName())) + if (column_node && !primary_key_set.contains(column_node->getColumnName())) return false; for (const auto & child : node->getChildren()) { - if (!onlyIndexColumns(child, primary_key_set)) - return false; + if (child && !onlyIndexColumns(child, primary_key_set)) + return false; } return true; @@ -191,16 +178,8 @@ bool onlyIndexColumns(const QueryTreeNodePtr & node, const std::unordered_setas() != nullptr) - return false; - - for (const auto & child : node->getChildren()) - { - if (!onlyConstants(child)) - return false; - } - - return true; + /// if it's only constant it will be already calculated + return node->as() != nullptr; } const std::unordered_map & getRelationMap() @@ -322,13 +301,13 @@ void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expres { for (const auto & table_expression : table_expressions) { - auto metadata = getStorageMetadata(table_expression); - if (!metadata) + auto snapshot = getStorageSnapshot(table_expression); + if (!snapshot || !snapshot->metadata) continue; - const auto primary_key = metadata->getColumnsRequiredForPrimaryKey(); + const auto primary_key = snapshot->metadata->getColumnsRequiredForPrimaryKey(); const std::unordered_set primary_key_set(primary_key.begin(), primary_key.end()); - const auto & query_tree_constraint = metadata->getConstraints().getQueryTreeData(context); + const auto & query_tree_constraint = snapshot->metadata->getConstraints().getQueryTreeData(context, table_expression); const auto & graph = query_tree_constraint.getGraph(); QueryTreeNodes primary_key_only_nodes; @@ -369,22 +348,22 @@ void optimizeWithConstraints(Analyzer::CNF & cnf, const QueryTreeNodes & table_e for (const auto & table_expression : table_expressions) { - auto metadata = getStorageMetadata(table_expression); - if (!metadata) + auto snapshot = getStorageSnapshot(table_expression); + if (!snapshot || !snapshot->metadata) continue; - const auto & constraints = metadata->getConstraints(); - const auto & query_tree_constraint = constraints.getQueryTreeData(context); - const auto & compare_graph = query_tree_constraint.getGraph(); + const auto & constraints = snapshot->metadata->getConstraints(); + const auto & query_tree_constraints = constraints.getQueryTreeData(context, table_expression); + const auto & compare_graph = query_tree_constraints.getGraph(); cnf.filterAlwaysTrueGroups([&](const auto & group) { /// remove always true groups from CNF - return !checkIfGroupAlwaysTrueFullMatch(group, constraints, context) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); + return !checkIfGroupAlwaysTrueFullMatch(group, query_tree_constraints) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); }) .filterAlwaysFalseAtoms([&](const Analyzer::CNF::AtomicFormula & atom) { /// remove always false atoms from CNF - return !checkIfAtomAlwaysFalseFullMatch(atom, constraints, context) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); + return !checkIfAtomAlwaysFalseFullMatch(atom, query_tree_constraints) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); }) .transformAtoms([&](const auto & atom) { diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 4c4cdd85e2e..62f9a7867b4 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -57,6 +57,9 @@ QueryTreeNodePtr normalizeAtom(const QueryTreeNodePtr & atom, const ContextPtr & auto inverted_node = function_node->clone(); auto * inverted_function_node = inverted_node->as(); auto function_resolver = FunctionFactory::instance().get(it->second, context); + auto & arguments = inverted_function_node->getArguments().getNodes(); + assert(arguments.size() == 2); + std::swap(arguments[0], arguments[1]); inverted_function_node->resolveAsFunction(function_resolver); return inverted_node; } @@ -215,7 +218,7 @@ ComparisonGraph::ComparisonGraph(const NodeContainer & atomic_formulas, Co return constraint_node->getTreeHash() == node->getTreeHash() && constraint_node->getColumnName() == node->getColumnName(); else - return constraint_node->getTreeHash() == node->getTreeHash(); + return constraint_node->isEqual(*node); })) { return {}; @@ -278,7 +281,6 @@ ComparisonGraph::ComparisonGraph(const NodeContainer & atomic_formulas, Co const auto * function_node = tryGetFunctionNode(atom); if (function_node && not_equals_functions.contains(functionName(atom))) { - const auto & arguments = getArguments(function_node); if (arguments.size() == 2) { @@ -782,7 +784,7 @@ std::pair, std::vector> ComparisonGraph::bui } } - return {lower, upper}; + return {std::move(lower), std::move(upper)}; } template class ComparisonGraph; diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index b804a6e106a..3f4d264b4a1 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -13,7 +13,11 @@ #include #include +#include +#include +#include +#include namespace DB { @@ -178,62 +182,63 @@ std::vector ConstraintsDescription::getAtomsById(const return result; } -const ConstraintsDescription::QueryTreeData & ConstraintsDescription::getQueryTreeData(const ContextPtr & context) const +ConstraintsDescription::QueryTreeData ConstraintsDescription::getQueryTreeData(const ContextPtr & context, const QueryTreeNodePtr & table_node) const { - if (!query_tree_data) + QueryTreeData data; + std::vector atomic_constraints_data; + + QueryAnalysisPass pass(table_node); + + for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { - QueryTreeData data; - std::vector atomic_constraints_data; - for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + auto query_tree = buildQueryTree(constraint->as()->expr->ptr(), context); + pass.run(query_tree, context); + + const auto cnf = Analyzer::CNF::toCNF(query_tree, context) + .pullNotOutFunctions(context); + for (const auto & group : cnf.getStatements()) { - auto query_tree = buildQueryTree(constraint->as()->expr->ptr(), context); - const auto cnf = Analyzer::CNF::toCNF(query_tree, context) - .pullNotOutFunctions(context); - for (const auto & group : cnf.getStatements()) - { - data.cnf_constraints.emplace_back(group.begin(), group.end()); + data.cnf_constraints.emplace_back(group.begin(), group.end()); - if (group.size() == 1) - atomic_constraints_data.emplace_back(*group.begin()); - } - - data.constraints.push_back(std::move(query_tree)); + if (group.size() == 1) + atomic_constraints_data.emplace_back(*group.begin()); } - for (size_t i = 0; i < data.cnf_constraints.size(); ++i) - for (size_t j = 0; j < data.cnf_constraints[i].size(); ++j) - data.query_node_to_atom_ids[data.cnf_constraints[i][j].node_with_hash].push_back({i, j}); - - /// build graph - if (constraints.empty()) - { - data.graph = std::make_unique>(QueryTreeNodes()); - } - else - { - static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; - - QueryTreeNodes constraints_for_graph; - for (const auto & atomic_formula : atomic_constraints_data) - { - Analyzer::CNF::AtomicFormula atom{atomic_formula.negative, atomic_formula.node_with_hash.node->clone()}; - atom = Analyzer::CNF::pushNotIntoFunction(atom, context); - - auto * function_node = atom.node_with_hash.node->as(); - if (function_node && relations.contains(function_node->getFunctionName())) - { - assert(!atom.negative); - constraints_for_graph.push_back(atom.node_with_hash.node); - } - } - - } - - query_tree_data.emplace(std::move(data)); + data.constraints.push_back(std::move(query_tree)); } - return *query_tree_data; + for (size_t i = 0; i < data.cnf_constraints.size(); ++i) + for (size_t j = 0; j < data.cnf_constraints[i].size(); ++j) + data.query_node_to_atom_ids[data.cnf_constraints[i][j].node_with_hash].push_back({i, j}); + + /// build graph + if (constraints.empty()) + { + data.graph = std::make_unique>(QueryTreeNodes(), context); + } + else + { + static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; + + QueryTreeNodes constraints_for_graph; + for (const auto & atomic_formula : atomic_constraints_data) + { + Analyzer::CNF::AtomicFormula atom{atomic_formula.negative, atomic_formula.node_with_hash.node->clone()}; + atom = Analyzer::CNF::pushNotIntoFunction(atom, context); + + auto * function_node = atom.node_with_hash.node->as(); + if (function_node && relations.contains(function_node->getFunctionName())) + { + assert(!atom.negative); + constraints_for_graph.push_back(atom.node_with_hash.node); + } + } + data.graph = std::make_unique>(constraints_for_graph, context); + } + + return data; } + const QueryTreeNodes & ConstraintsDescription::QueryTreeData::getConstraints() const { return constraints; diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index fcfb533ac8b..2c34ef1ef37 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -75,7 +75,7 @@ public: friend ConstraintsDescription; }; - const QueryTreeData & getQueryTreeData(const ContextPtr & context) const; + QueryTreeData getQueryTreeData(const ContextPtr & context, const QueryTreeNodePtr & table_node) const; private: std::vector> buildConstraintData() const; @@ -87,8 +87,6 @@ private: std::vector> cnf_constraints; std::map ast_to_atom_ids; - mutable std::optional query_tree_data; - std::unique_ptr> graph; }; From 1d735b37735d467774de187984bd0fdf8b744040 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 16 Mar 2023 15:47:27 +0000 Subject: [PATCH 034/233] Fix build --- .../Optimizations/distinctReadInOrder.cpp | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 6334594de30..04f10cfa821 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -26,34 +26,6 @@ static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_st return path_actions; } -static const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name) -{ - /// find alias in output - const ActionsDAG::Node * output_alias = nullptr; - for (const auto * node : actions->getOutputs()) - { - if (node->result_name == output_name) - { - output_alias = node; - break; - } - } - if (!output_alias) - return nullptr; - - /// find original(non alias) node it refers to - const ActionsDAG::Node * node = output_alias; - while (node && node->type == ActionsDAG::ActionType::ALIAS) - { - chassert(!node->children.empty()); - node = node->children.front(); - } - if (node && node->type != ActionsDAG::ActionType::INPUT) - return nullptr; - - return node; -} - static std::set getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector & dag_stack) { From e07a8e1fd28a5f53b03392abd12dbca755ba0a5f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Mar 2023 15:59:48 +0000 Subject: [PATCH 035/233] Some fixes with tests --- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 9 +- src/Analyzer/QueryTreePassManager.cpp | 2 - src/Interpreters/ComparisonGraph.cpp | 15 +- ..._constraints_simple_optimization.reference | 58 ++ .../01622_constraints_simple_optimization.sql | 4 + ...2_constraints_where_optimization.reference | 52 ++ .../01622_constraints_where_optimization.sql | 7 +- .../01623_constraints_column_swap.reference | 114 +++ .../01623_constraints_column_swap.sql | 4 + .../01625_constraints_index_append.reference | 101 +++ .../01625_constraints_index_append.sql | 4 + .../0_stateless/01626_cnf_test.reference | 750 ++++++++++++++++++ tests/queries/0_stateless/01626_cnf_test.sql | 6 + 13 files changed, 1115 insertions(+), 11 deletions(-) diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 27aea91d261..781fb857a07 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -255,10 +255,10 @@ Analyzer::CNF::OrGroup createIndexHintGroup( for (const auto & atom : group) { const auto * function_node = atom.node_with_hash.node->as(); - if (!function_node || getRelationMap().contains(function_node->getFunctionName())) + if (!function_node || !getRelationMap().contains(function_node->getFunctionName())) continue; - const auto & arguments = function_node->getArguments().getNodes(); + const auto & arguments = function_node->getArguments().getNodes(); if (arguments.size() != 2) continue; @@ -279,6 +279,7 @@ Analyzer::CNF::OrGroup createIndexHintGroup( { auto helper_node = function_node->clone(); auto & helper_function_node = helper_node->as(); + helper_function_node.getArguments().getNodes()[index] = primary_key_node->clone(); auto reverse_function_name = getReverseRelationMap().at(mostStrict(expected_result, actual_result)); helper_function_node.resolveAsFunction(FunctionFactory::instance().get(reverse_function_name, context)); result.insert(Analyzer::CNF::AtomicFormula{atom.negative, std::move(helper_node)}); @@ -307,6 +308,7 @@ void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expres const auto primary_key = snapshot->metadata->getColumnsRequiredForPrimaryKey(); const std::unordered_set primary_key_set(primary_key.begin(), primary_key.end()); + const auto & query_tree_constraint = snapshot->metadata->getConstraints().getQueryTreeData(context, table_expression); const auto & graph = query_tree_constraint.getGraph(); @@ -391,9 +393,6 @@ void optimizeNode(QueryTreeNodePtr & node, const QueryTreeNodes & table_expressi optimizeWithConstraints(*cnf, table_expressions, context); auto new_node = cnf->toQueryTree(context); - if (!new_node) - return; - node = std::move(new_node); } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index bd9a2d4618c..a9b812d71b4 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -148,8 +148,6 @@ private: /** ClickHouse query tree pass manager. * - * TODO: Support setting convert_query_to_cnf. - * TODO: Support setting optimize_using_constraints. * TODO: Support setting optimize_substitute_columns. * TODO: Support GROUP BY injective function elimination. * TODO: Support setting optimize_move_functions_out_of_any. diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 62f9a7867b4..53e63903c43 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -550,9 +550,18 @@ std::optional ComparisonGraph::getEqualConst(const Node & node) cons return std::nullopt; const size_t index = hash_it->second; - return graph.vertices[index].hasConstant() - ? std::optional{graph.vertices[index].getConstant()} - : std::nullopt; + + if (!graph.vertices[index].hasConstant()) + return std::nullopt; + + if constexpr (with_ast) + return graph.vertices[index].getConstant(); + else + { + const auto & constant = getConstantValue(graph.vertices[index].getConstant()); + auto constant_node = std::make_shared(constant, node->getResultType()); + return constant_node; + } } template diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 7e012e1a17b..f92b8432b65 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -35,11 +35,69 @@ SELECT count() AS `count()` FROM constraint_test_constants WHERE (c > 100) OR (b > 100) +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: b, result_type: Int64, source_id: 3 + CONSTANT id: 9, constant_value: UInt64_100, constant_value_type: UInt8 + FUNCTION id: 10, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: c, result_type: Int64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants + WHERE + FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants + WHERE + FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 7ec9e1a3158..8b7a339984d 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -98,8 +98,12 @@ SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> -- A AND NOT A EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); DROP TABLE constraint_test_constants; diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index c7c516025f2..52aca371a6a 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -1,14 +1,66 @@ SELECT count() FROM t_constraints_where WHERE 0 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE 0 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE 0 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE b < 8 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 SELECT count() FROM t_constraints_where +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index 6a9d1ba9f6b..33fa62368b0 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -8,9 +8,13 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) DROP TABLE t_constraints_where; @@ -18,6 +22,7 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10 INSERT INTO t_constraints_where VALUES (1, 7); -EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumtion -> (b < 20) -> 0; +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 7ae4516fe9e..eceaa7122dc 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -3,21 +3,135 @@ SELECT (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 1 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 10, nodes: 2 + COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 + CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 7, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 14, nodes: 2 + FUNCTION id: 15, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 16, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 17, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 10, nodes: 2 + COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 + CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 7, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 14, nodes: 2 + FUNCTION id: 15, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 16, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 17, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 10, nodes: 2 + COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 + CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 7, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 14, nodes: 2 + COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 + CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 1 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 10, nodes: 2 + COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 + CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 7, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 14, nodes: 2 + COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` FROM column_swap_test_test WHERE b = 0 diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index c81b37c8428..359826c9879 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -13,9 +13,13 @@ INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2); INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference index 0df5c429d9e..518cfb53453 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.reference +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -2,14 +2,115 @@ SELECT i AS i FROM index_append_test_test PREWHERE a = 0 WHERE (a = 0) AND indexHint((i + 40) > 0) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.index_append_test_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: indexHint, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 1 + FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + FUNCTION id: 10, function_name: plus, function_type: ordinary, result_type: Int64 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_40, constant_value_type: UInt8 + CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 15, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 16, nodes: 2 + COLUMN id: 17, column_name: a, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_0, constant_value_type: UInt8 SELECT i AS i FROM index_append_test_test PREWHERE a < 0 +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.index_append_test_test + WHERE + FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: a, result_type: UInt32, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_0, constant_value_type: UInt8 SELECT i AS i FROM index_append_test_test PREWHERE a >= 0 WHERE (a >= 0) AND indexHint((i + 40) > 0) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.index_append_test_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: indexHint, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 1 + FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + FUNCTION id: 10, function_name: plus, function_type: ordinary, result_type: Int64 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_40, constant_value_type: UInt8 + CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 15, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 16, nodes: 2 + COLUMN id: 17, column_name: a, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_0, constant_value_type: UInt8 SELECT i AS i FROM index_append_test_test PREWHERE (2 * b) < 100 WHERE ((2 * b) < 100) AND indexHint(i < 100) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.index_append_test_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: indexHint, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 1 + FUNCTION id: 8, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_100, constant_value_type: UInt8 + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + FUNCTION id: 14, function_name: multiply, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 15, nodes: 2 + CONSTANT id: 16, constant_value: UInt64_2, constant_value_type: UInt8 + COLUMN id: 17, column_name: b, result_type: UInt64, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql index fbffc9c7f10..29a27804200 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.sql +++ b/tests/queries/0_stateless/01625_constraints_index_append.sql @@ -10,8 +10,12 @@ CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2); EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0; +EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0; EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0; +EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0; EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0; +EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0; EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100; +EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100; DROP TABLE index_append_test_test; diff --git a/tests/queries/0_stateless/01626_cnf_test.reference b/tests/queries/0_stateless/01626_cnf_test.reference index 081215c9fb2..846bfd45670 100644 --- a/tests/queries/0_stateless/01626_cnf_test.reference +++ b/tests/queries/0_stateless/01626_cnf_test.reference @@ -1,18 +1,768 @@ SELECT i FROM cnf_test WHERE (i <= 2) AND (i <= 1) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.cnf_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 10, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE (i <= 2) OR (i <= 1) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.cnf_test + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 10, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.cnf_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 8 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 3 + FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 12, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 19, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 3 + FUNCTION id: 22, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 25, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 26, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 27, nodes: 2 + COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 29, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 30, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 31, nodes: 2 + COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 33, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 34, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 35, nodes: 3 + FUNCTION id: 36, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 37, nodes: 2 + COLUMN id: 38, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 39, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 40, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 41, nodes: 2 + COLUMN id: 42, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 43, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 44, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 45, nodes: 2 + COLUMN id: 46, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 47, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 48, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 49, nodes: 3 + FUNCTION id: 50, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 51, nodes: 2 + COLUMN id: 52, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 53, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 54, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 55, nodes: 2 + COLUMN id: 56, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 57, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 58, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 59, nodes: 2 + COLUMN id: 60, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 61, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 62, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 63, nodes: 3 + FUNCTION id: 64, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 65, nodes: 2 + COLUMN id: 66, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 67, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 68, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 69, nodes: 2 + COLUMN id: 70, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 71, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 72, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 73, nodes: 2 + COLUMN id: 74, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 75, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 76, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 77, nodes: 3 + FUNCTION id: 78, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 79, nodes: 2 + COLUMN id: 80, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 81, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 82, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 83, nodes: 2 + COLUMN id: 84, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 85, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 86, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 87, nodes: 2 + COLUMN id: 88, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 89, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 90, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 91, nodes: 3 + FUNCTION id: 92, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 93, nodes: 2 + COLUMN id: 94, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 95, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 96, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 97, nodes: 2 + COLUMN id: 98, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 99, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 100, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 101, nodes: 2 + COLUMN id: 102, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 103, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 104, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 105, nodes: 3 + FUNCTION id: 106, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 107, nodes: 2 + COLUMN id: 108, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 109, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 110, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 111, nodes: 2 + COLUMN id: 112, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 113, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 114, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 115, nodes: 2 + COLUMN id: 116, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 117, constant_value: UInt64_4, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i <= 3) OR (i <= 2) OR (i <= 5)) AND ((i <= 3) OR (i <= 2) OR (i <= 6)) AND ((i <= 3) OR (i <= 5) OR (i <= 1)) AND ((i <= 3) OR (i <= 6) OR (i <= 1)) AND ((i <= 2) OR (i <= 5) OR (i <= 4)) AND ((i <= 2) OR (i <= 6) OR (i <= 4)) AND ((i <= 5) OR (i <= 1) OR (i <= 4)) AND ((i <= 6) OR (i <= 1) OR (i <= 4)) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.cnf_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 8 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 3 + FUNCTION id: 8, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 12, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 19, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 3 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 25, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 26, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 27, nodes: 2 + COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 29, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 30, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 31, nodes: 2 + COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 33, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 34, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 35, nodes: 3 + FUNCTION id: 36, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 37, nodes: 2 + COLUMN id: 38, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 39, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 40, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 41, nodes: 2 + COLUMN id: 42, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 43, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 44, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 45, nodes: 2 + COLUMN id: 46, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 47, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 48, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 49, nodes: 3 + FUNCTION id: 50, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 51, nodes: 2 + COLUMN id: 52, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 53, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 54, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 55, nodes: 2 + COLUMN id: 56, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 57, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 58, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 59, nodes: 2 + COLUMN id: 60, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 61, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 62, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 63, nodes: 3 + FUNCTION id: 64, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 65, nodes: 2 + COLUMN id: 66, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 67, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 68, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 69, nodes: 2 + COLUMN id: 70, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 71, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 72, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 73, nodes: 2 + COLUMN id: 74, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 75, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 76, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 77, nodes: 3 + FUNCTION id: 78, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 79, nodes: 2 + COLUMN id: 80, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 81, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 82, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 83, nodes: 2 + COLUMN id: 84, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 85, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 86, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 87, nodes: 2 + COLUMN id: 88, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 89, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 90, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 91, nodes: 3 + FUNCTION id: 92, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 93, nodes: 2 + COLUMN id: 94, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 95, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 96, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 97, nodes: 2 + COLUMN id: 98, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 99, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 100, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 101, nodes: 2 + COLUMN id: 102, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 103, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 104, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 105, nodes: 3 + FUNCTION id: 106, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 107, nodes: 2 + COLUMN id: 108, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 109, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 110, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 111, nodes: 2 + COLUMN id: 112, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 113, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 114, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 115, nodes: 2 + COLUMN id: 116, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 117, constant_value: UInt64_6, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 5) OR (i > 8)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 8)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 8)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 8)) AND ((i > 5) OR (i > 3) OR (i > 7)) AND ((i > 5) OR (i > 4) OR (i > 7)) AND ((i > 5) OR (i > 8) OR (i > 7)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 6) OR (i > 4) OR (i > 7)) AND ((i > 6) OR (i > 8) OR (i > 7)) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.cnf_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 18 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 3 + FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 12, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 19, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 3 + FUNCTION id: 22, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 25, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 26, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 27, nodes: 2 + COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 29, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 30, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 31, nodes: 2 + COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 33, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 34, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 35, nodes: 3 + FUNCTION id: 36, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 37, nodes: 2 + COLUMN id: 38, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 39, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 40, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 41, nodes: 2 + COLUMN id: 42, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 43, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 44, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 45, nodes: 2 + COLUMN id: 46, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 47, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 48, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 49, nodes: 3 + FUNCTION id: 50, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 51, nodes: 2 + COLUMN id: 52, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 53, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 54, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 55, nodes: 2 + COLUMN id: 56, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 57, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 58, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 59, nodes: 2 + COLUMN id: 60, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 61, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 62, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 63, nodes: 3 + FUNCTION id: 64, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 65, nodes: 2 + COLUMN id: 66, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 67, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 68, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 69, nodes: 2 + COLUMN id: 70, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 71, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 72, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 73, nodes: 2 + COLUMN id: 74, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 75, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 76, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 77, nodes: 3 + FUNCTION id: 78, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 79, nodes: 2 + COLUMN id: 80, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 81, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 82, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 83, nodes: 2 + COLUMN id: 84, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 85, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 86, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 87, nodes: 2 + COLUMN id: 88, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 89, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 90, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 91, nodes: 3 + FUNCTION id: 92, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 93, nodes: 2 + COLUMN id: 94, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 95, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 96, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 97, nodes: 2 + COLUMN id: 98, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 99, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 100, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 101, nodes: 2 + COLUMN id: 102, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 103, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 104, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 105, nodes: 3 + FUNCTION id: 106, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 107, nodes: 2 + COLUMN id: 108, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 109, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 110, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 111, nodes: 2 + COLUMN id: 112, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 113, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 114, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 115, nodes: 2 + COLUMN id: 116, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 117, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 118, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 119, nodes: 3 + FUNCTION id: 120, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 121, nodes: 2 + COLUMN id: 122, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 123, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 124, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 125, nodes: 2 + COLUMN id: 126, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 127, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 128, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 129, nodes: 2 + COLUMN id: 130, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 131, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 132, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 133, nodes: 3 + FUNCTION id: 134, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 135, nodes: 2 + COLUMN id: 136, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 137, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 138, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 139, nodes: 2 + COLUMN id: 140, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 141, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 142, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 143, nodes: 2 + COLUMN id: 144, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 145, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 146, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 147, nodes: 3 + FUNCTION id: 148, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 149, nodes: 2 + COLUMN id: 150, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 151, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 152, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 153, nodes: 2 + COLUMN id: 154, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 155, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 156, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 157, nodes: 2 + COLUMN id: 158, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 159, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 160, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 161, nodes: 3 + FUNCTION id: 162, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 163, nodes: 2 + COLUMN id: 164, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 165, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 166, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 167, nodes: 2 + COLUMN id: 168, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 169, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 170, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 171, nodes: 2 + COLUMN id: 172, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 173, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 174, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 175, nodes: 3 + FUNCTION id: 176, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 177, nodes: 2 + COLUMN id: 178, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 179, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 180, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 181, nodes: 2 + COLUMN id: 182, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 183, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 184, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 185, nodes: 2 + COLUMN id: 186, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 187, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 188, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 189, nodes: 3 + FUNCTION id: 190, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 191, nodes: 2 + COLUMN id: 192, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 193, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 194, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 195, nodes: 2 + COLUMN id: 196, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 197, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 198, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 199, nodes: 2 + COLUMN id: 200, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 201, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 202, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 203, nodes: 3 + FUNCTION id: 204, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 205, nodes: 2 + COLUMN id: 206, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 207, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 208, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 209, nodes: 2 + COLUMN id: 210, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 211, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 212, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 213, nodes: 2 + COLUMN id: 214, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 215, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 216, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 217, nodes: 3 + FUNCTION id: 218, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 219, nodes: 2 + COLUMN id: 220, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 221, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 222, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 223, nodes: 2 + COLUMN id: 224, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 225, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 226, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 227, nodes: 2 + COLUMN id: 228, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 229, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 230, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 231, nodes: 3 + FUNCTION id: 232, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 233, nodes: 2 + COLUMN id: 234, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 235, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 236, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 237, nodes: 2 + COLUMN id: 238, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 239, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 240, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 241, nodes: 2 + COLUMN id: 242, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 243, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 244, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 245, nodes: 3 + FUNCTION id: 246, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 247, nodes: 2 + COLUMN id: 248, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 249, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 250, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 251, nodes: 2 + COLUMN id: 252, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 253, constant_value: UInt64_6, constant_value_type: UInt8 + FUNCTION id: 254, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 255, nodes: 2 + COLUMN id: 256, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 257, constant_value: UInt64_4, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i > 2) OR (i > 1) OR (i > 7)) AND (i <= 5) AND (i <= 6) AND ((i > 3) OR (i > 4) OR (i > 8)) +QUERY id: 0 + PROJECTION COLUMNS + i Int64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.cnf_test + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 4 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 3 + FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 + FUNCTION id: 12, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 19, constant_value: UInt64_7, constant_value_type: UInt8 + FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 3 + FUNCTION id: 22, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 25, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 26, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 27, nodes: 2 + COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 29, constant_value: UInt64_4, constant_value_type: UInt8 + FUNCTION id: 30, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 31, nodes: 2 + COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 33, constant_value: UInt64_8, constant_value_type: UInt8 + FUNCTION id: 34, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 35, nodes: 2 + COLUMN id: 36, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 37, constant_value: UInt64_5, constant_value_type: UInt8 + FUNCTION id: 38, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 39, nodes: 2 + COLUMN id: 40, column_name: i, result_type: Int64, source_id: 3 + CONSTANT id: 41, constant_value: UInt64_6, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/01626_cnf_test.sql b/tests/queries/0_stateless/01626_cnf_test.sql index 8db732bc227..d54b636f748 100644 --- a/tests/queries/0_stateless/01626_cnf_test.sql +++ b/tests/queries/0_stateless/01626_cnf_test.sql @@ -5,14 +5,20 @@ DROP TABLE IF EXISTS cnf_test; CREATE TABLE cnf_test (i Int64) ENGINE = MergeTree() ORDER BY i; EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2)); +EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2)); +EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2)) OR ((i > 3) AND (i > 4)) OR ((i > 5) AND (i > 6)); +EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2)) OR ((i > 3) AND (i > 4)) OR ((i > 5) AND (i > 6)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > 3) OR (i > 4)) AND ((i > 5) OR (i > 6))); +EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > 3) OR (i > 4)) AND ((i > 5) OR (i > 6))); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6)); +EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6)); +EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6)); DROP TABLE cnf_test; From 65e5bfe8207898ca8ea670ec8ef65f89c68ef712 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Mar 2023 17:19:24 +0000 Subject: [PATCH 036/233] Use set --- src/Analyzer/Passes/CNF.cpp | 1 - src/Analyzer/Passes/CNF.h | 20 +------------------ src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 7 +------ 3 files changed, 2 insertions(+), 26 deletions(-) diff --git a/src/Analyzer/Passes/CNF.cpp b/src/Analyzer/Passes/CNF.cpp index 38d316a13f3..68a95a509f8 100644 --- a/src/Analyzer/Passes/CNF.cpp +++ b/src/Analyzer/Passes/CNF.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int TOO_MANY_TEMPORARY_COLUMNS; - extern const int LOGICAL_ERROR; } namespace Analyzer diff --git a/src/Analyzer/Passes/CNF.h b/src/Analyzer/Passes/CNF.h index 8ddbf7efd85..ec639cd6679 100644 --- a/src/Analyzer/Passes/CNF.h +++ b/src/Analyzer/Passes/CNF.h @@ -24,27 +24,9 @@ public: bool operator<(const AtomicFormula & rhs) const; }; - struct SetAtomicFormulaHash - { - size_t operator()(const std::set & or_group) const - { - SipHash hash; - for (const auto & atomic_formula : or_group) - { - SipHash atomic_formula_hash; - atomic_formula_hash.update(atomic_formula.negative); - atomic_formula_hash.update(atomic_formula.node_with_hash.hash); - - hash.update(atomic_formula_hash.get64()); - } - - return hash.get64(); - } - }; - // Different hash is generated for different order, so we use std::set using OrGroup = std::set; - using AndGroup = std::unordered_set; + using AndGroup = std::set; std::string dump() const; diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 781fb857a07..f807b096a78 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -37,7 +37,7 @@ enum class MatchState : uint8_t MatchState match(const Analyzer::CNF::AtomicFormula & a, const Analyzer::CNF::AtomicFormula & b) { using enum MatchState; - if (a.node_with_hash.hash != b.node_with_hash.hash) + if (a.node_with_hash != b.node_with_hash) return NONE; return a.negative == b.negative ? FULL_MATCH : PARTIAL_MATCH; @@ -402,11 +402,6 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - static bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &) - { - return parent->as() == nullptr; - } - void visitImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); From 3210331480139c1a28755cf7b422c03fc83d7ed4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Mar 2023 17:20:21 +0000 Subject: [PATCH 037/233] Remove query tree tests --- ..._constraints_simple_optimization.reference | 58 -- .../01622_constraints_simple_optimization.sql | 4 - ...2_constraints_where_optimization.reference | 52 -- .../01622_constraints_where_optimization.sql | 5 - .../01623_constraints_column_swap.reference | 114 --- .../01623_constraints_column_swap.sql | 8 +- .../01625_constraints_index_append.reference | 101 --- .../01625_constraints_index_append.sql | 4 - .../0_stateless/01626_cnf_test.reference | 750 ------------------ tests/queries/0_stateless/01626_cnf_test.sql | 6 - 10 files changed, 4 insertions(+), 1098 deletions(-) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index f92b8432b65..7e012e1a17b 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -35,69 +35,11 @@ SELECT count() AS `count()` FROM constraint_test_constants WHERE (c > 100) OR (b > 100) -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants - WHERE - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 8, column_name: b, result_type: Int64, source_id: 3 - CONSTANT id: 9, constant_value: UInt64_100, constant_value_type: UInt8 - FUNCTION id: 10, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 11, nodes: 2 - COLUMN id: 12, column_name: c, result_type: Int64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants - WHERE - FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 - CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants - WHERE - FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 - CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 8b7a339984d..7ec9e1a3158 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -98,12 +98,8 @@ SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> -- A AND NOT A EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); DROP TABLE constraint_test_constants; diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index 52aca371a6a..c7c516025f2 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -1,66 +1,14 @@ SELECT count() FROM t_constraints_where WHERE 0 -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where - WHERE - CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE 0 -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where - WHERE - CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE 0 -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where - WHERE - CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE b < 8 -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where - WHERE - FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 - CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 SELECT count() FROM t_constraints_where -QUERY id: 0 - PROJECTION COLUMNS - count() UInt64 - PROJECTION - LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 - JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index 33fa62368b0..1b297942a74 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -8,13 +8,9 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) DROP TABLE t_constraints_where; @@ -23,6 +19,5 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index eceaa7122dc..7ae4516fe9e 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -3,135 +3,21 @@ SELECT (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 1 -QUERY id: 0 - PROJECTION COLUMNS - plus(cityHash64(a), 10) UInt64 - plus(b, 3) UInt64 - PROJECTION - LIST id: 1, nodes: 2 - FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 5, nodes: 1 - COLUMN id: 6, column_name: a, result_type: String, source_id: 7 - CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 - FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 10, nodes: 2 - COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 - CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 - JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test - WHERE - FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 14, nodes: 2 - FUNCTION id: 15, function_name: cityHash64, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 16, nodes: 1 - COLUMN id: 6, column_name: a, result_type: String, source_id: 7 - CONSTANT id: 17, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 0 -QUERY id: 0 - PROJECTION COLUMNS - plus(cityHash64(a), 10) UInt64 - plus(b, 3) UInt64 - PROJECTION - LIST id: 1, nodes: 2 - FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 5, nodes: 1 - COLUMN id: 6, column_name: a, result_type: String, source_id: 7 - CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 - FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 10, nodes: 2 - COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 - CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 - JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test - WHERE - FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 14, nodes: 2 - FUNCTION id: 15, function_name: cityHash64, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 16, nodes: 1 - COLUMN id: 6, column_name: a, result_type: String, source_id: 7 - CONSTANT id: 17, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 0 -QUERY id: 0 - PROJECTION COLUMNS - plus(cityHash64(a), 10) UInt64 - plus(b, 3) UInt64 - PROJECTION - LIST id: 1, nodes: 2 - FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 5, nodes: 1 - COLUMN id: 6, column_name: a, result_type: String, source_id: 7 - CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 - FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 10, nodes: 2 - COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 - CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 - JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test - WHERE - FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 14, nodes: 2 - COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 - CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 1 -QUERY id: 0 - PROJECTION COLUMNS - plus(cityHash64(a), 10) UInt64 - plus(b, 3) UInt64 - PROJECTION - LIST id: 1, nodes: 2 - FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 5, nodes: 1 - COLUMN id: 6, column_name: a, result_type: String, source_id: 7 - CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 - FUNCTION id: 9, function_name: plus, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 10, nodes: 2 - COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 - CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 - JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test - WHERE - FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 14, nodes: 2 - COLUMN id: 11, column_name: b, result_type: UInt64, source_id: 7 - CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` FROM column_swap_test_test WHERE b = 0 diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 359826c9879..873ebbed729 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -13,13 +13,13 @@ INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2); INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; +--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; +--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; +--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; +--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference index 518cfb53453..0df5c429d9e 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.reference +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -2,115 +2,14 @@ SELECT i AS i FROM index_append_test_test PREWHERE a = 0 WHERE (a = 0) AND indexHint((i + 40) > 0) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.index_append_test_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: indexHint, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 1 - FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - FUNCTION id: 10, function_name: plus, function_type: ordinary, result_type: Int64 - ARGUMENTS - LIST id: 11, nodes: 2 - COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_40, constant_value_type: UInt8 - CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 - FUNCTION id: 15, function_name: equals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 16, nodes: 2 - COLUMN id: 17, column_name: a, result_type: UInt32, source_id: 3 - CONSTANT id: 18, constant_value: UInt64_0, constant_value_type: UInt8 SELECT i AS i FROM index_append_test_test PREWHERE a < 0 -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.index_append_test_test - WHERE - FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - COLUMN id: 6, column_name: a, result_type: UInt32, source_id: 3 - CONSTANT id: 7, constant_value: UInt64_0, constant_value_type: UInt8 SELECT i AS i FROM index_append_test_test PREWHERE a >= 0 WHERE (a >= 0) AND indexHint((i + 40) > 0) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.index_append_test_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: indexHint, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 1 - FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - FUNCTION id: 10, function_name: plus, function_type: ordinary, result_type: Int64 - ARGUMENTS - LIST id: 11, nodes: 2 - COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_40, constant_value_type: UInt8 - CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 - FUNCTION id: 15, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 16, nodes: 2 - COLUMN id: 17, column_name: a, result_type: UInt32, source_id: 3 - CONSTANT id: 18, constant_value: UInt64_0, constant_value_type: UInt8 SELECT i AS i FROM index_append_test_test PREWHERE (2 * b) < 100 WHERE ((2 * b) < 100) AND indexHint(i < 100) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.index_append_test_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: indexHint, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 1 - FUNCTION id: 8, function_name: less, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 11, constant_value: UInt64_100, constant_value_type: UInt8 - FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 13, nodes: 2 - FUNCTION id: 14, function_name: multiply, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 15, nodes: 2 - CONSTANT id: 16, constant_value: UInt64_2, constant_value_type: UInt8 - COLUMN id: 17, column_name: b, result_type: UInt64, source_id: 3 - CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql index 29a27804200..fbffc9c7f10 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.sql +++ b/tests/queries/0_stateless/01625_constraints_index_append.sql @@ -10,12 +10,8 @@ CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2); EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0; -EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0; EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0; -EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0; EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0; -EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0; EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100; -EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100; DROP TABLE index_append_test_test; diff --git a/tests/queries/0_stateless/01626_cnf_test.reference b/tests/queries/0_stateless/01626_cnf_test.reference index 846bfd45670..081215c9fb2 100644 --- a/tests/queries/0_stateless/01626_cnf_test.reference +++ b/tests/queries/0_stateless/01626_cnf_test.reference @@ -1,768 +1,18 @@ SELECT i FROM cnf_test WHERE (i <= 2) AND (i <= 1) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.cnf_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 8, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 10, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 11, nodes: 2 - COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE (i <= 2) OR (i <= 1) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.cnf_test - WHERE - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 8, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 10, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 11, nodes: 2 - COLUMN id: 12, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.cnf_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 8 - FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 3 - FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 11, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 12, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 16, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 19, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 21, nodes: 3 - FUNCTION id: 22, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 25, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 26, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 27, nodes: 2 - COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 29, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 30, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 31, nodes: 2 - COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 33, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 34, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 35, nodes: 3 - FUNCTION id: 36, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 37, nodes: 2 - COLUMN id: 38, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 39, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 40, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 41, nodes: 2 - COLUMN id: 42, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 43, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 44, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 45, nodes: 2 - COLUMN id: 46, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 47, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 48, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 49, nodes: 3 - FUNCTION id: 50, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 51, nodes: 2 - COLUMN id: 52, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 53, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 54, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 55, nodes: 2 - COLUMN id: 56, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 57, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 58, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 59, nodes: 2 - COLUMN id: 60, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 61, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 62, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 63, nodes: 3 - FUNCTION id: 64, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 65, nodes: 2 - COLUMN id: 66, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 67, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 68, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 69, nodes: 2 - COLUMN id: 70, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 71, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 72, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 73, nodes: 2 - COLUMN id: 74, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 75, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 76, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 77, nodes: 3 - FUNCTION id: 78, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 79, nodes: 2 - COLUMN id: 80, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 81, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 82, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 83, nodes: 2 - COLUMN id: 84, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 85, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 86, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 87, nodes: 2 - COLUMN id: 88, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 89, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 90, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 91, nodes: 3 - FUNCTION id: 92, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 93, nodes: 2 - COLUMN id: 94, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 95, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 96, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 97, nodes: 2 - COLUMN id: 98, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 99, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 100, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 101, nodes: 2 - COLUMN id: 102, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 103, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 104, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 105, nodes: 3 - FUNCTION id: 106, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 107, nodes: 2 - COLUMN id: 108, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 109, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 110, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 111, nodes: 2 - COLUMN id: 112, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 113, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 114, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 115, nodes: 2 - COLUMN id: 116, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 117, constant_value: UInt64_4, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i <= 3) OR (i <= 2) OR (i <= 5)) AND ((i <= 3) OR (i <= 2) OR (i <= 6)) AND ((i <= 3) OR (i <= 5) OR (i <= 1)) AND ((i <= 3) OR (i <= 6) OR (i <= 1)) AND ((i <= 2) OR (i <= 5) OR (i <= 4)) AND ((i <= 2) OR (i <= 6) OR (i <= 4)) AND ((i <= 5) OR (i <= 1) OR (i <= 4)) AND ((i <= 6) OR (i <= 1) OR (i <= 4)) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.cnf_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 8 - FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 3 - FUNCTION id: 8, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 11, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 12, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 15, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 19, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 21, nodes: 3 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 25, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 26, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 27, nodes: 2 - COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 29, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 30, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 31, nodes: 2 - COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 33, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 34, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 35, nodes: 3 - FUNCTION id: 36, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 37, nodes: 2 - COLUMN id: 38, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 39, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 40, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 41, nodes: 2 - COLUMN id: 42, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 43, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 44, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 45, nodes: 2 - COLUMN id: 46, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 47, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 48, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 49, nodes: 3 - FUNCTION id: 50, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 51, nodes: 2 - COLUMN id: 52, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 53, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 54, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 55, nodes: 2 - COLUMN id: 56, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 57, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 58, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 59, nodes: 2 - COLUMN id: 60, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 61, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 62, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 63, nodes: 3 - FUNCTION id: 64, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 65, nodes: 2 - COLUMN id: 66, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 67, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 68, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 69, nodes: 2 - COLUMN id: 70, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 71, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 72, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 73, nodes: 2 - COLUMN id: 74, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 75, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 76, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 77, nodes: 3 - FUNCTION id: 78, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 79, nodes: 2 - COLUMN id: 80, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 81, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 82, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 83, nodes: 2 - COLUMN id: 84, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 85, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 86, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 87, nodes: 2 - COLUMN id: 88, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 89, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 90, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 91, nodes: 3 - FUNCTION id: 92, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 93, nodes: 2 - COLUMN id: 94, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 95, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 96, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 97, nodes: 2 - COLUMN id: 98, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 99, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 100, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 101, nodes: 2 - COLUMN id: 102, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 103, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 104, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 105, nodes: 3 - FUNCTION id: 106, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 107, nodes: 2 - COLUMN id: 108, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 109, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 110, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 111, nodes: 2 - COLUMN id: 112, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 113, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 114, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 115, nodes: 2 - COLUMN id: 116, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 117, constant_value: UInt64_6, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 5) OR (i > 8)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 8)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 8)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 8)) AND ((i > 5) OR (i > 3) OR (i > 7)) AND ((i > 5) OR (i > 4) OR (i > 7)) AND ((i > 5) OR (i > 8) OR (i > 7)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 6) OR (i > 4) OR (i > 7)) AND ((i > 6) OR (i > 8) OR (i > 7)) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.cnf_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 18 - FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 3 - FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 11, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 12, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 16, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 19, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 21, nodes: 3 - FUNCTION id: 22, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 25, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 26, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 27, nodes: 2 - COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 29, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 30, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 31, nodes: 2 - COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 33, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 34, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 35, nodes: 3 - FUNCTION id: 36, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 37, nodes: 2 - COLUMN id: 38, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 39, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 40, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 41, nodes: 2 - COLUMN id: 42, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 43, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 44, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 45, nodes: 2 - COLUMN id: 46, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 47, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 48, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 49, nodes: 3 - FUNCTION id: 50, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 51, nodes: 2 - COLUMN id: 52, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 53, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 54, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 55, nodes: 2 - COLUMN id: 56, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 57, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 58, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 59, nodes: 2 - COLUMN id: 60, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 61, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 62, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 63, nodes: 3 - FUNCTION id: 64, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 65, nodes: 2 - COLUMN id: 66, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 67, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 68, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 69, nodes: 2 - COLUMN id: 70, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 71, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 72, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 73, nodes: 2 - COLUMN id: 74, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 75, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 76, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 77, nodes: 3 - FUNCTION id: 78, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 79, nodes: 2 - COLUMN id: 80, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 81, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 82, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 83, nodes: 2 - COLUMN id: 84, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 85, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 86, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 87, nodes: 2 - COLUMN id: 88, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 89, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 90, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 91, nodes: 3 - FUNCTION id: 92, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 93, nodes: 2 - COLUMN id: 94, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 95, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 96, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 97, nodes: 2 - COLUMN id: 98, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 99, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 100, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 101, nodes: 2 - COLUMN id: 102, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 103, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 104, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 105, nodes: 3 - FUNCTION id: 106, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 107, nodes: 2 - COLUMN id: 108, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 109, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 110, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 111, nodes: 2 - COLUMN id: 112, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 113, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 114, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 115, nodes: 2 - COLUMN id: 116, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 117, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 118, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 119, nodes: 3 - FUNCTION id: 120, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 121, nodes: 2 - COLUMN id: 122, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 123, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 124, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 125, nodes: 2 - COLUMN id: 126, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 127, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 128, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 129, nodes: 2 - COLUMN id: 130, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 131, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 132, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 133, nodes: 3 - FUNCTION id: 134, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 135, nodes: 2 - COLUMN id: 136, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 137, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 138, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 139, nodes: 2 - COLUMN id: 140, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 141, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 142, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 143, nodes: 2 - COLUMN id: 144, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 145, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 146, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 147, nodes: 3 - FUNCTION id: 148, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 149, nodes: 2 - COLUMN id: 150, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 151, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 152, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 153, nodes: 2 - COLUMN id: 154, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 155, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 156, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 157, nodes: 2 - COLUMN id: 158, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 159, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 160, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 161, nodes: 3 - FUNCTION id: 162, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 163, nodes: 2 - COLUMN id: 164, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 165, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 166, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 167, nodes: 2 - COLUMN id: 168, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 169, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 170, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 171, nodes: 2 - COLUMN id: 172, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 173, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 174, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 175, nodes: 3 - FUNCTION id: 176, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 177, nodes: 2 - COLUMN id: 178, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 179, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 180, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 181, nodes: 2 - COLUMN id: 182, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 183, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 184, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 185, nodes: 2 - COLUMN id: 186, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 187, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 188, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 189, nodes: 3 - FUNCTION id: 190, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 191, nodes: 2 - COLUMN id: 192, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 193, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 194, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 195, nodes: 2 - COLUMN id: 196, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 197, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 198, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 199, nodes: 2 - COLUMN id: 200, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 201, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 202, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 203, nodes: 3 - FUNCTION id: 204, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 205, nodes: 2 - COLUMN id: 206, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 207, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 208, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 209, nodes: 2 - COLUMN id: 210, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 211, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 212, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 213, nodes: 2 - COLUMN id: 214, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 215, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 216, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 217, nodes: 3 - FUNCTION id: 218, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 219, nodes: 2 - COLUMN id: 220, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 221, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 222, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 223, nodes: 2 - COLUMN id: 224, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 225, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 226, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 227, nodes: 2 - COLUMN id: 228, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 229, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 230, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 231, nodes: 3 - FUNCTION id: 232, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 233, nodes: 2 - COLUMN id: 234, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 235, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 236, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 237, nodes: 2 - COLUMN id: 238, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 239, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 240, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 241, nodes: 2 - COLUMN id: 242, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 243, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 244, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 245, nodes: 3 - FUNCTION id: 246, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 247, nodes: 2 - COLUMN id: 248, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 249, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 250, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 251, nodes: 2 - COLUMN id: 252, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 253, constant_value: UInt64_6, constant_value_type: UInt8 - FUNCTION id: 254, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 255, nodes: 2 - COLUMN id: 256, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 257, constant_value: UInt64_4, constant_value_type: UInt8 SELECT i FROM cnf_test WHERE ((i > 2) OR (i > 1) OR (i > 7)) AND (i <= 5) AND (i <= 6) AND ((i > 3) OR (i > 4) OR (i > 8)) -QUERY id: 0 - PROJECTION COLUMNS - i Int64 - PROJECTION - LIST id: 1, nodes: 1 - COLUMN id: 2, column_name: i, result_type: Int64, source_id: 3 - JOIN TREE - TABLE id: 3, table_name: default.cnf_test - WHERE - FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 5, nodes: 4 - FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 3 - FUNCTION id: 8, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 9, nodes: 2 - COLUMN id: 10, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 - FUNCTION id: 12, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 16, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 19, constant_value: UInt64_7, constant_value_type: UInt8 - FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 21, nodes: 3 - FUNCTION id: 22, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 24, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 25, constant_value: UInt64_3, constant_value_type: UInt8 - FUNCTION id: 26, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 27, nodes: 2 - COLUMN id: 28, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 29, constant_value: UInt64_4, constant_value_type: UInt8 - FUNCTION id: 30, function_name: greater, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 31, nodes: 2 - COLUMN id: 32, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 33, constant_value: UInt64_8, constant_value_type: UInt8 - FUNCTION id: 34, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 35, nodes: 2 - COLUMN id: 36, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 37, constant_value: UInt64_5, constant_value_type: UInt8 - FUNCTION id: 38, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 39, nodes: 2 - COLUMN id: 40, column_name: i, result_type: Int64, source_id: 3 - CONSTANT id: 41, constant_value: UInt64_6, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/01626_cnf_test.sql b/tests/queries/0_stateless/01626_cnf_test.sql index d54b636f748..8db732bc227 100644 --- a/tests/queries/0_stateless/01626_cnf_test.sql +++ b/tests/queries/0_stateless/01626_cnf_test.sql @@ -5,20 +5,14 @@ DROP TABLE IF EXISTS cnf_test; CREATE TABLE cnf_test (i Int64) ENGINE = MergeTree() ORDER BY i; EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2)); -EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2)); -EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2)) OR ((i > 3) AND (i > 4)) OR ((i > 5) AND (i > 6)); -EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2)) OR ((i > 3) AND (i > 4)) OR ((i > 5) AND (i > 6)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > 3) OR (i > 4)) AND ((i > 5) OR (i > 6))); -EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > 3) OR (i > 4)) AND ((i > 5) OR (i > 6))); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6)); -EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6)); -EXPLAIN QUERY TREE SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6)); DROP TABLE cnf_test; From 45512333392114649a77bd6dccf26d331397a355 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 16 Mar 2023 20:50:21 +0000 Subject: [PATCH 038/233] Remove leftovers --- src/Processors/QueryPlan/ExpressionStep.cpp | 2 -- src/Processors/QueryPlan/FilterStep.cpp | 2 -- .../QueryPlan/updateDataStreams.cpp | 30 ------------------- 3 files changed, 34 deletions(-) delete mode 100644 src/Processors/QueryPlan/updateDataStreams.cpp diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 22216cbfd27..8a1e10f0643 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -80,8 +80,6 @@ void ExpressionStep::updateOutputStream() return; const ActionsDAGPtr & actions = actions_dag; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "ActionsDAG dump:\n{}", actions->dumpDAG()); - const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index afca6903623..d63fded918d 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -110,8 +110,6 @@ void FilterStep::updateOutputStream() return; const ActionsDAGPtr & actions = actions_dag; - // LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "ActionsDAG dump:\n{}", actions->dumpDAG()); - const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/updateDataStreams.cpp b/src/Processors/QueryPlan/updateDataStreams.cpp deleted file mode 100644 index 6e4c9570143..00000000000 --- a/src/Processors/QueryPlan/updateDataStreams.cpp +++ /dev/null @@ -1,30 +0,0 @@ -#include -#include "Processors/QueryPlan/ITransformingStep.h" - -namespace DB -{ - -// constexpr bool debug_logging_enabled = false; - -// class UpdateDataStreams : public QueryPlanVisitor -// { -// public: -// explicit UpdateDataStreams(QueryPlan::Node * root_) : QueryPlanVisitor(root_) { } - -// static bool visitTopDownImpl(QueryPlan::Node * /*current_node*/, QueryPlan::Node * /*parent_node*/) -// { -// return true; -// } - -// static void visitBottomUpImpl(QueryPlan::Node * current_node, QueryPlan::Node * parent_node) -// { -// if (parent_node->children.size() != 1) -// return; - -// chassert(current_node->step->hasOutputStream()); - -// if (auto * parent_transform_step = dynamic_cast(parent_node->step.get()); parent_transform_step) -// parent_transform_step->updateInputStream(current_node->step->getOutputStream()); -// } -// }; -} From d6efe7fc212c73b865d18352fed44b325dd2a5c1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Mar 2023 08:08:07 +0000 Subject: [PATCH 039/233] Fix --- .../tests/gtest_comparison_graph.cpp | 48 ++++++++--------- ..._constraints_simple_optimization.reference | 36 +++++++++++++ .../01622_constraints_simple_optimization.sql | 4 ++ ...2_constraints_where_optimization.reference | 52 +++++++++++++++++++ .../01622_constraints_where_optimization.sql | 5 ++ .../01625_constraints_index_append.reference | 8 +++ .../01625_constraints_index_append.sh | 35 +++++++++++++ .../01625_constraints_index_append.sql | 17 ------ 8 files changed, 164 insertions(+), 41 deletions(-) create mode 100755 tests/queries/0_stateless/01625_constraints_index_append.sh delete mode 100644 tests/queries/0_stateless/01625_constraints_index_append.sql diff --git a/src/Interpreters/tests/gtest_comparison_graph.cpp b/src/Interpreters/tests/gtest_comparison_graph.cpp index 72e72b4b802..32dcb3de6b3 100644 --- a/src/Interpreters/tests/gtest_comparison_graph.cpp +++ b/src/Interpreters/tests/gtest_comparison_graph.cpp @@ -9,11 +9,11 @@ using namespace DB; -static ComparisonGraph getGraph(const String & query) +static ComparisonGraph<> getGraph(const String & query) { ParserExpressionList parser(false); ASTPtr ast = parseQuery(parser, query, 0, 0); - return ComparisonGraph(ast->children); + return ComparisonGraph<>(ast->children); } TEST(ComparisonGraph, Bounds) @@ -47,8 +47,8 @@ TEST(ComparisonGraph, Bounds) auto x = std::make_shared("x"); auto y = std::make_shared("y"); - ASSERT_EQ(graph.compare(x, y), ComparisonGraph::CompareResult::LESS); - ASSERT_EQ(graph.compare(y, x), ComparisonGraph::CompareResult::GREATER); + ASSERT_EQ(graph.compare(x, y), ComparisonGraphCompareResult::LESS); + ASSERT_EQ(graph.compare(y, x), ComparisonGraphCompareResult::GREATER); } } @@ -93,7 +93,7 @@ TEST(ComparisonGraph, Components) TEST(ComparisonGraph, Compare) { - using CompareResult = ComparisonGraph::CompareResult; + using enum ComparisonGraphCompareResult; { String query = "a >= b, c >= b"; @@ -102,7 +102,7 @@ TEST(ComparisonGraph, Compare) auto a = std::make_shared("a"); auto c = std::make_shared("c"); - ASSERT_EQ(graph.compare(a, c), CompareResult::UNKNOWN); + ASSERT_EQ(graph.compare(a, c), UNKNOWN); } { @@ -113,9 +113,9 @@ TEST(ComparisonGraph, Compare) auto b = std::make_shared("b"); auto c = std::make_shared("c"); - ASSERT_EQ(graph.compare(a, c), CompareResult::GREATER); - ASSERT_EQ(graph.compare(a, b), CompareResult::GREATER_OR_EQUAL); - ASSERT_EQ(graph.compare(b, c), CompareResult::GREATER); + ASSERT_EQ(graph.compare(a, c), GREATER); + ASSERT_EQ(graph.compare(a, b), GREATER_OR_EQUAL); + ASSERT_EQ(graph.compare(b, c), GREATER); } { @@ -126,9 +126,9 @@ TEST(ComparisonGraph, Compare) auto b = std::make_shared("b"); auto c = std::make_shared("c"); - ASSERT_EQ(graph.compare(a, b), CompareResult::NOT_EQUAL); - ASSERT_EQ(graph.compare(a, c), CompareResult::GREATER); - ASSERT_EQ(graph.compare(b, c), CompareResult::UNKNOWN); + ASSERT_EQ(graph.compare(a, b), NOT_EQUAL); + ASSERT_EQ(graph.compare(a, c), GREATER); + ASSERT_EQ(graph.compare(b, c), UNKNOWN); } { @@ -154,17 +154,17 @@ TEST(ComparisonGraph, Compare) auto lit_3 = std::make_shared(3u); auto lit_4 = std::make_shared(4u); - ASSERT_EQ(graph.compare(lit_3, a), CompareResult::LESS_OR_EQUAL); - ASSERT_FALSE(graph.isAlwaysCompare(CompareResult::LESS, lit_3, a)); - ASSERT_TRUE(graph.isAlwaysCompare(CompareResult::LESS, lit_2, a)); + ASSERT_EQ(graph.compare(lit_3, a), LESS_OR_EQUAL); + ASSERT_FALSE(graph.isAlwaysCompare(LESS, lit_3, a)); + ASSERT_TRUE(graph.isAlwaysCompare(LESS, lit_2, a)); - ASSERT_EQ(graph.compare(b, lit_2), CompareResult::GREATER); - ASSERT_EQ(graph.compare(b, lit_3), CompareResult::GREATER); - ASSERT_EQ(graph.compare(b, lit_4), CompareResult::UNKNOWN); + ASSERT_EQ(graph.compare(b, lit_2), GREATER); + ASSERT_EQ(graph.compare(b, lit_3), GREATER); + ASSERT_EQ(graph.compare(b, lit_4), UNKNOWN); - ASSERT_EQ(graph.compare(d, lit_2), CompareResult::GREATER); - ASSERT_EQ(graph.compare(d, lit_3), CompareResult::GREATER_OR_EQUAL); - ASSERT_EQ(graph.compare(d, lit_4), CompareResult::UNKNOWN); + ASSERT_EQ(graph.compare(d, lit_2), GREATER); + ASSERT_EQ(graph.compare(d, lit_3), GREATER_OR_EQUAL); + ASSERT_EQ(graph.compare(d, lit_4), UNKNOWN); } { @@ -176,8 +176,8 @@ TEST(ComparisonGraph, Compare) auto lit_3 = std::make_shared(3); auto lit_15 = std::make_shared(15); - ASSERT_EQ(graph.compare(a, lit_8), CompareResult::UNKNOWN); - ASSERT_EQ(graph.compare(a, lit_3), CompareResult::GREATER); - ASSERT_EQ(graph.compare(a, lit_15), CompareResult::LESS); + ASSERT_EQ(graph.compare(a, lit_8), UNKNOWN); + ASSERT_EQ(graph.compare(a, lit_3), GREATER); + ASSERT_EQ(graph.compare(a, lit_15), LESS); } } diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 7e012e1a17b..529351180b3 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -38,8 +38,44 @@ WHERE (c > 100) OR (b > 100) SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants + WHERE + FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants + WHERE + FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SELECT count() AS `count()` FROM constraint_test_constants +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.constraint_test_constants diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 7ec9e1a3158..21d75a48587 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -98,8 +98,12 @@ SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> -- A AND NOT A EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); +-- EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); ---> the order of the generated checks is not consistent EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); DROP TABLE constraint_test_constants; diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index c7c516025f2..52aca371a6a 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -1,14 +1,66 @@ SELECT count() FROM t_constraints_where WHERE 0 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE 0 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE 0 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SELECT count() FROM t_constraints_where WHERE b < 8 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + WHERE + FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 SELECT count() FROM t_constraints_where +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index 1b297942a74..33fa62368b0 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -8,9 +8,13 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) DROP TABLE t_constraints_where; @@ -19,5 +23,6 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference index 0df5c429d9e..591d8a85897 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.reference +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -2,14 +2,22 @@ SELECT i AS i FROM index_append_test_test PREWHERE a = 0 WHERE (a = 0) AND indexHint((i + 40) > 0) +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1 +1 SELECT i AS i FROM index_append_test_test PREWHERE a < 0 +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1 +0 SELECT i AS i FROM index_append_test_test PREWHERE a >= 0 WHERE (a >= 0) AND indexHint((i + 40) > 0) +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1 +1 SELECT i AS i FROM index_append_test_test PREWHERE (2 * b) < 100 WHERE ((2 * b) < 100) AND indexHint(i < 100) +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1 +1 diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sh b/tests/queries/0_stateless/01625_constraints_index_append.sh new file mode 100755 index 00000000000..f17ea422409 --- /dev/null +++ b/tests/queries/0_stateless/01625_constraints_index_append.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS index_append_test_test;" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i;" +$CLICKHOUSE_CLIENT --query "INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2);" + +function run_with_settings() +{ + query="$1 SETTINGS convert_query_to_cnf = 1\ + , optimize_using_constraints = 1\ + , optimize_move_to_prewhere = 1\ + , optimize_substitute_columns = 1\ + , optimize_append_index = 1" + + $CLICKHOUSE_CLIENT --query="$query" + +} + +run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0" | grep -Fac "indexHint" +run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0" | grep -Fac "indexHint" +run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0" | grep -Fac "indexHint" +run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100" | grep -Fac "indexHint" + +$CLICKHOUSE_CLIENT --query "DROP TABLE index_append_test_test;" diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql deleted file mode 100644 index fbffc9c7f10..00000000000 --- a/tests/queries/0_stateless/01625_constraints_index_append.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET convert_query_to_cnf = 1; -SET optimize_using_constraints = 1; -SET optimize_move_to_prewhere = 1; -SET optimize_substitute_columns = 1; -SET optimize_append_index = 1; - -DROP TABLE IF EXISTS index_append_test_test; - -CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i; -INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2); - -EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0; -EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0; -EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0; -EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100; - -DROP TABLE index_append_test_test; From 0fb9f9ffe6018558cbbc26818b272dc191901d00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Mar 2023 13:38:01 +0000 Subject: [PATCH 040/233] Add support for substitute column --- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 307 +++++++++++++++++- .../AddIndexConstraintsOptimizer.cpp | 2 +- src/Interpreters/ComparisonGraph.h | 2 +- .../SubstituteColumnOptimizer.cpp | 6 +- .../WhereConstraintsOptimizer.cpp | 12 +- .../tests/gtest_comparison_graph.cpp | 4 +- src/Storages/ConstraintsDescription.cpp | 8 +- src/Storages/ConstraintsDescription.h | 6 +- ...ergeTreeIndexHypothesisMergedCondition.cpp | 12 +- .../MergeTreeIndexHypothesisMergedCondition.h | 6 +- .../01623_constraints_column_swap.reference | 236 ++++++++++++++ .../01623_constraints_column_swap.sql | 16 +- 12 files changed, 580 insertions(+), 37 deletions(-) diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index f807b096a78..591f2a1557c 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -9,7 +9,13 @@ #include #include +#include + #include +#include "Analyzer/HashUtils.h" +#include "Analyzer/IQueryTreeNode.h" +#include "Interpreters/ComparisonGraph.h" +#include "base/types.h" namespace DB { @@ -344,6 +350,286 @@ void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expres } } +struct ColumnPrice +{ + Int64 compressed_size{0}; + Int64 uncompressed_size{0}; + + ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_) + : compressed_size(compressed_size_) + , uncompressed_size(uncompressed_size_) + { + } + + bool operator<(const ColumnPrice & that) const + { + return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size); + } + + ColumnPrice & operator+=(const ColumnPrice & that) + { + compressed_size += that.compressed_size; + uncompressed_size += that.uncompressed_size; + return *this; + } + + ColumnPrice & operator-=(const ColumnPrice & that) + { + compressed_size -= that.compressed_size; + uncompressed_size -= that.uncompressed_size; + return *this; + } +}; + +using ColumnPriceByName = std::unordered_map; +using ColumnPriceByQueryNode = QueryTreeNodePtrWithHashMap; + +class ComponentCollectorVisitor : public ConstInDepthQueryTreeVisitor +{ +public: + ComponentCollectorVisitor( + std::set & components_, + QueryTreeNodePtrWithHashMap & query_node_to_component_, + const ComparisonGraph & graph_) + : components(components_), query_node_to_component(query_node_to_component_), graph(graph_) + {} + + void visitImpl(const QueryTreeNodePtr & node) + { + if (auto id = graph.getComponentId(node)) + { + query_node_to_component.emplace(node, *id); + components.insert(*id); + } + } + +private: + std::set & components; + QueryTreeNodePtrWithHashMap & query_node_to_component; + + const ComparisonGraph & graph; +}; + +class ColumnNameCollectorVisitor : public ConstInDepthQueryTreeVisitor +{ +public: + ColumnNameCollectorVisitor( + std::unordered_set & column_names_, + const QueryTreeNodePtrWithHashMap & query_node_to_component_) + : column_names(column_names_), query_node_to_component(query_node_to_component_) + {} + + bool needChildVisit(const VisitQueryTreeNodeType & parent, const VisitQueryTreeNodeType &) + { + return !query_node_to_component.contains(parent); + } + + void visitImpl(const QueryTreeNodePtr & node) + { + if (query_node_to_component.contains(node)) + return; + + if (const auto * column_node = node->as()) + column_names.insert(column_node->getColumnName()); + } + +private: + std::unordered_set & column_names; + const QueryTreeNodePtrWithHashMap & query_node_to_component; +}; + +class SubstituteColumnVisitor : public InDepthQueryTreeVisitor +{ +public: + SubstituteColumnVisitor( + const QueryTreeNodePtrWithHashMap & query_node_to_component_, + const std::unordered_map & id_to_query_node_map_, + ContextPtr context_) + : query_node_to_component(query_node_to_component_), id_to_query_node_map(id_to_query_node_map_), context(std::move(context_)) + {} + + void visitImpl(QueryTreeNodePtr & node) + { + auto component_id_it = query_node_to_component.find(node); + if (component_id_it == query_node_to_component.end()) + return; + + const auto component_id = component_id_it->second; + auto new_node = id_to_query_node_map.at(component_id)->clone(); + + if (!node->getResultType()->equals(*new_node->getResultType())) + { + node = buildCastFunction(new_node, node->getResultType(), context); + return; + } + + node = std::move(new_node); + } + +private: + const QueryTreeNodePtrWithHashMap & query_node_to_component; + const std::unordered_map & id_to_query_node_map; + ContextPtr context; +}; + +ColumnPrice calculatePrice( + const ColumnPriceByName & column_prices, + const std::unordered_set & column_names) +{ + ColumnPrice result(0, 0); + + for (const auto & column : column_names) + { + if (auto it = column_prices.find(column); it != column_prices.end()) + result += it->second; + } + + return result; +} + + +void bruteForce( + const ComparisonGraph & graph, + const std::vector & components, + size_t current_component, + const ColumnPriceByName & column_prices, + ColumnPrice current_price, + std::vector & expressions_stack, + ColumnPrice & min_price, + std::vector & min_expressions) +{ + if (current_component == components.size()) + { + if (current_price < min_price) + { + min_price = current_price; + min_expressions = expressions_stack; + } + return; + } + + for (const auto & node : graph.getComponent(components[current_component])) + { + std::unordered_set column_names; + ColumnNameCollectorVisitor column_name_collector{column_names, {}}; + column_name_collector.visit(node); + + ColumnPrice expression_price = calculatePrice(column_prices, column_names); + + expressions_stack.push_back(node); + current_price += expression_price; + + ColumnPriceByName new_prices(column_prices); + for (const auto & column : column_names) + new_prices.insert_or_assign(column, ColumnPrice(0, 0)); + + bruteForce(graph, + components, + current_component + 1, + new_prices, + current_price, + expressions_stack, + min_price, + min_expressions); + + current_price -= expression_price; + expressions_stack.pop_back(); + } +} + +void substituteColumns(QueryNode & query_node, const QueryTreeNodes & table_expressions, const ContextPtr & context) +{ + static constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024; + static constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000; + + for (const auto & table_expression : table_expressions) + { + auto snapshot = getStorageSnapshot(table_expression); + if (!snapshot || !snapshot->metadata) + continue; + + const auto column_sizes = snapshot->storage.getColumnSizes(); + if (column_sizes.empty()) + return; + + auto query_tree_constraint = snapshot->metadata->getConstraints().getQueryTreeData(context, table_expression); + const auto & graph = query_tree_constraint.getGraph(); + + auto run_for_all = [&](const auto function) + { + function(query_node.getProjectionNode()); + + if (query_node.hasWhere()) + function(query_node.getWhere()); + + if (query_node.hasPrewhere()) + function(query_node.getPrewhere()); + + if (query_node.hasHaving()) + function(query_node.getHaving()); + }; + + std::set components; + QueryTreeNodePtrWithHashMap query_node_to_component; + std::unordered_set column_names; + + run_for_all([&](QueryTreeNodePtr & node) + { + ComponentCollectorVisitor component_collector{components, query_node_to_component, graph}; + component_collector.visit(node); + ColumnNameCollectorVisitor column_name_collector{column_names, query_node_to_component}; + column_name_collector.visit(node); + }); + + ColumnPriceByName column_prices; + const auto primary_key = snapshot->metadata->getColumnsRequiredForPrimaryKey(); + + for (const auto & [column_name, column_size] : column_sizes) + column_prices.insert_or_assign(column_name, ColumnPrice(column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed)); + + for (const auto & column_name : primary_key) + column_prices.insert_or_assign(column_name, ColumnPrice(INDEX_PRICE, INDEX_PRICE)); + + for (const auto & column_name : column_names) + column_prices.insert_or_assign(column_name, ColumnPrice(0, 0)); + + std::unordered_map id_to_query_node_map; + std::vector components_list; + + for (const auto component_id : components) + { + auto component = graph.getComponent(component_id); + if (component.size() == 1) + id_to_query_node_map[component_id] = component.front(); + else + components_list.push_back(component_id); + } + + std::vector expressions_stack; + ColumnPrice min_price(std::numeric_limits::max(), std::numeric_limits::max()); + std::vector min_expressions; + + bruteForce(graph, + components_list, + 0, + column_prices, + ColumnPrice(0, 0), + expressions_stack, + min_price, + min_expressions); + + for (size_t i = 0; i < components_list.size(); ++i) + id_to_query_node_map[components_list[i]] = min_expressions[i]; + + SubstituteColumnVisitor substitute_column{query_node_to_component, id_to_query_node_map, context}; + + run_for_all([&](QueryTreeNodePtr & node) + { + substitute_column.visit(node); + }); + } +} + void optimizeWithConstraints(Analyzer::CNF & cnf, const QueryTreeNodes & table_expressions, const ContextPtr & context) { cnf.pullNotOutFunctions(context); @@ -410,11 +696,24 @@ public: auto table_expressions = extractTableExpressions(query_node->getJoinTree()); - if (query_node->hasWhere()) - optimizeNode(query_node->getWhere(), table_expressions, getContext()); + const auto & context = getContext(); + const auto & settings = context->getSettingsRef(); - if (query_node->hasPrewhere()) - optimizeNode(query_node->getPrewhere(), table_expressions, getContext()); + bool has_filter = false; + const auto optimize_filter = [&](QueryTreeNodePtr & filter_node) + { + if (filter_node == nullptr) + return; + + optimizeNode(query_node->getWhere(), table_expressions, context); + has_filter = true; + }; + + optimize_filter(query_node->getWhere()); + optimize_filter(query_node->getPrewhere()); + + if (has_filter && settings.optimize_substitute_columns) + substituteColumns(*query_node, table_expressions, context); } }; diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp index 15adc737f6f..b9d56ec2ff8 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.cpp +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -104,7 +104,7 @@ namespace /// we can add to expression 'indexHint(I < A)' condition. CNFQuery::OrGroup createIndexHintGroup( const CNFQuery::OrGroup & group, - const ComparisonGraph<> & graph, + const ComparisonGraph & graph, const ASTs & primary_key_only_asts) { CNFQuery::OrGroup result; diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 19b23917917..ecfc617ac8a 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -33,7 +33,7 @@ concept ComparisonGraphNodeType = std::same_as || std::same_as +template class ComparisonGraph { public: diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index 4cc9572749f..c4aef89fed2 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -32,13 +32,13 @@ public: struct Data { - const ComparisonGraph<> & graph; + const ComparisonGraph & graph; std::set & components; std::unordered_map & old_name; std::unordered_map & component; UInt64 & current_id; - Data(const ComparisonGraph<> & graph_, + Data(const ComparisonGraph & graph_, std::set & components_, std::unordered_map & old_name_, std::unordered_map & component_, @@ -165,7 +165,7 @@ ColumnPrice calculatePrice( /// price of all columns on which ast depends. /// TODO: branch-and-bound void bruteforce( - const ComparisonGraph<> & graph, + const ComparisonGraph & graph, const std::vector & components, size_t current_component, const ColumnPriceByName & column_prices, diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index e3934e8ea7f..91c19fa264e 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -74,7 +74,7 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const Cons return false; } -bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph<> & graph) +bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph & graph) { /// We try to find at least one atom that is always true by using comparison graph. for (const auto & atom : group) @@ -82,7 +82,7 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { - const auto expected = ComparisonGraph<>::atomToCompareResult(atom); + const auto expected = ComparisonGraph::atomToCompareResult(atom); if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1])) return true; } @@ -108,20 +108,20 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const return false; } -bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<> & graph) +bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph) { const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { /// TODO: special support for != - const auto expected = ComparisonGraph<>::atomToCompareResult(atom); + const auto expected = ComparisonGraph::atomToCompareResult(atom); return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]); } return false; } -void replaceToConstants(ASTPtr & term, const ComparisonGraph<> & graph) +void replaceToConstants(ASTPtr & term, const ComparisonGraph & graph) { const auto equal_constant = graph.getEqualConst(term); if (equal_constant) @@ -135,7 +135,7 @@ void replaceToConstants(ASTPtr & term, const ComparisonGraph<> & graph) } } -CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<> & graph) +CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph) { CNFQuery::AtomicFormula result; result.negative = atom.negative; diff --git a/src/Interpreters/tests/gtest_comparison_graph.cpp b/src/Interpreters/tests/gtest_comparison_graph.cpp index 32dcb3de6b3..96a78241c8e 100644 --- a/src/Interpreters/tests/gtest_comparison_graph.cpp +++ b/src/Interpreters/tests/gtest_comparison_graph.cpp @@ -9,11 +9,11 @@ using namespace DB; -static ComparisonGraph<> getGraph(const String & query) +static ComparisonGraph getGraph(const String & query) { ParserExpressionList parser(false); ASTPtr ast = parseQuery(parser, query, 0, 0); - return ComparisonGraph<>(ast->children); + return ComparisonGraph(ast->children); } TEST(ComparisonGraph, Bounds) diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 3f4d264b4a1..db37ac7c4c3 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -110,7 +110,7 @@ std::vector ConstraintsDescription::getAtomicConstraint return constraint_data; } -std::unique_ptr> ConstraintsDescription::buildGraph() const +std::unique_ptr> ConstraintsDescription::buildGraph() const { static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; @@ -128,7 +128,7 @@ std::unique_ptr> ConstraintsDescription::buildGraph() const } } - return std::make_unique>(constraints_for_graph); + return std::make_unique>(constraints_for_graph); } ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextPtr context, @@ -150,7 +150,7 @@ ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextP return res; } -const ComparisonGraph<> & ConstraintsDescription::getGraph() const +const ComparisonGraph & ConstraintsDescription::getGraph() const { return *graph; } @@ -313,7 +313,7 @@ void ConstraintsDescription::update() { cnf_constraints.clear(); ast_to_atom_ids.clear(); - graph = std::make_unique>(ASTs()); + graph = std::make_unique>(ASTs()); return; } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 2c34ef1ef37..33bd8e1abf9 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -43,7 +43,7 @@ public: const std::vector> & getConstraintData() const; std::vector getAtomicConstraintData() const; - const ComparisonGraph<> & getGraph() const; + const ComparisonGraph & getGraph() const; ConstraintsExpressions getExpressions(ContextPtr context, const NamesAndTypesList & source_columns_) const; @@ -79,7 +79,7 @@ public: private: std::vector> buildConstraintData() const; - std::unique_ptr> buildGraph() const; + std::unique_ptr> buildGraph() const; void update(); ASTs constraints; @@ -87,7 +87,7 @@ private: std::vector> cnf_constraints; std::map ast_to_atom_ids; - std::unique_ptr> graph; + std::unique_ptr> graph; }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp index 2bb6857a855..4227ffc6873 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp @@ -108,7 +108,7 @@ bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const func->name = "greaterOrEquals"; } - const auto weak_graph = std::make_unique>(active_atomic_formulas); + const auto weak_graph = std::make_unique>(active_atomic_formulas); bool useless = true; expression_cnf->iterateGroups( @@ -146,7 +146,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree values.push_back(granule->met); } - const ComparisonGraph<> * graph = nullptr; + const ComparisonGraph * graph = nullptr; { std::lock_guard lock(cache_mutex); @@ -170,7 +170,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { - const auto expected = ComparisonGraph<>::atomToCompareResult(atom); + const auto expected = ComparisonGraph::atomToCompareResult(atom); if (graph->isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) { /// If graph failed use matching. @@ -188,7 +188,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree return !always_false; } -std::unique_ptr> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector & values) const +std::unique_ptr> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector & values) const { ASTs active_atomic_formulas(atomic_constraints); for (size_t i = 0; i < values.size(); ++i) @@ -199,10 +199,10 @@ std::unique_ptr> MergeTreeIndexhypothesisMergedCondition::buil std::begin(index_to_compare_atomic_hypotheses[i]), std::end(index_to_compare_atomic_hypotheses[i])); } - return std::make_unique>(active_atomic_formulas); + return std::make_unique>(active_atomic_formulas); } -const ComparisonGraph<> * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const +const ComparisonGraph * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const { auto [it, inserted] = graph_cache.try_emplace(values); if (inserted) diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h index f08cfba6ca0..3ab82f4d3ee 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h @@ -20,8 +20,8 @@ public: private: void addConstraints(const ConstraintsDescription & constraints_description); - std::unique_ptr> buildGraph(const std::vector & values) const; - const ComparisonGraph<> * getGraph(const std::vector & values) const; + std::unique_ptr> buildGraph(const std::vector & values) const; + const ComparisonGraph * getGraph(const std::vector & values) const; ASTPtr expression_ast; std::unique_ptr expression_cnf; @@ -29,7 +29,7 @@ private: /// Part analysis can be done in parallel. /// So, we have shared answer and graph cache. mutable std::mutex cache_mutex; - mutable std::unordered_map, std::unique_ptr>> graph_cache; + mutable std::unordered_map, std::unique_ptr>> graph_cache; mutable std::unordered_map, bool> answer_cache; std::vector> index_to_compare_atomic_hypotheses; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 7ae4516fe9e..520bd16ae25 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -3,51 +3,287 @@ SELECT (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 1 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test_test WHERE b = 1 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` FROM column_swap_test_test WHERE b = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, a AS a FROM column_swap_test_test WHERE cityHash64(a) = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + a String + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 + COLUMN id: 9, column_name: a, result_type: String, source_id: 7 + JOIN TREE + TABLE id: 7, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 13, nodes: 1 + COLUMN id: 14, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 SELECT (cityHash64(a) AS b) + 10 AS `plus(b, 10)`, a AS a FROM column_swap_test_test WHERE cityHash64(a) = 0 +QUERY id: 0 + PROJECTION COLUMNS + plus(b, 10) UInt64 + a String + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 + COLUMN id: 9, column_name: a, result_type: String, source_id: 7 + JOIN TREE + TABLE id: 7, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: cityHash64, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 13, nodes: 1 + COLUMN id: 14, column_name: a, result_type: String, source_id: 7 + CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 SELECT a AS `substring(reverse(b), 1, 1)`, a AS a FROM column_swap_test_test WHERE a = \'c\' +QUERY id: 0 + PROJECTION COLUMNS + substring(reverse(b), 1, 1) String + a String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: String, source_id: 3 + COLUMN id: 4, column_name: a, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'c\', constant_value_type: String SELECT a AS `substring(reverse(b), 1, 1)`, a AS a FROM column_swap_test_test WHERE a = \'c\' +QUERY id: 0 + PROJECTION COLUMNS + substring(reverse(b), 1, 1) String + a String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: String, source_id: 3 + COLUMN id: 4, column_name: a, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'c\', constant_value_type: String SELECT a AS t1, a AS t2 FROM column_swap_test_test WHERE a = \'c\' +QUERY id: 0 + PROJECTION COLUMNS + t1 String + t2 String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: String, source_id: 3 + COLUMN id: 4, column_name: a, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'c\', constant_value_type: String SELECT a AS `substring(reverse(b), 1, 1)` FROM column_swap_test_test WHERE a = \'c\' +QUERY id: 0 + PROJECTION COLUMNS + substring(reverse(b), 1, 1) String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.column_swap_test_test + WHERE + FUNCTION id: 4, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: a, result_type: String, source_id: 3 + CONSTANT id: 7, constant_value: \'c\', constant_value_type: String SELECT a FROM t_bad_constraint +QUERY id: 0 + PROJECTION COLUMNS + a UInt32 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.t_bad_constraint diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 873ebbed729..97e014d9c25 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -13,17 +13,20 @@ INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2); INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; ---EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; ---EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; ---EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; ---EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test_test WHERE b = 0; +EXPLAIN QUERY TREE SELECT b + 10, a FROM column_swap_test_test WHERE b = 0; DROP TABLE column_swap_test_test; @@ -31,9 +34,13 @@ CREATE TABLE column_swap_test_test (i Int64, a String, b String, CONSTRAINT c1 A INSERT INTO column_swap_test_test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; DROP TABLE column_swap_test_test; @@ -44,5 +51,6 @@ CREATE TABLE t_bad_constraint(a UInt32, s String, CONSTRAINT c1 ASSUME a = toUIn INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000); EXPLAIN SYNTAX SELECT a FROM t_bad_constraint; +EXPLAIN QUERY TREE SELECT a FROM t_bad_constraint; DROP TABLE t_bad_constraint; From 7f8622752c3d34b64a1fb371d4fae6f8128469cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Mar 2023 14:17:16 +0000 Subject: [PATCH 041/233] Update description --- src/Analyzer/Passes/ConvertQueryToCNFPass.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.h b/src/Analyzer/Passes/ConvertQueryToCNFPass.h index 71fb28bdf85..232af3b015e 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.h +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.h @@ -8,9 +8,9 @@ namespace DB class ConvertQueryToCNFPass final : public IQueryTreePass { public: - String getName() override { return "ConvertQueryToCnfPass"; } + String getName() override { return "ConvertQueryToCNFPass"; } - String getDescription() override { return "Convert query to CNF"; } + String getDescription() override { return "Convert query to CNF and apply optimizations using constraints"; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; From a2c0e2e4e7e2865ad1ac70fd4d38f8995dfa26b2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Mar 2023 14:55:17 +0000 Subject: [PATCH 042/233] Use pointer --- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 591f2a1557c..d5d11a19ffe 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -415,18 +415,18 @@ class ColumnNameCollectorVisitor : public ConstInDepthQueryTreeVisitor & column_names_, - const QueryTreeNodePtrWithHashMap & query_node_to_component_) + const QueryTreeNodePtrWithHashMap * query_node_to_component_) : column_names(column_names_), query_node_to_component(query_node_to_component_) {} bool needChildVisit(const VisitQueryTreeNodeType & parent, const VisitQueryTreeNodeType &) { - return !query_node_to_component.contains(parent); + return !query_node_to_component || !query_node_to_component->contains(parent); } void visitImpl(const QueryTreeNodePtr & node) { - if (query_node_to_component.contains(node)) + if (query_node_to_component && query_node_to_component->contains(node)) return; if (const auto * column_node = node->as()) @@ -435,7 +435,7 @@ public: private: std::unordered_set & column_names; - const QueryTreeNodePtrWithHashMap & query_node_to_component; + const QueryTreeNodePtrWithHashMap * query_node_to_component; }; class SubstituteColumnVisitor : public InDepthQueryTreeVisitor @@ -511,7 +511,7 @@ void bruteForce( for (const auto & node : graph.getComponent(components[current_component])) { std::unordered_set column_names; - ColumnNameCollectorVisitor column_name_collector{column_names, {}}; + ColumnNameCollectorVisitor column_name_collector{column_names, nullptr}; column_name_collector.visit(node); ColumnPrice expression_price = calculatePrice(column_prices, column_names); @@ -577,7 +577,7 @@ void substituteColumns(QueryNode & query_node, const QueryTreeNodes & table_expr { ComponentCollectorVisitor component_collector{components, query_node_to_component, graph}; component_collector.visit(node); - ColumnNameCollectorVisitor column_name_collector{column_names, query_node_to_component}; + ColumnNameCollectorVisitor column_name_collector{column_names, &query_node_to_component}; column_name_collector.visit(node); }); From 23644f655bd648010a414f5f9e096284408ec37f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 20 Mar 2023 13:23:25 +0000 Subject: [PATCH 043/233] Add support for custom key in new analyzer --- src/Planner/Planner.cpp | 80 +++++++++++++++++++++++ src/Planner/PlannerExpressionAnalysis.cpp | 37 ++++++----- src/Planner/PlannerExpressionAnalysis.h | 6 ++ 3 files changed, 105 insertions(+), 18 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2242bf92e6b..db7f3305565 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -38,6 +38,7 @@ #include #include +#include #include #include #include @@ -69,6 +70,10 @@ #include #include +#include + +#include + namespace DB { @@ -1188,6 +1193,69 @@ void Planner::buildPlanForQueryNode() collectTableExpressionData(query_tree, *planner_context); collectSets(query_tree, *planner_context); + const auto & settings = query_context->getSettingsRef(); + const auto & table_expression_data = planner_context->getTableExpressionNodeToData(); + + auto & mutable_context = planner_context->getMutableQueryContext(); + + QueryTreeNodePtr parallel_replicas_custom_filter_node{nullptr}; + if (table_expression_data.size() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + { + LOG_WARNING(&Poco::Logger::get("Planner"), "Joins are not supported with parallel replicas. Query will be executed without using them."); + mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); + mutable_context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (table_expression_data.size() == 1 && !settings.parallel_replicas_custom_key.value.empty()) + { + const auto & table_expression = (*table_expression_data.begin()).first; + + StoragePtr storage{nullptr}; + if (const auto * table_node = table_expression->as()) + storage = table_node->getStorage(); + else if (const auto * table_function_node = table_expression->as()) + storage = table_function_node->getStorage(); + + std::cout << "COUNT: " << settings.parallel_replicas_count << std::endl; + if (settings.parallel_replicas_count > 1) + { + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *query_context)) + { + LOG_TRACE(&Poco::Logger::get("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); + if (!storage) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); + + auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + settings.parallel_replicas_custom_key_filter_type, + *storage, + query_context); + + parallel_replicas_custom_filter_node = buildQueryTree(parallel_replicas_custom_filter_ast, query_context); + QueryAnalysisPass pass(table_expression); + pass.run(parallel_replicas_custom_filter_node, query_context); + } + else if (settings.parallel_replica_offset > 0) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Parallel replicas processing with custom_key has been requested " + "(setting 'max_parallel_replicas') but the table does not have custom_key defined for it " + "or it's invalid (settings `parallel_replicas_custom_key`)"); + } + } + else if (storage) + { + if (auto * distributed = dynamic_cast(storage.get()); + distributed && canUseCustomKey(settings, *distributed->getCluster(), *query_context)) + { + select_query_info.use_custom_key = true; + mutable_context->setSetting("distributed_group_by_no_merge", 2); + } + } + } + auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, select_query_info, @@ -1206,6 +1274,15 @@ void Planner::buildPlanForQueryNode() if (select_query_options.to_stage == QueryProcessingStage::FetchColumns) return; + std::optional parallel_replicas_custom_filter_info; + if (parallel_replicas_custom_filter_node) + { + ActionsChain chain; + parallel_replicas_custom_filter_info = analyzeFilter(parallel_replicas_custom_filter_node, query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), planner_context, chain); + parallel_replicas_custom_filter_info->remove_filter_column = true; + select_query_info.filter_asts.push_back(parallel_replicas_custom_filter_node->toAST()); + } + PlannerQueryProcessingInfo query_processing_info(from_stage, select_query_options.to_stage); QueryAnalysisResult query_analysis_result(query_tree, query_processing_info, planner_context); auto expression_analysis_result = buildExpressionAnalysisResult(query_tree, @@ -1237,6 +1314,9 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasWhere()) addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute); + if (parallel_replicas_custom_filter_info) + addFilterStep(query_plan, *parallel_replicas_custom_filter_info, "Parallel replica custom key filter", result_actions_to_execute); + if (expression_analysis_result.hasAggregation()) { const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 11444503c5f..9cd542b4bc1 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -29,24 +29,6 @@ namespace ErrorCodes namespace { -/** Construct filter analysis result for filter expression node - * Actions before filter are added into into actions chain. - * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. - */ -FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, - const ColumnsWithTypeAndName & input_columns, - const PlannerContextPtr & planner_context, - ActionsChain & actions_chain) -{ - FilterAnalysisResult result; - - result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); - result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; - actions_chain.addStep(std::make_unique(result.filter_actions)); - - return result; -} - /** Construct aggregation analysis result if query tree has GROUP BY or aggregates. * Actions before aggregation are added into actions chain, if result is not null optional. */ @@ -458,6 +440,25 @@ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, } +/** Construct filter analysis result for filter expression node + * Actions before filter are added into into actions chain. + * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. + */ +FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, + const ColumnsWithTypeAndName & input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + FilterAnalysisResult result; + + result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); + result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; + actions_chain.addStep(std::make_unique(result.filter_actions)); + + return result; +} + + PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNodePtr & query_tree, const ColumnsWithTypeAndName & join_tree_input_columns, const PlannerContextPtr & planner_context, diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index 792cfdec2ff..ddf9f120cdb 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -7,6 +7,7 @@ #include +#include #include #include #include @@ -169,6 +170,11 @@ private: LimitByAnalysisResult limit_by_analysis_result; }; +FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, + const ColumnsWithTypeAndName & input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain); + /// Build expression analysis result for query tree, join tree input columns and planner context PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNodePtr & query_tree, const ColumnsWithTypeAndName & join_tree_input_columns, From d8493780b2bfa2aec5781c87ea33544e97b43910 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Mar 2023 11:57:51 +0000 Subject: [PATCH 044/233] Query cache: Enable compression and squashing of result blocks ClickHouse reads table data in blocks of 'max_block_size' rows. Due to filtering, aggregation, etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting 'query_cache_squash_partial_results' (enabled by default) now controls if result blocks are squashed (if they are tiny) or split (if they are large) into blocks of 'max_block_size' size before insertion into the query result cache. This reduces performance of writes into the query cache but improves compressability of cache entries and provides more natural block granularity when query results are later served from the query cache. Entries in the query cache are now also compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads from the query cache. To disable compression, use setting 'query_cache_compress_entries'. --- docs/en/operations/query-cache.md | 14 + docs/en/operations/settings/settings.md | 22 ++ src/Columns/ColumnCompressed.cpp | 2 +- src/Core/Settings.h | 2 + src/Interpreters/Cache/QueryCache.cpp | 106 ++++++- src/Interpreters/Cache/QueryCache.h | 26 +- src/Interpreters/executeQuery.cpp | 14 +- src/Processors/Chunk.cpp | 33 ++- src/Processors/Chunk.h | 1 + .../StreamInQueryCacheTransform.cpp | 9 +- .../Transforms/StreamInQueryCacheTransform.h | 7 +- .../System/StorageSystemQueryCache.cpp | 6 +- ...query_cache_disabled_compression.reference | 2 + ...02494_query_cache_disabled_compression.sql | 12 + ...ery_cache_squash_partial_results.reference | 275 ++++++++++++++++++ ...494_query_cache_squash_partial_results.sql | 48 +++ 16 files changed, 538 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_disabled_compression.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_disabled_compression.sql create mode 100644 tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 1a486de7904..b50c4e5bbf4 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -103,6 +103,20 @@ cached - for that use setting [query_cache_min_query_runs](settings/settings.md# Entries in the query cache become stale after a certain time period (time-to-live). By default, this period is 60 seconds but a different value can be specified at session, profile or query level using setting [query_cache_ttl](settings/settings.md#query-cache-ttl). +Entries in the query cache are compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads +from the query cache. To disable compression, use setting [query_cache_compress_entries](settings/settings.md#query-cache-compress-entries). + +ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#settings-max_block_size) rows. Due to filtering, aggregation, +etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting +[query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks +are squashed (if they are tiny) or split (if they are large) into blocks of 'max_block_size' size before insertion into the query result +cache. This reduces performance of writes into the query cache but improves compression rate of cache entries and provides more natural +block granularity when query results are later served from the query cache. + +As a result, the query cache stores for each query multiple (partial) +result blocks. While this behavior is a good default, it can be suppressed using setting +[query_cache_squash_partial_query_results](settings/settings.md#query-cache-squash-partial-query-results). + Also, results of queries with non-deterministic functions such as `rand()` and `now()` are not cached. This can be overruled using setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index daaa79e90db..12c2ff8597e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1425,6 +1425,28 @@ Possible values: Default value: `0` +## query_cache_compress_entries {#query-cache-compress-entries} + +Compress entries in the [query cache](../query-cache.md). Lessens the memory consumption of the query cache at the cost of slower inserts into / reads from it. + +Possible values: + +- 0 - Disabled +- 1 - Enabled + +Default value: `1` + +## query_cache_squash_partial_results {#query-cache-squash-partial-results} + +Squash partial result blocks to blocks of size [max_block_size](#setting-max_block_size). Reduces performance of inserts into the [query cache](../query-cache.md) but improves the compressability of cache entries (see [query_cache_compress-entries](#query_cache_compress_entries)). + +Possible values: + +- 0 - Disabled +- 1 - Enabled + +Default value: `1` + ## query_cache_ttl {#query-cache-ttl} After this time in seconds entries in the [query cache](../query-cache.md) become stale. diff --git a/src/Columns/ColumnCompressed.cpp b/src/Columns/ColumnCompressed.cpp index cdf604d89f7..aead31da3e9 100644 --- a/src/Columns/ColumnCompressed.cpp +++ b/src/Columns/ColumnCompressed.cpp @@ -24,7 +24,7 @@ std::shared_ptr> ColumnCompressed::compressBuffer(const void * data, si Memory<> compressed(max_dest_size); - auto compressed_size = LZ4_compress_default( + int compressed_size = LZ4_compress_default( reinterpret_cast(data), compressed.data(), static_cast(data_size), diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3770d7f73a0..ca4bc8f324a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -557,6 +557,8 @@ class IColumn; M(Bool, query_cache_store_results_of_queries_with_nondeterministic_functions, false, "Store results of queries with non-deterministic functions (e.g. rand(), now()) in the query cache", 0) \ M(UInt64, query_cache_min_query_runs, 0, "Minimum number a SELECT query must run before its result is stored in the query cache", 0) \ M(Milliseconds, query_cache_min_query_duration, 0, "Minimum time in milliseconds for a query to run for its result to be stored in the query cache.", 0) \ + M(Bool, query_cache_compress_entries, true, "Compress cache entries.", 0) \ + M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \ M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \ M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \ \ diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index ce2373a8af9..b37cfbb17f4 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -122,12 +122,15 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) QueryCache::Key::Key( ASTPtr ast_, - Block header_, const std::optional & username_, - std::chrono::time_point expires_at_) + Block header_, + const std::optional & username_, + std::chrono::time_point expires_at_, + bool is_compressed_) : ast(removeQueryCacheSettings(ast_)) , header(header_) , username(username_) , expires_at(expires_at_) + , is_compressed(is_compressed_) { } @@ -153,7 +156,7 @@ size_t QueryCache::KeyHasher::operator()(const Key & key) const return res; } -size_t QueryCache::QueryResultWeight::operator()(const QueryResult & chunks) const +size_t QueryCache::QueryResultWeight::operator()(const Chunks & chunks) const { size_t res = 0; for (const auto & chunk : chunks) @@ -168,12 +171,16 @@ bool QueryCache::IsStale::operator()(const Key & key) const QueryCache::Writer::Writer(Cache & cache_, const Key & key_, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_, - std::chrono::milliseconds min_query_runtime_) + std::chrono::milliseconds min_query_runtime_, + bool squash_partial_results_, + size_t max_block_size_) : cache(cache_) , key(key_) , max_entry_size_in_bytes(max_entry_size_in_bytes_) , max_entry_size_in_rows(max_entry_size_in_rows_) , min_query_runtime(min_query_runtime_) + , squash_partial_results(squash_partial_results_) + , max_block_size(max_block_size_) { if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { @@ -211,6 +218,8 @@ void QueryCache::Writer::finalizeWrite() std::lock_guard lock(mutex); + chassert(!was_finalized); + if (std::chrono::duration_cast(std::chrono::system_clock::now() - query_start_time) < min_query_runtime) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.queryStringFromAst()); @@ -224,7 +233,67 @@ void QueryCache::Writer::finalizeWrite() return; } + if (squash_partial_results) + { + // Squash partial result chunks to chunks of size 'max_block_size' each. This costs some performance but provides a more natural + // compression of neither too small nor big blocks. Also, it will look like 'max_block_size' is respected when the query result is + // served later on from the query cache. + + Chunks squashed_chunks; + size_t rows_remaining_in_squashed = 0; /// how many further rows can the last squashed chunk consume until it reaches max_block_size + + for (const auto & chunk : *query_result) + { + const size_t rows_chunk = chunk.getNumRows(); + size_t rows_chunk_processed = 0; + + if (rows_chunk == 0) + continue; + + while (true) + { + if (rows_remaining_in_squashed == 0) + { + Chunk empty_chunk = Chunk(chunk.cloneEmptyColumns(), 0); + squashed_chunks.push_back(std::move(empty_chunk)); + rows_remaining_in_squashed = max_block_size; + } + + const size_t rows_to_append = std::min(rows_chunk - rows_chunk_processed, rows_remaining_in_squashed); + squashed_chunks.back().append(chunk, rows_chunk_processed, rows_to_append); + rows_chunk_processed += rows_to_append; + rows_remaining_in_squashed += rows_to_append; + + if (rows_chunk_processed == rows_chunk) + break; + } + } + + *query_result = std::move(squashed_chunks); + } + + if (key.is_compressed) + { + Chunks compressed_chunks; + const Chunks & decompressed_chunks = *query_result; + for (const auto & decompressed_chunk : decompressed_chunks) + { + const Columns & decompressed_columns = decompressed_chunk.getColumns(); + Columns compressed_columns; + for (const auto & decompressed_column : decompressed_columns) + { + auto compressed_column = decompressed_column->compress(); + compressed_columns.push_back(compressed_column); + } + Chunk compressed_chunk(compressed_columns, decompressed_chunk.getNumRows()); + compressed_chunks.push_back(std::move(compressed_chunk)); + } + *query_result = std::move(compressed_chunks); + } + cache.set(key, query_result); + + was_finalized = true; } QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guard &) @@ -249,7 +318,28 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar return; } - pipe = Pipe(std::make_shared(entry->key.header, entry->mapped)); + if (!entry->key.is_compressed) + pipe = Pipe(std::make_shared(entry->key.header, entry->mapped)); + else + { + auto decompressed_chunks = std::make_shared(); + const Chunks & compressed_chunks = *entry->mapped; + for (const auto & compressed_chunk : compressed_chunks) + { + const Columns & compressed_chunk_columns = compressed_chunk.getColumns(); + Columns decompressed_columns; + for (const auto & compressed_column : compressed_chunk_columns) + { + auto column = compressed_column->decompress(); + decompressed_columns.push_back(column); + } + Chunk decompressed_chunk(decompressed_columns, compressed_chunk.getNumRows()); + decompressed_chunks->push_back(std::move(decompressed_chunk)); + } + + pipe = Pipe(std::make_shared(entry->key.header, decompressed_chunks)); + } + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); } @@ -277,10 +367,10 @@ QueryCache::Reader QueryCache::createReader(const Key & key) return Reader(cache, key, lock); } -QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::milliseconds min_query_runtime) +QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size) { std::lock_guard lock(mutex); - return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime); + return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime, squash_partial_results, max_block_size); } void QueryCache::reset() @@ -308,7 +398,7 @@ std::vector QueryCache::dump() const } QueryCache::QueryCache() - : cache(std::make_unique>()) + : cache(std::make_unique>()) { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 763e797ac07..316694ee406 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -50,16 +50,19 @@ public: /// When does the entry expire? const std::chrono::time_point expires_at; + /// Is the entry compressed? + const bool is_compressed; + Key(ASTPtr ast_, - Block header_, const std::optional & username_, - std::chrono::time_point expires_at_); + Block header_, + const std::optional & username_, + std::chrono::time_point expires_at_, + bool is_compressed); bool operator==(const Key & other) const; String queryStringFromAst() const; }; - using QueryResult = Chunks; - private: struct KeyHasher { @@ -68,7 +71,7 @@ private: struct QueryResultWeight { - size_t operator()(const QueryResult & chunks) const; + size_t operator()(const Chunks & chunks) const; }; struct IsStale @@ -77,7 +80,7 @@ private: }; /// query --> query result - using Cache = CacheBase; + using Cache = CacheBase; /// query --> query execution count using TimesExecuted = std::unordered_map; @@ -109,12 +112,17 @@ public: const size_t max_entry_size_in_rows; const std::chrono::time_point query_start_time = std::chrono::system_clock::now(); /// Writer construction and finalizeWrite() coincide with query start/end const std::chrono::milliseconds min_query_runtime; - std::shared_ptr query_result TSA_GUARDED_BY(mutex) = std::make_shared(); + const bool squash_partial_results; + const size_t max_block_size; + std::shared_ptr query_result TSA_GUARDED_BY(mutex) = std::make_shared(); std::atomic skip_insert = false; + bool was_finalized = false; Writer(Cache & cache_, const Key & key_, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_, - std::chrono::milliseconds min_query_runtime_); + std::chrono::milliseconds min_query_runtime_, + bool squash_partial_results_, + size_t max_block_size_); friend class QueryCache; /// for createWriter() }; @@ -136,7 +144,7 @@ public: void updateConfiguration(const Poco::Util::AbstractConfiguration & config); Reader createReader(const Key & key); - Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime); + Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size); void reset(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 74394879191..83e88f5b2e5 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -726,7 +726,8 @@ static std::tuple executeQueryImpl( QueryCache::Key key( ast, res.pipeline.getHeader(), std::make_optional(context->getUserName()), - std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl)); + /*dummy value for expires_at*/ std::chrono::system_clock::from_time_t(1), + /*dummy value for is_compressed*/ true); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -748,13 +749,18 @@ static std::tuple executeQueryImpl( QueryCache::Key key( ast, res.pipeline.getHeader(), settings.query_cache_share_between_users ? std::nullopt : std::make_optional(context->getUserName()), - std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl)); + std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), + settings.query_cache_compress_entries); const size_t num_query_runs = query_cache->recordQueryRun(key); if (num_query_runs > settings.query_cache_min_query_runs) { - auto stream_in_query_cache_transform = std::make_shared(res.pipeline.getHeader(), query_cache, key, - std::chrono::milliseconds(context->getSettings().query_cache_min_query_duration.totalMilliseconds())); + auto stream_in_query_cache_transform = + std::make_shared( + res.pipeline.getHeader(), query_cache, key, + std::chrono::milliseconds(context->getSettings().query_cache_min_query_duration.totalMilliseconds()), + context->getSettings().query_cache_squash_partial_results, + context->getSettings().max_block_size); res.pipeline.streamIntoQueryCache(stream_in_query_cache_transform); } } diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index bbfa1683cf6..0a4b2413e4c 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -23,11 +23,11 @@ Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) checkNumRowsIsConsistent(); } -static Columns unmuteColumns(MutableColumns && mut_columns) +static Columns unmuteColumns(MutableColumns && mutable_columns) { Columns columns; - columns.reserve(mut_columns.size()); - for (auto & col : mut_columns) + columns.reserve(mutable_columns.size()); + for (auto & col : mutable_columns) columns.emplace_back(std::move(col)); return columns; @@ -78,23 +78,23 @@ void Chunk::checkNumRowsIsConsistent() MutableColumns Chunk::mutateColumns() { size_t num_columns = columns.size(); - MutableColumns mut_columns(num_columns); + MutableColumns mutable_columns(num_columns); for (size_t i = 0; i < num_columns; ++i) - mut_columns[i] = IColumn::mutate(std::move(columns[i])); + mutable_columns[i] = IColumn::mutate(std::move(columns[i])); columns.clear(); num_rows = 0; - return mut_columns; + return mutable_columns; } MutableColumns Chunk::cloneEmptyColumns() const { size_t num_columns = columns.size(); - MutableColumns mut_columns(num_columns); + MutableColumns mutable_columns(num_columns); for (size_t i = 0; i < num_columns; ++i) - mut_columns[i] = columns[i]->cloneEmpty(); - return mut_columns; + mutable_columns[i] = columns[i]->cloneEmpty(); + return mutable_columns; } Columns Chunk::detachColumns() @@ -171,14 +171,19 @@ std::string Chunk::dumpStructure() const void Chunk::append(const Chunk & chunk) { - MutableColumns mutation = mutateColumns(); - for (size_t position = 0; position < mutation.size(); ++position) + append(chunk, 0, chunk.getNumRows()); +} + +void Chunk::append(const Chunk & chunk, size_t from, size_t length) +{ + MutableColumns mutable_columns = mutateColumns(); + for (size_t position = 0; position < mutable_columns.size(); ++position) { auto column = chunk.getColumns()[position]; - mutation[position]->insertRangeFrom(*column, 0, column->size()); + mutable_columns[position]->insertRangeFrom(*column, from, length); } - size_t rows = mutation[0]->size(); - setColumns(std::move(mutation), rows); + size_t rows = mutable_columns[0]->size(); + setColumns(std::move(mutable_columns), rows); } void ChunkMissingValues::setBit(size_t column_idx, size_t row_idx) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 15d91431b68..6f2097b71f1 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -102,6 +102,7 @@ public: std::string dumpStructure() const; void append(const Chunk & chunk); + void append(const Chunk & chunk, size_t from, size_t length); // append rows [from, from+length) of chunk private: Columns columns; diff --git a/src/Processors/Transforms/StreamInQueryCacheTransform.cpp b/src/Processors/Transforms/StreamInQueryCacheTransform.cpp index 1ba57ea8ed2..6156123f9f8 100644 --- a/src/Processors/Transforms/StreamInQueryCacheTransform.cpp +++ b/src/Processors/Transforms/StreamInQueryCacheTransform.cpp @@ -4,9 +4,14 @@ namespace DB { StreamInQueryCacheTransform::StreamInQueryCacheTransform( - const Block & header_, QueryCachePtr cache, const QueryCache::Key & cache_key, std::chrono::milliseconds min_query_duration) + const Block & header_, + QueryCachePtr cache, + const QueryCache::Key & cache_key, + std::chrono::milliseconds min_query_duration, + bool squash_partial_results, + size_t max_block_size) : ISimpleTransform(header_, header_, false) - , cache_writer(cache->createWriter(cache_key, min_query_duration)) + , cache_writer(cache->createWriter(cache_key, min_query_duration, squash_partial_results, max_block_size)) { } diff --git a/src/Processors/Transforms/StreamInQueryCacheTransform.h b/src/Processors/Transforms/StreamInQueryCacheTransform.h index 15d977cd445..42472a708fe 100644 --- a/src/Processors/Transforms/StreamInQueryCacheTransform.h +++ b/src/Processors/Transforms/StreamInQueryCacheTransform.h @@ -10,7 +10,12 @@ class StreamInQueryCacheTransform : public ISimpleTransform { public: StreamInQueryCacheTransform( - const Block & header_, QueryCachePtr cache, const QueryCache::Key & cache_key, std::chrono::milliseconds min_query_duration); + const Block & header_, + QueryCachePtr cache, + const QueryCache::Key & cache_key, + std::chrono::milliseconds min_query_duration, + bool squash_partial_results, + size_t max_block_size); protected: void transform(Chunk & chunk) override; diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 2cbcc773ad6..5e4a1e662e2 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -16,6 +16,7 @@ NamesAndTypesList StorageSystemQueryCache::getNamesAndTypes() {"key_hash", std::make_shared()}, {"expires_at", std::make_shared()}, {"stale", std::make_shared()}, + {"compressed", std::make_shared()}, {"shared", std::make_shared()}, {"result_size", std::make_shared()} }; @@ -47,8 +48,9 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[1]->insert(key.ast->getTreeHash().first); res_columns[2]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); res_columns[3]->insert(key.expires_at < std::chrono::system_clock::now()); - res_columns[4]->insert(!key.username.has_value()); - res_columns[5]->insert(QueryCache::QueryResultWeight()(*query_result)); + res_columns[4]->insert(key.is_compressed); + res_columns[5]->insert(!key.username.has_value()); + res_columns[6]->insert(QueryCache::QueryResultWeight()(*query_result)); } } diff --git a/tests/queries/0_stateless/02494_query_cache_disabled_compression.reference b/tests/queries/0_stateless/02494_query_cache_disabled_compression.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_disabled_compression.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02494_query_cache_disabled_compression.sql b/tests/queries/0_stateless/02494_query_cache_disabled_compression.sql new file mode 100644 index 00000000000..ca95ffd918d --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_disabled_compression.sql @@ -0,0 +1,12 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SET allow_experimental_query_cache = true; + +SYSTEM DROP QUERY CACHE; + +-- Run query and store result in query cache but without compression which is on by default +SELECT 1 SETTINGS use_query_cache = true, query_cache_compress_entries = false; + +-- Run again to check that no bad things happen and that the result is as expected +SELECT 1 SETTINGS use_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference new file mode 100644 index 00000000000..e3ffe57ae3e --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference @@ -0,0 +1,275 @@ +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +- +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +-------------------- +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +- +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +ghi +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl diff --git a/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql new file mode 100644 index 00000000000..eee633b747e --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql @@ -0,0 +1,48 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SET allow_experimental_query_cache = true; + +SYSTEM DROP QUERY CACHE; + +DROP TABLE IF EXISTS t; + +-- Create test table with "many" rows +CREATE TABLE t(c String) ENGINE=MergeTree ORDER BY c; +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); +INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); + +-- Run query which reads multiple chunks (small max_block_size), cache result in query cache, force squashing of partial results +SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true, query_cache_squash_partial_results = true; + +SELECT '-'; + +-- Run again to check that no bad things happen and that the result is as expected +SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true; + +SELECT '--------------------'; + +-- Run query which reads multiple chunks (small max_block_size), cache result in query cache, but **disable** squashing of partial results +SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true, query_cache_squash_partial_results = false; + +SELECT '-'; + +-- Run again to check that no bad things happen and that the result is as expected +SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true; + +DROP TABLE t; From d72c0def6e9c6dc1700c56d32e20d3004e942672 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 21 Mar 2023 16:30:13 +0000 Subject: [PATCH 045/233] Improve search of original node name in ActionsDAG --- src/Interpreters/ActionsDAG.cpp | 22 +++++++++---------- src/Interpreters/ActionsDAG.h | 13 ++++++++++- src/Processors/QueryPlan/ExpressionStep.cpp | 4 ++-- src/Processors/QueryPlan/FilterStep.cpp | 4 ++-- .../Optimizations/distinctReadInOrder.cpp | 3 ++- .../Optimizations/removeRedundantDistinct.cpp | 6 +++-- 6 files changed, 32 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 66d4fc73f2b..fbddcf31aa0 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2316,23 +2316,21 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( return result_dag; } -const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name) +FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_) + :actions(actions_) { - /// find alias in output - const ActionsDAG::Node * output_alias = nullptr; for (const auto * node : actions->getOutputs()) - { - if (node->result_name == output_name) - { - output_alias = node; - break; - } - } - if (!output_alias) + index.emplace(node->result_name, node); +} + +const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & output_name) +{ + const auto it = index.find(output_name); + if (it == index.end()) return nullptr; /// find original(non alias) node it refers to - const ActionsDAG::Node * node = output_alias; + const ActionsDAG::Node * node = it->second; while (node && node->type == ActionsDAG::ActionType::ALIAS) { chassert(!node->children.empty()); diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 771fce673c8..13871c5d308 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -374,7 +374,18 @@ private: static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); }; -const ActionsDAG::Node * getOriginalNodeForOutputAlias(const ActionsDAGPtr & actions, const String & output_name); +class FindOriginalNodeForOutputName +{ + using NameToNodeIndex = std::unordered_map; + +public: + explicit FindOriginalNodeForOutputName(const ActionsDAGPtr & actions); + const ActionsDAG::Node* find(const String& output_name); + +private: + ActionsDAGPtr actions; + NameToNodeIndex index; +}; /// This is an ugly way to bypass impossibility to forward declare ActionDAG::Node. struct ActionDAGNodes diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 8a1e10f0643..a93ccb2e020 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -79,7 +79,7 @@ void ExpressionStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - const ActionsDAGPtr & actions = actions_dag; + FindOriginalNodeForOutputName original_node_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { @@ -88,7 +88,7 @@ void ExpressionStep::updateOutputStream() const auto & origin_column = desc.column_name; for (const auto & column : output_stream->header) { - const auto * original_node = getOriginalNodeForOutputAlias(actions, column.name); + const auto * original_node = original_node_finder.find(column.name); if (original_node && original_node->result_name == origin_column) { alias = column.name; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index d63fded918d..bab94b62f23 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -109,7 +109,7 @@ void FilterStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - const ActionsDAGPtr & actions = actions_dag; + FindOriginalNodeForOutputName original_node_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { @@ -118,7 +118,7 @@ void FilterStep::updateOutputStream() const auto & origin_column = desc.column_name; for (const auto & column : output_stream->header) { - const auto * original_node = getOriginalNodeForOutputAlias(actions, column.name); + const auto * original_node = original_node_finder.find(column.name); if (original_node && original_node->result_name == origin_column) { alias = column.name; diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 04f10cfa821..0a3a4094a66 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -30,6 +30,7 @@ static std::set getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector & dag_stack) { auto actions = buildActionsForPlanPath(dag_stack); + FindOriginalNodeForOutputName original_node_finder(actions); std::set original_distinct_columns; for (const auto & column : distinct_columns) { @@ -37,7 +38,7 @@ getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std: if (isColumnConst(*column.column)) continue; - const auto * input_node = getOriginalNodeForOutputAlias(actions, column.name); + const auto * input_node = original_node_finder.find(column.name); if (!input_node) break; diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index 1ad90358acc..a6029d673e3 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -72,10 +72,11 @@ namespace logDebug("distinct_columns size", distinct_columns.size()); std::set original_distinct_columns; + FindOriginalNodeForOutputName original_node_finder(path_actions); for (const auto & column : distinct_columns) { logDebug("distinct column name", column); - const auto * alias_node = getOriginalNodeForOutputAlias(path_actions, String(column)); + const auto * alias_node = original_node_finder.find(String(column)); if (!alias_node) { logDebug("original name for alias is not found", column); @@ -242,9 +243,10 @@ namespace logActionsDAG("distinct pass: merged DAG", path_actions); /// compare columns of two DISTINCTs + FindOriginalNodeForOutputName original_node_finder(path_actions); for (const auto & column : distinct_columns) { - const auto * alias_node = getOriginalNodeForOutputAlias(path_actions, String(column)); + const auto * alias_node = original_node_finder.find(String(column)); if (!alias_node) return false; From 7504e107acb4fc48a793fb542f17693b058b98c7 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 21 Mar 2023 22:11:19 +0000 Subject: [PATCH 046/233] Fix CPU usage counters in segmentator thread --- .../Formats/Impl/ParallelParsingInputFormat.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 5ba32251a71..5881854571c 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -18,6 +18,10 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr CurrentThread::attachToGroup(thread_group); setThreadName("Segmentator"); + + Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + UInt64 last_profile_events_update_time = 0; + try { while (!parsing_finished) @@ -50,6 +54,15 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr if (!have_more_data) break; + + // Segmentator thread can be long-living, so we have to manually update performance counters for CPU progress to be correct + constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds + UInt64 total_elapsed_microseconds = total_stopwatch.elapsedMicroseconds(); + if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) + { + CurrentThread::updatePerformanceCounters(); + last_profile_events_update_time = total_elapsed_microseconds; + } } } catch (...) From f3c12b854386f9c57fe5f8a40193eaae3471a9f8 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 22 Mar 2023 13:51:07 +0000 Subject: [PATCH 047/233] fix per-thread perf counters update --- src/Common/CurrentThread.cpp | 7 +++++++ src/Common/CurrentThread.h | 1 + src/Common/ThreadStatus.cpp | 11 +++++++++++ src/Common/ThreadStatus.h | 4 ++++ src/Common/mysqlxx/Pool.cpp | 14 +------------- .../Formats/Impl/ParallelParsingInputFormat.cpp | 11 +---------- src/QueryPipeline/ReadProgressCallback.cpp | 15 +++------------ src/QueryPipeline/ReadProgressCallback.h | 6 ++---- 8 files changed, 30 insertions(+), 39 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 7fd82426522..6ec46d6508c 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -25,6 +25,13 @@ void CurrentThread::updatePerformanceCounters() current_thread->updatePerformanceCounters(); } +void CurrentThread::updatePerformanceCountersIfNeeded() +{ + if (unlikely(!current_thread)) + return; + current_thread->updatePerformanceCountersIfNeeded(); +} + bool CurrentThread::isInitialized() { return current_thread; diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 3b16163b1ba..373f7aa3e10 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -53,6 +53,7 @@ public: /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats static void updatePerformanceCounters(); + static void updatePerformanceCountersIfNeeded(); static ProfileEvents::Counters & getProfileEvents(); inline ALWAYS_INLINE static MemoryTracker * getMemoryTracker() diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index aa1690890d8..91c695216a8 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -217,6 +217,17 @@ void ThreadStatus::updatePerformanceCounters() } } +void ThreadStatus::updatePerformanceCountersIfNeeded() +{ + constexpr UInt64 performance_counters_update_period_microseconds = 10 * 1000; // 10 milliseconds + UInt64 total_elapsed_microseconds = stopwatch.elapsedMicroseconds(); + if (last_performance_counters_update_time + performance_counters_update_period_microseconds < total_elapsed_microseconds) + { + CurrentThread::updatePerformanceCounters(); + last_performance_counters_update_time = total_elapsed_microseconds; + } +} + void ThreadStatus::onFatalError() { if (fatal_error_callback) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 77c924f9650..79474f292ec 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -202,6 +203,8 @@ private: /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; std::unique_ptr taskstats; + Stopwatch stopwatch{CLOCK_MONOTONIC_COARSE}; + UInt64 last_performance_counters_update_time = 0; /// See setInternalThread() bool internal_thread = false; @@ -265,6 +268,7 @@ public: /// Update several ProfileEvents counters void updatePerformanceCounters(); + void updatePerformanceCountersIfNeeded(); /// Update ProfileEvents and dumps info to system.query_thread_log void finalizePerformanceCounters(); diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 6cd1ae8b399..6438d76cc3a 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -10,22 +10,10 @@ #include #include #include +#include #include -namespace -{ - -inline uint64_t clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC) -{ - struct timespec ts; - clock_gettime(clock_type, &ts); - return uint64_t(ts.tv_sec * 1000000000LL + ts.tv_nsec); -} - -} - - namespace mysqlxx { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 5881854571c..a2e5074efb1 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -19,9 +19,6 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr setThreadName("Segmentator"); - Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - UInt64 last_profile_events_update_time = 0; - try { while (!parsing_finished) @@ -56,13 +53,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr break; // Segmentator thread can be long-living, so we have to manually update performance counters for CPU progress to be correct - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = total_stopwatch.elapsedMicroseconds(); - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) - { - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } + CurrentThread::updatePerformanceCountersIfNeeded(); } } catch (...) diff --git a/src/QueryPipeline/ReadProgressCallback.cpp b/src/QueryPipeline/ReadProgressCallback.cpp index 6692b0f96bd..0f50d56f1a5 100644 --- a/src/QueryPipeline/ReadProgressCallback.cpp +++ b/src/QueryPipeline/ReadProgressCallback.cpp @@ -112,22 +112,13 @@ bool ReadProgressCallback::onProgress(uint64_t read_rows, uint64_t read_bytes, c size_t total_rows = progress.total_rows_to_read; - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = total_stopwatch.elapsedMicroseconds(); + CurrentThread::updatePerformanceCountersIfNeeded(); - std::lock_guard lock(last_profile_events_update_time_mutex); - { - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) - { - /// TODO: Should be done in PipelineExecutor. - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } - } + std::lock_guard lock(limits_and_quotas_mutex); /// TODO: Should be done in PipelineExecutor. for (const auto & limits : storage_limits) - limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); + limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_stopwatch.elapsedMicroseconds()); if (quota) quota->used({QuotaType::READ_ROWS, value.read_rows}, {QuotaType::READ_BYTES, value.read_bytes}); diff --git a/src/QueryPipeline/ReadProgressCallback.h b/src/QueryPipeline/ReadProgressCallback.h index c8f0d4cf537..08f2f9fc99b 100644 --- a/src/QueryPipeline/ReadProgressCallback.h +++ b/src/QueryPipeline/ReadProgressCallback.h @@ -38,10 +38,8 @@ private: /// The approximate total number of rows to read. For progress bar. std::atomic_size_t total_rows_approx = 0; - Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Time with waiting time. - /// According to total_stopwatch in microseconds. - UInt64 last_profile_events_update_time = 0; - std::mutex last_profile_events_update_time_mutex; + std::mutex limits_and_quotas_mutex; + Stopwatch total_stopwatch{CLOCK_MONOTONIC_COARSE}; /// Including waiting time bool update_profile_events = true; }; From a4d1cd514d3ffd6f109dab4d9184b47d01ea8fa9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Mar 2023 08:58:56 +0000 Subject: [PATCH 048/233] Refactor --- programs/copier/ClusterCopierApp.cpp | 9 +-- .../extract-from-config/ExtractFromConfig.cpp | 11 +--- programs/server/Server.cpp | 8 +-- src/Common/ZooKeeper/ZooKeeper.cpp | 24 +++++-- src/Common/ZooKeeper/ZooKeeper.h | 6 +- .../examples/zk_many_watches_reconnect.cpp | 66 ------------------- src/Interpreters/Context.cpp | 8 +-- ...get_abandonable_lock_in_all_partitions.cpp | 2 +- .../get_current_inserts_in_replicated.cpp | 2 +- 9 files changed, 37 insertions(+), 99 deletions(-) delete mode 100644 src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 297648280aa..822289dd89c 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -1,4 +1,5 @@ #include "ClusterCopierApp.h" +#include #include #include #include @@ -12,11 +13,6 @@ namespace fs = std::filesystem; namespace DB { -namespace ErrorCodes -{ - extern const int EXCESSIVE_ELEMENT_IN_CONFIG; -} - /// ClusterCopierApp void ClusterCopierApp::initialize(Poco::Util::Application & self) @@ -197,8 +193,7 @@ void ClusterCopierApp::mainImpl() if (!task_file.empty()) copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); - if (config().has("zookeeper") && config().has("keeper")) - throw Exception("Both ZooKeeper and Keeper are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + zkutil::validateZooKeeperConfig(config()); copier->init(); copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); diff --git a/programs/extract-from-config/ExtractFromConfig.cpp b/programs/extract-from-config/ExtractFromConfig.cpp index 75b0d311fdb..5305c61b730 100644 --- a/programs/extract-from-config/ExtractFromConfig.cpp +++ b/programs/extract-from-config/ExtractFromConfig.cpp @@ -20,11 +20,6 @@ #include -namespace DB::ErrorCodes -{ - extern const int EXCESSIVE_ELEMENT_IN_CONFIG; -} - static void setupLogging(const std::string & log_level) { Poco::AutoPtr channel(new Poco::ConsoleChannel); @@ -95,11 +90,9 @@ static std::vector extractFromConfig( { DB::ConfigurationPtr bootstrap_configuration(new Poco::Util::XMLConfiguration(config_xml)); - if (bootstrap_configuration->has("zookeeper") && bootstrap_configuration->has("keeper")) - throw DB::Exception(DB::ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both ZooKeeper and Keeper are specified"); + zkutil::validateZooKeeperConfig(*bootstrap_configuration); - zkutil::ZooKeeperPtr zookeeper; - zookeeper = std::make_shared( + zkutil::ZooKeeperPtr zookeeper = std::make_shared( *bootstrap_configuration, bootstrap_configuration->has("zookeeper") ? "zookeeper" : "keeper", nullptr); zkutil::ZooKeeperNodeCache zk_node_cache([&] { return zookeeper; }); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 79c14dac0a9..802bee7fad5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -815,10 +815,8 @@ try } ); - if (config().has("zookeeper") && config().has("keeper")) - throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both ZooKeeper and Keeper are specified"); - - bool has_zookeeper = config().has("zookeeper") || config().has("keeper") || config().has("keeper_server"); + zkutil::validateZooKeeperConfig(config()); + bool has_zookeeper = zkutil::hasZooKeeperConfig(config()); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); zkutil::EventPtr main_config_zk_changed_event = std::make_shared(); @@ -1310,7 +1308,7 @@ try { /// We do not load ZooKeeper configuration on the first config loading /// because TestKeeper server is not started yet. - if (config->has("zookeeper") || config->has("keeper") || config->has("keeper_server")) + if (zkutil::hasZooKeeperConfig(config)) global_context->reloadZooKeeperIfChanged(config); global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index bda1b168a14..f9d851f9697 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int NO_ELEMENTS_IN_CONFIG; + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; } } @@ -1335,16 +1336,29 @@ String getSequentialNodeName(const String & prefix, UInt64 number) return name; } -String getZookeeperConfigName(const Poco::Util::AbstractConfiguration & config) +void validateZooKeeperConfig(const Poco::Util::AbstractConfiguration & config) +{ + if (config.has("zookeeper") && config.has("keeper")) + throw DB::Exception(DB::ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both ZooKeeper and Keeper are specified"); +} + +bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server) +{ + return config.has("zookeeper") || config.has("keeper") || (allow_keeper_server && config.has("keeper_server")); +} + +String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server) { if (config.has("zookeeper")) return "zookeeper"; - else if (config.has("keeper")) + + if (config.has("keeper")) return "keeper"; - else if (config.has("keeper_server")) + + if (allow_keeper_server && config.has("keeper_server")) return "keeper_server"; - else - throw DB::Exception("There is no Zookeeper configuration in server config", DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + throw DB::Exception(DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); } } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index d02fbbedd86..8776497a41d 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -667,6 +667,10 @@ String extractZooKeeperPath(const String & path, bool check_starts_with_slash, P String getSequentialNodeName(const String & prefix, UInt64 number); -String getZookeeperConfigName(const Poco::Util::AbstractConfiguration & config); +void validateZooKeeperConfig(const Poco::Util::AbstractConfiguration & config); + +bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server = true); + +String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server = true); } diff --git a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp deleted file mode 100644 index aad8913ca8b..00000000000 --- a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp +++ /dev/null @@ -1,66 +0,0 @@ -#include -#include -#include -#include - -/// A tool for reproducing https://issues.apache.org/jira/browse/ZOOKEEPER-706 -/// Original libzookeeper can't reconnect the session if the length of SET_WATCHES message -/// exceeds jute.maxbuffer (0xfffff by default). -/// This happens when the number of watches exceeds ~29000. -/// -/// Session reconnect can be caused by forbidding packets to the current zookeeper server, e.g. -/// sudo ip6tables -A OUTPUT -d mtzoo01it.haze.yandex.net -j REJECT - -const size_t N_THREADS = 100; - -int main(int argc, char ** argv) -{ - try - { - if (argc != 3) - { - std::cerr << "usage: " << argv[0] << " " << std::endl; - return 3; - } - - DB::ConfigProcessor processor(argv[1], false, true); - auto config = processor.loadConfig().configuration; - zkutil::ZooKeeper zk(*config, zkutil::getZookeeperConfigName(*config), nullptr); - zkutil::EventPtr watch = std::make_shared(); - - /// NOTE: setting watches in multiple threads because doing it in a single thread is too slow. - size_t watches_per_thread = std::stoull(argv[2]) / N_THREADS; - std::vector threads; - for (size_t i_thread = 0; i_thread < N_THREADS; ++i_thread) - { - threads.emplace_back([&, i_thread] - { - for (size_t i = 0; i < watches_per_thread; ++i) - zk.exists("/clickhouse/nonexistent_node" + std::to_string(i * N_THREADS + i_thread), nullptr, watch); - }); - } - for (size_t i_thread = 0; i_thread < N_THREADS; ++i_thread) - threads[i_thread].join(); - - while (true) - { - std::cerr << "WAITING..." << std::endl; - sleep(10); - } - } - catch (Poco::Exception & e) - { - std::cerr << "Exception: " << e.displayText() << std::endl; - return 1; - } - catch (std::exception & e) - { - std::cerr << "std::exception: " << e.what() << std::endl; - return 3; - } - catch (...) - { - std::cerr << "Some exception" << std::endl; - return 2; - } -} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9c133a60ea6..e51a831684f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2360,7 +2360,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const const auto & config = shared->zookeeper_config ? *shared->zookeeper_config : getConfigRef(); if (!shared->zookeeper) - shared->zookeeper = std::make_shared(config, zkutil::getZookeeperConfigName(config), getZooKeeperLog()); + shared->zookeeper = std::make_shared(config, zkutil::getZooKeeperConfigName(config), getZooKeeperLog()); else if (shared->zookeeper->expired()) { Stopwatch watch; @@ -2399,9 +2399,9 @@ bool Context::tryCheckClientConnectionToMyKeeperCluster() const { try { + const auto config_name = zkutil::getZooKeeperConfigName(getConfigRef()); /// If our server is part of main Keeper cluster - if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper") || checkZooKeeperConfigIsLocal(getConfigRef(), "keeper") - || (!getConfigRef().has("zookeeper") && !getConfigRef().has("keeper") && getConfigRef().has("keeper_server"))) + if (config_name == "keeper_server" || checkZooKeeperConfigIsLocal(getConfigRef(), config_name)) { LOG_DEBUG(shared->log, "Keeper server is participant of the main zookeeper cluster, will try to connect to it"); getZooKeeper(); @@ -2600,7 +2600,7 @@ void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper_config = config; - reloadZooKeeperIfChangedImpl(config, zkutil::getZookeeperConfigName(*config), shared->zookeeper, getZooKeeperLog()); + reloadZooKeeperIfChangedImpl(config, zkutil::getZooKeeperConfigName(*config), shared->zookeeper, getZooKeeperLog()); } void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config) diff --git a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp index e1faa67eb45..9e2b2a83b98 100644 --- a/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp +++ b/src/Storages/examples/get_abandonable_lock_in_all_partitions.cpp @@ -26,7 +26,7 @@ try auto config = processor.loadConfig().configuration; String root_path = argv[2]; - zkutil::ZooKeeper zk(*config, zkutil::getZookeeperConfigName(*config), nullptr); + zkutil::ZooKeeper zk(*config, zkutil::getZooKeeperConfigName(*config), nullptr); String temp_path = root_path + "/temp"; String blocks_path = root_path + "/block_numbers"; diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index 9ba75b81440..d77b0f5177d 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -29,7 +29,7 @@ try auto config = processor.loadConfig().configuration; String zookeeper_path = argv[2]; - auto zookeeper = std::make_shared(*config, zkutil::getZookeeperConfigName(*config), nullptr); + auto zookeeper = std::make_shared(*config, zkutil::getZooKeeperConfigName(*config), nullptr); std::unordered_map> current_inserts; From d6cbc5d05b34f630f64445105632ccb8b2429470 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Mar 2023 12:58:39 +0000 Subject: [PATCH 049/233] Fix tests --- programs/server/Server.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 8 +- src/Common/ZooKeeper/ZooKeeper.h | 4 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 98 ++++++++++++------- src/Common/ZooKeeper/ZooKeeperArgs.h | 2 +- src/Interpreters/Context.cpp | 2 +- .../helpers/zookeeper_secure_config.xml | 4 +- .../enable_keeper1.xml | 8 ++ .../enable_keeper2.xml | 8 ++ .../configs/enable_keeper3.xml | 41 ++++++++ .../configs/keeper_config.xml | 12 +-- .../configs/remote_servers.xml | 6 +- .../configs/zookeeper_config.xml | 12 +-- .../configs_keeper_server/remote_servers.xml | 18 ---- .../configs_keeper_server/use_keeper.xml | 12 --- .../test_alternative_keeper_config/test.py | 31 ++++-- .../test_keeper_server.py | 61 ------------ .../configs/enable_keeper1.xml | 6 +- .../configs/enable_keeper2.xml | 6 +- .../configs/enable_keeper3.xml | 6 +- .../configs/keeper_config_with_allow_list.xml | 4 +- .../keeper_config_with_allow_list_all.xml | 4 +- .../keeper_config_without_allow_list.xml | 4 +- .../test_keeper_four_word_command/test.py | 9 +- .../configs/enable_keeper1.xml | 2 + .../configs/enable_keeper_three_nodes_1.xml | 2 + .../configs/enable_keeper_three_nodes_2.xml | 2 + .../configs/enable_keeper_three_nodes_3.xml | 2 + .../configs/enable_keeper_two_nodes_1.xml | 2 + .../configs/enable_keeper_two_nodes_2.xml | 2 + .../configs/keeper_config.xml | 2 + .../configs/zookeeper_config_root_a.xml | 4 +- 32 files changed, 204 insertions(+), 182 deletions(-) rename tests/integration/test_alternative_keeper_config/{configs_keeper_server => configs}/enable_keeper1.xml (79%) rename tests/integration/test_alternative_keeper_config/{configs_keeper_server => configs}/enable_keeper2.xml (79%) create mode 100644 tests/integration/test_alternative_keeper_config/configs/enable_keeper3.xml delete mode 100644 tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml delete mode 100644 tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml delete mode 100644 tests/integration/test_alternative_keeper_config/test_keeper_server.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 802bee7fad5..7e120e0fa17 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1308,7 +1308,7 @@ try { /// We do not load ZooKeeper configuration on the first config loading /// because TestKeeper server is not started yet. - if (zkutil::hasZooKeeperConfig(config)) + if (zkutil::hasZooKeeperConfig(*config)) global_context->reloadZooKeeperIfChanged(config); global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index f9d851f9697..c9062d9fd4c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1342,12 +1342,12 @@ void validateZooKeeperConfig(const Poco::Util::AbstractConfiguration & config) throw DB::Exception(DB::ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both ZooKeeper and Keeper are specified"); } -bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server) +bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config) { - return config.has("zookeeper") || config.has("keeper") || (allow_keeper_server && config.has("keeper_server")); + return config.has("zookeeper") || config.has("keeper") || (config.has("keeper_server") && config.getBool("keeper_server.use_cluster", true)); } -String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server) +String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config) { if (config.has("zookeeper")) return "zookeeper"; @@ -1355,7 +1355,7 @@ String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config, if (config.has("keeper")) return "keeper"; - if (allow_keeper_server && config.has("keeper_server")) + if (config.has("keeper_server") && config.getBool("keeper_server.use_cluster", true)) return "keeper_server"; throw DB::Exception(DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 8776497a41d..41f8d110964 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -669,8 +669,8 @@ String getSequentialNodeName(const String & prefix, UInt64 number); void validateZooKeeperConfig(const Poco::Util::AbstractConfiguration & config); -bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server = true); +bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config); -String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config, bool allow_keeper_server = true); +String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config); } diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 3d6d75ba5d5..7cd85db931b 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -19,8 +19,8 @@ namespace zkutil ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, const String & config_name) { - if (endsWith(config_name, "keeper_server")) - initFromKeeperServerSection(config, config_name); + if (config_name == "keeper_server") + initFromKeeperServerSection(config); else initFromKeeperSection(config, config_name); @@ -52,49 +52,79 @@ ZooKeeperArgs::ZooKeeperArgs(const String & hosts_string) splitInto<','>(hosts, hosts_string); } -void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) +void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfiguration & config) { - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_name, keys); + static constexpr std::string_view config_name = "keeper_server"; - bool secure = false; - String tcp_port; - String tcp_port_secure; - for (const auto & key : keys) + if (auto key = std::string{config_name} + ".tcp_port_secure"; + config.has(key)) { - if (key == "tcp_port_secure") - { - secure = true; - tcp_port_secure = config.getString(config_name + "." + key); - } - else if (key == "tcp_port") - { - tcp_port = config.getString(config_name + "." + key); - } - else if (key == "coordination_settings") - { - if (config.has(config_name + "." + key + ".operation_timeout_ms")) - operation_timeout_ms = config.getInt(config_name + "." + key + ".operation_timeout_ms"); - if (config.has(config_name + "." + key + ".session_timeout_ms")) - session_timeout_ms = config.getInt(config_name + "." + key + ".session_timeout_ms"); - } + auto tcp_port_secure = config.getString(key); + + if (tcp_port_secure.empty()) + throw KeeperException("Empty tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); } - if (secure && tcp_port_secure.empty()) - throw KeeperException("No tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); - if (!secure && tcp_port.empty()) - throw KeeperException("No tcp_port in config file", Coordination::Error::ZBADARGUMENTS); + bool secure{false}; + std::string tcp_port; + if (auto tcp_port_secure_key = std::string{config_name} + ".tcp_port_secure"; + config.has(tcp_port_secure_key)) + { + secure = true; + tcp_port = config.getString(tcp_port_secure_key); + } + else if (auto tcp_port_key = std::string{config_name} + ".tcp_port"; + config.has(tcp_port_key)) + { + tcp_port = config.getString(tcp_port_key); + } - config.keys(config_name + ".raft_configuration", keys); + if (tcp_port.empty()) + throw KeeperException("No tcp_port or tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); + + if (auto coordination_key = std::string{config_name} + ".coordination_settings"; + config.has(coordination_key)) + { + if (auto operation_timeout_key = coordination_key + ".operation_timeout_ms"; + config.has(operation_timeout_key)) + operation_timeout_ms = config.getInt(operation_timeout_key); + + if (auto session_timeout_key = coordination_key + ".session_timeout_ms"; + config.has(session_timeout_key)) + session_timeout_key = config.getInt(session_timeout_key); + } + + Poco::Util::AbstractConfiguration::Keys keys; + std::string raft_configuration_key = std::string{config_name} + ".raft_configuration"; + config.keys(raft_configuration_key, keys); for (const auto & key : keys) { if (startsWith(key, "server")) - { hosts.push_back( - (secure ? "secure://" : "") + config.getString(config_name + ".raft_configuration." + key + ".hostname") + ":" - + (secure ? tcp_port_secure : tcp_port)); + (secure ? "secure://" : "") + config.getString(raft_configuration_key + "." + key + ".hostname") + ":" + tcp_port); + } + + static constexpr std::array load_balancing_keys + { + ".zookeeper_load_balancing", + ".keeper_load_balancing" + }; + + for (const auto * load_balancing_key : load_balancing_keys) + { + if (auto load_balancing_config = std::string{config_name} + load_balancing_key; + config.has(load_balancing_config)) + { + String load_balancing_str = config.getString(load_balancing_config); + /// Use magic_enum to avoid dependency from dbms (`SettingFieldLoadBalancingTraits::fromString(...)`) + auto load_balancing = magic_enum::enum_cast(Poco::toUpper(load_balancing_str)); + if (!load_balancing) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknown load balancing: {}", load_balancing_str); + get_priority_load_balancing.load_balancing = *load_balancing; + break; } } + } void ZooKeeperArgs::initFromKeeperSection(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) @@ -144,7 +174,7 @@ void ZooKeeperArgs::initFromKeeperSection(const Poco::Util::AbstractConfiguratio { implementation = config.getString(config_name + "." + key); } - else if (key == "zookeeper_load_balancing") + else if (key == "zookeeper_load_balancing" || key == "keeper_load_balancing") { String load_balancing_str = config.getString(config_name + "." + key); /// Use magic_enum to avoid dependency from dbms (`SettingFieldLoadBalancingTraits::fromString(...)`) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index fa97f8b860f..3f33721804f 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -34,7 +34,7 @@ struct ZooKeeperArgs DB::GetPriorityForLoadBalancing get_priority_load_balancing; private: - void initFromKeeperServerSection(const Poco::Util::AbstractConfiguration & config, const std::string & config_name); + void initFromKeeperServerSection(const Poco::Util::AbstractConfiguration & config); void initFromKeeperSection(const Poco::Util::AbstractConfiguration & config, const std::string & config_name); }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e51a831684f..107adc3ac57 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2624,7 +2624,7 @@ void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & bool Context::hasZooKeeper() const { - return getConfigRef().has("zookeeper") || getConfigRef().has("keeper") || getConfigRef().has("keeper_server"); + return zkutil::hasZooKeeperConfig(getConfigRef()); } bool Context::hasAuxiliaryZooKeeper(const String & name) const diff --git a/tests/integration/helpers/zookeeper_secure_config.xml b/tests/integration/helpers/zookeeper_secure_config.xml index 349bcbc5eba..3e3706988ab 100644 --- a/tests/integration/helpers/zookeeper_secure_config.xml +++ b/tests/integration/helpers/zookeeper_secure_config.xml @@ -1,5 +1,5 @@ - + zoo1 2281 @@ -13,5 +13,5 @@ 2281 15000 - + diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml b/tests/integration/test_alternative_keeper_config/configs/enable_keeper1.xml similarity index 79% rename from tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml rename to tests/integration/test_alternative_keeper_config/configs/enable_keeper1.xml index 7c2e283e89f..fbdece06085 100644 --- a/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper1.xml +++ b/tests/integration/test_alternative_keeper_config/configs/enable_keeper1.xml @@ -28,6 +28,14 @@ true 2 + + 3 + node3 + 9234 + true + true + 3 + diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml b/tests/integration/test_alternative_keeper_config/configs/enable_keeper2.xml similarity index 79% rename from tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml rename to tests/integration/test_alternative_keeper_config/configs/enable_keeper2.xml index 618e6a04aec..dc3ce6c30c4 100644 --- a/tests/integration/test_alternative_keeper_config/configs_keeper_server/enable_keeper2.xml +++ b/tests/integration/test_alternative_keeper_config/configs/enable_keeper2.xml @@ -28,6 +28,14 @@ true 2 + + 3 + node3 + 9234 + true + true + 3 + diff --git a/tests/integration/test_alternative_keeper_config/configs/enable_keeper3.xml b/tests/integration/test_alternative_keeper_config/configs/enable_keeper3.xml new file mode 100644 index 00000000000..af2566565e4 --- /dev/null +++ b/tests/integration/test_alternative_keeper_config/configs/enable_keeper3.xml @@ -0,0 +1,41 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + 75 + trace + + + + + 1 + node1 + 9234 + true + 3 + + + 2 + node2 + 9234 + true + true + 2 + + + 3 + node3 + 9234 + true + true + 3 + + + + diff --git a/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml b/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml index bd783b83853..b62e2728085 100644 --- a/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml +++ b/tests/integration/test_alternative_keeper_config/configs/keeper_config.xml @@ -1,16 +1,16 @@ - zoo1 - 2181 + node1 + 9181 - zoo2 - 2181 + node2 + 9181 - zoo3 - 2181 + node3 + 9181 3000 diff --git a/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml b/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml index e77cc5c65e6..5b453fdeb67 100644 --- a/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml +++ b/tests/integration/test_alternative_keeper_config/configs/remote_servers.xml @@ -6,12 +6,14 @@ node1 9000 - node2 9000 - + + node3 + 9000 + diff --git a/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml b/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml index 7a0d7c1de92..31913bb6e2c 100644 --- a/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml +++ b/tests/integration/test_alternative_keeper_config/configs/zookeeper_config.xml @@ -1,16 +1,16 @@ - zoo1 - 2181 + node1 + 9181 - zoo2 - 2181 + node2 + 9181 - zoo3 - 2181 + node3 + 9181 3000 diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml b/tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml deleted file mode 100644 index e77cc5c65e6..00000000000 --- a/tests/integration/test_alternative_keeper_config/configs_keeper_server/remote_servers.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - - - - node1 - 9000 - - - - node2 - 9000 - - - - - - diff --git a/tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml b/tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml deleted file mode 100644 index b250f06cf81..00000000000 --- a/tests/integration/test_alternative_keeper_config/configs_keeper_server/use_keeper.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - diff --git a/tests/integration/test_alternative_keeper_config/test.py b/tests/integration/test_alternative_keeper_config/test.py index 8784cb9be56..d2cfc4fe25e 100644 --- a/tests/integration/test_alternative_keeper_config/test.py +++ b/tests/integration/test_alternative_keeper_config/test.py @@ -8,18 +8,30 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/keeper_config.xml"], + main_configs=[ + "configs/remote_servers.xml", + "configs/keeper_config.xml", + "configs/enable_keeper1.xml", + ], macros={"replica": "node1"}, ) node2 = cluster.add_instance( "node2", - with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config.xml"], + main_configs=[ + "configs/remote_servers.xml", + "configs/zookeeper_config.xml", + "configs/enable_keeper2.xml", + ], macros={"replica": "node2"}, ) +node3 = cluster.add_instance( + "node3", + main_configs=["configs/remote_servers.xml", "configs/enable_keeper3.xml"], + macros={"replica": "node3"}, +) + @pytest.fixture(scope="module", autouse=True) def started_cluster(): @@ -45,10 +57,9 @@ def test_create_insert(started_cluster): node1.query("INSERT INTO tbl VALUES (1, 'str1')") node2.query("INSERT INTO tbl VALUES (1, 'str1')") # Test deduplication - node2.query("INSERT INTO tbl VALUES (2, 'str2')") + node3.query("INSERT INTO tbl VALUES (2, 'str2')") - expected = [[1, "str1"], [2, "str2"]] - assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) - assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) - assert node1.query("CHECK TABLE tbl") == "1\n" - assert node2.query("CHECK TABLE tbl") == "1\n" + for node in [node1, node2, node3]: + expected = [[1, "str1"], [2, "str2"]] + assert node.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) + assert node.query("CHECK TABLE tbl") == "1\n" diff --git a/tests/integration/test_alternative_keeper_config/test_keeper_server.py b/tests/integration/test_alternative_keeper_config/test_keeper_server.py deleted file mode 100644 index 9c61e076671..00000000000 --- a/tests/integration/test_alternative_keeper_config/test_keeper_server.py +++ /dev/null @@ -1,61 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -cluster = ClickHouseCluster(__file__) - -node1 = cluster.add_instance( - "node1", - with_zookeeper=True, - main_configs=[ - "configs_keeper_server/remote_servers.xml", - "configs_keeper_server/enable_keeper1.xml", - "configs_keeper_server/use_keeper.xml", - ], - macros={"replica": "node1"}, -) - -node2 = cluster.add_instance( - "node2", - with_zookeeper=True, - main_configs=[ - "configs_keeper_server/remote_servers.xml", - "configs_keeper_server/enable_keeper2.xml", - ], - macros={"replica": "node2"}, -) - - -@pytest.fixture(scope="module", autouse=True) -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_create_insert(started_cluster): - node1.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'test_cluster' NO DELAY") - node1.query( - """ - CREATE TABLE tbl ON CLUSTER 'test_cluster' ( - id Int64, - str String - ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') - ORDER BY id - """ - ) - - node1.query("INSERT INTO tbl VALUES (1, 'str1')") - node2.query("INSERT INTO tbl VALUES (1, 'str1')") # Test deduplication - node2.query("INSERT INTO tbl VALUES (2, 'str2')") - - expected = [[1, "str1"], [2, "str2"]] - assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) - assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) - assert node1.query("CHECK TABLE tbl") == "1\n" - assert node2.query("CHECK TABLE tbl") == "1\n" diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml index a686c96e426..0ec413ac2ec 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml @@ -1,5 +1,7 @@ - + + false + 9181 1 /var/lib/clickhouse/coordination/log @@ -39,4 +41,4 @@ - + diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml index 9818d32a74a..fde345f67b3 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml @@ -1,5 +1,7 @@ - + + false + 9181 2 /var/lib/clickhouse/coordination/log @@ -39,4 +41,4 @@ - + diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml index 5a883fac3f6..84a8a402b46 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml @@ -1,5 +1,7 @@ - + + false + 9181 3 /var/lib/clickhouse/coordination/log @@ -39,4 +41,4 @@ - + diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list.xml index feafd3f6b44..b06f845ecf7 100644 --- a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list.xml +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list.xml @@ -1,4 +1,4 @@ - + 9181 1 @@ -21,4 +21,4 @@ - + diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list_all.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list_all.xml index 523e6b2fa27..46c2681c581 100644 --- a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list_all.xml +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_allow_list_all.xml @@ -1,4 +1,4 @@ - + 9181 3 @@ -21,4 +21,4 @@ - + diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_allow_list.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_allow_list.xml index 891f8a2ec12..cd5dea882af 100644 --- a/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_allow_list.xml +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_allow_list.xml @@ -1,4 +1,4 @@ - + 9181 2 @@ -20,4 +20,4 @@ - + diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 412780c8f0f..df333ec479e 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -1,14 +1,7 @@ -import socket import pytest from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils -import random -import string -import os import time -from multiprocessing.dummy import Pool -from helpers.test_tools import assert_eq_with_retry -from io import StringIO import csv import re @@ -23,7 +16,7 @@ node3 = cluster.add_instance( "node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True ) -from kazoo.client import KazooClient, KazooState +from kazoo.client import KazooClient def wait_nodes(): diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml index c1d38a1de52..03307e912f6 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml @@ -1,5 +1,7 @@ + false + 9181 1 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml index d2717283a8d..39a60afffec 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml @@ -1,5 +1,7 @@ + false + 9181 1 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml index 5924ee1c2dc..0f2d5ff912b 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml @@ -1,5 +1,7 @@ + false + 9181 2 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml index d261e4f67f3..f5061fe0f36 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml @@ -1,5 +1,7 @@ + false + 9181 3 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml index 697986638d7..57585080e0f 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml @@ -1,5 +1,7 @@ + false + 9181 1 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml index 967940e1e2b..35493e22270 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml @@ -1,5 +1,7 @@ + false + 9181 2 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml index 9b50f2c6c41..38ef4295b01 100644 --- a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml +++ b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml @@ -1,5 +1,7 @@ + false + 9181 1 /var/lib/clickhouse/coordination/logs diff --git a/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml b/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml index d3c62862002..6c413378524 100644 --- a/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml +++ b/tests/integration/test_zookeeper_config/configs/zookeeper_config_root_a.xml @@ -1,5 +1,5 @@ - + zoo1 2181 @@ -14,5 +14,5 @@ 3000 /root_a - + From 28ca2de86d3982b1b16aa292d5df6fe75ecda7d3 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 24 Mar 2023 14:58:50 +0000 Subject: [PATCH 050/233] fix --- src/Common/ThreadStatus.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 91c695216a8..16ce73cda20 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -223,7 +223,7 @@ void ThreadStatus::updatePerformanceCountersIfNeeded() UInt64 total_elapsed_microseconds = stopwatch.elapsedMicroseconds(); if (last_performance_counters_update_time + performance_counters_update_period_microseconds < total_elapsed_microseconds) { - CurrentThread::updatePerformanceCounters(); + updatePerformanceCounters(); last_performance_counters_update_time = total_elapsed_microseconds; } } From 98c9b1f75cc9452bedc129c3e41fbe794f42a0b1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 24 Mar 2023 15:09:27 +0000 Subject: [PATCH 051/233] Automatic style fix --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- 50 files changed, 23 insertions(+), 57 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer From 86b40a1b792e493666a7ad7998bcd6a53e1569fe Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Mar 2023 10:03:40 +0000 Subject: [PATCH 052/233] address PR review --- src/Analyzer/Passes/CNF.cpp | 25 ++++++++++++------- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 3 ++- src/Analyzer/Passes/ConvertQueryToCNFPass.h | 6 ++--- src/Analyzer/QueryTreePassManager.cpp | 2 +- src/Interpreters/ComparisonGraph.h | 4 +-- 5 files changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Analyzer/Passes/CNF.cpp b/src/Analyzer/Passes/CNF.cpp index 68a95a509f8..f53d833c107 100644 --- a/src/Analyzer/Passes/CNF.cpp +++ b/src/Analyzer/Passes/CNF.cpp @@ -110,8 +110,10 @@ private: class PushNotVisitor { public: - explicit PushNotVisitor(ContextPtr context) - : current_context(std::move(context)) + explicit PushNotVisitor(const ContextPtr & context) + : not_function_resolver(FunctionFactory::instance().get("not", context)) + , or_function_resolver(FunctionFactory::instance().get("or", context)) + , and_function_resolver(FunctionFactory::instance().get("and", context)) {} void visit(QueryTreeNodePtr & node, bool add_negation) @@ -123,7 +125,7 @@ public: if (!function_node || !isLogicalFunction(*function_node)) { if (add_negation) - node = createFunctionNode(FunctionFactory::instance().get("not", current_context), std::move(node)); + node = createFunctionNode(not_function_resolver, std::move(node)); return; } @@ -132,8 +134,10 @@ public: { if (add_negation) { - auto function_resolver = FunctionFactory::instance().get(function_name == "and" ? "or" : "and", current_context); - function_node->resolveAsFunction(function_resolver); + if (function_name == "and") + function_node->resolveAsFunction(or_function_resolver); + else + function_node->resolveAsFunction(and_function_resolver); } auto & arguments = function_node->getArguments().getNodes(); @@ -150,7 +154,9 @@ public: } private: - ContextPtr current_context; + const FunctionOverloadResolverPtr not_function_resolver; + const FunctionOverloadResolverPtr or_function_resolver; + const FunctionOverloadResolverPtr and_function_resolver; }; class PushOrVisitor @@ -303,9 +309,10 @@ bool CNF::AtomicFormula::operator==(const AtomicFormula & rhs) const bool CNF::AtomicFormula::operator<(const AtomicFormula & rhs) const { - return node_with_hash.hash == rhs.node_with_hash.hash - ? negative < rhs.negative - : node_with_hash.hash < rhs.node_with_hash.hash; + if (node_with_hash.hash > rhs.node_with_hash.hash) + return false; + + return node_with_hash.hash < rhs.node_with_hash.hash || negative < rhs.negative; } std::string CNF::dump() const diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index d5d11a19ffe..5998237f3ce 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -711,6 +711,7 @@ public: optimize_filter(query_node->getWhere()); optimize_filter(query_node->getPrewhere()); + optimize_filter(query_node->getHaving()); if (has_filter && settings.optimize_substitute_columns) substituteColumns(*query_node, table_expressions, context); @@ -719,7 +720,7 @@ public: } -void ConvertQueryToCNFPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +void ConvertLogicalExpressionToCNFPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { const auto & settings = context->getSettingsRef(); if (!settings.convert_query_to_cnf) diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.h b/src/Analyzer/Passes/ConvertQueryToCNFPass.h index 232af3b015e..5ed874db006 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.h +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.h @@ -5,12 +5,12 @@ namespace DB { -class ConvertQueryToCNFPass final : public IQueryTreePass +class ConvertLogicalExpressionToCNFPass final : public IQueryTreePass { public: - String getName() override { return "ConvertQueryToCNFPass"; } + String getName() override { return "ConvertLogicalExpressionToCNFPass"; } - String getDescription() override { return "Convert query to CNF and apply optimizations using constraints"; } + String getDescription() override { return "Convert logical expression to CNF and apply optimizations using constraints"; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a9b812d71b4..52fce8abaf2 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -233,7 +233,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index ecfc617ac8a..70543227b58 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -4,8 +4,8 @@ #include #include -#include "Analyzer/HashUtils.h" -#include "Analyzer/IQueryTreeNode.h" +#include +#include #include #include From fa8ea85f3d31dcc1a6cd5913af3ae103e00045f7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Mar 2023 13:07:24 +0000 Subject: [PATCH 053/233] Make custom key work with new analyzer --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Planner/Planner.cpp | 74 +-------- src/Planner/PlannerExpressionAnalysis.cpp | 37 +++-- src/Planner/PlannerExpressionAnalysis.h | 6 - src/Planner/PlannerJoinTree.cpp | 164 ++++++++++++++------ 5 files changed, 143 insertions(+), 142 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 273d81ff9f9..e471f279a14 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -518,15 +518,13 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); ASTPtr parallel_replicas_custom_filter_ast = nullptr; - if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY && !joined_tables.tablesWithColumns().empty()) + if (storage && context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY && !joined_tables.tablesWithColumns().empty()) { if (settings.parallel_replicas_count > 1) { if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *context)) { LOG_TRACE(log, "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); - if (!storage) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( settings.parallel_replicas_count, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c233941b6b5..e38f460e7c5 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1136,67 +1136,17 @@ void Planner::buildPlanForQueryNode() collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); - const auto & table_expression_data = planner_context->getTableExpressionNodeToData(); - auto & mutable_context = planner_context->getMutableQueryContext(); - - QueryTreeNodePtr parallel_replicas_custom_filter_node{nullptr}; - if (table_expression_data.size() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + if (planner_context->getTableExpressionNodeToData().size() > 1 + && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) { - LOG_WARNING(&Poco::Logger::get("Planner"), "Joins are not supported with parallel replicas. Query will be executed without using them."); + LOG_WARNING( + &Poco::Logger::get("Planner"), "Joins are not supported with parallel replicas. Query will be executed without using them."); + + auto & mutable_context = planner_context->getMutableQueryContext(); mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); mutable_context->setSetting("parallel_replicas_custom_key", String{""}); } - else if (table_expression_data.size() == 1 && !settings.parallel_replicas_custom_key.value.empty()) - { - const auto & table_expression = (*table_expression_data.begin()).first; - - StoragePtr storage{nullptr}; - if (const auto * table_node = table_expression->as()) - storage = table_node->getStorage(); - else if (const auto * table_function_node = table_expression->as()) - storage = table_function_node->getStorage(); - - std::cout << "COUNT: " << settings.parallel_replicas_count << std::endl; - if (settings.parallel_replicas_count > 1) - { - if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *query_context)) - { - LOG_TRACE(&Poco::Logger::get("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); - if (!storage) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); - - auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( - settings.parallel_replicas_count, - settings.parallel_replica_offset, - std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, - *storage, - query_context); - - parallel_replicas_custom_filter_node = buildQueryTree(parallel_replicas_custom_filter_ast, query_context); - QueryAnalysisPass pass(table_expression); - pass.run(parallel_replicas_custom_filter_node, query_context); - } - else if (settings.parallel_replica_offset > 0) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Parallel replicas processing with custom_key has been requested " - "(setting 'max_parallel_replicas') but the table does not have custom_key defined for it " - "or it's invalid (settings `parallel_replicas_custom_key`)"); - } - } - else if (storage) - { - if (auto * distributed = dynamic_cast(storage.get()); - distributed && canUseCustomKey(settings, *distributed->getCluster(), *query_context)) - { - select_query_info.use_custom_key = true; - mutable_context->setSetting("distributed_group_by_no_merge", 2); - } - } - } auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, @@ -1216,15 +1166,6 @@ void Planner::buildPlanForQueryNode() if (select_query_options.to_stage == QueryProcessingStage::FetchColumns) return; - std::optional parallel_replicas_custom_filter_info; - if (parallel_replicas_custom_filter_node) - { - ActionsChain chain; - parallel_replicas_custom_filter_info = analyzeFilter(parallel_replicas_custom_filter_node, query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), planner_context, chain); - parallel_replicas_custom_filter_info->remove_filter_column = true; - select_query_info.filter_asts.push_back(parallel_replicas_custom_filter_node->toAST()); - } - PlannerQueryProcessingInfo query_processing_info(from_stage, select_query_options.to_stage); QueryAnalysisResult query_analysis_result(query_tree, query_processing_info, planner_context); auto expression_analysis_result = buildExpressionAnalysisResult(query_tree, @@ -1262,9 +1203,6 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasWhere()) addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute); - if (parallel_replicas_custom_filter_info) - addFilterStep(query_plan, *parallel_replicas_custom_filter_info, "Parallel replica custom key filter", result_actions_to_execute); - if (expression_analysis_result.hasAggregation()) { const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index cd03346416e..b6b6b2e2d85 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -29,6 +29,24 @@ namespace ErrorCodes namespace { +/** Construct filter analysis result for filter expression node + * Actions before filter are added into into actions chain. + * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. + */ +FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, + const ColumnsWithTypeAndName & input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + FilterAnalysisResult result; + + result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); + result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; + actions_chain.addStep(std::make_unique(result.filter_actions)); + + return result; +} + /** Construct aggregation analysis result if query tree has GROUP BY or aggregates. * Actions before aggregation are added into actions chain, if result is not null optional. */ @@ -450,25 +468,6 @@ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, } -/** Construct filter analysis result for filter expression node - * Actions before filter are added into into actions chain. - * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. - */ -FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, - const ColumnsWithTypeAndName & input_columns, - const PlannerContextPtr & planner_context, - ActionsChain & actions_chain) -{ - FilterAnalysisResult result; - - result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); - result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; - actions_chain.addStep(std::make_unique(result.filter_actions)); - - return result; -} - - PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNodePtr & query_tree, const ColumnsWithTypeAndName & join_tree_input_columns, const PlannerContextPtr & planner_context, diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index ddf9f120cdb..792cfdec2ff 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -7,7 +7,6 @@ #include -#include #include #include #include @@ -170,11 +169,6 @@ private: LimitByAnalysisResult limit_by_analysis_result; }; -FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, - const ColumnsWithTypeAndName & input_columns, - const PlannerContextPtr & planner_context, - ActionsChain & actions_chain); - /// Build expression analysis result for query tree, join tree input columns and planner context PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNodePtr & query_tree, const ColumnsWithTypeAndName & join_tree_input_columns, diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 79005612b0a..a18ebbe8f9e 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,6 +1,7 @@ #include #include +#include "Storages/SelectQueryInfo.h" #include @@ -17,6 +18,7 @@ #include #include +#include #include #include @@ -47,6 +49,7 @@ #include #include #include +#include #include #include @@ -381,6 +384,46 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end()); } +FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, + SelectQueryInfo & table_expression_query_info, + PlannerContextPtr & planner_context) +{ + const auto & query_context = planner_context->getQueryContext(); + + auto filter_query_tree = buildQueryTree(filter_expression, query_context); + + QueryAnalysisPass query_analysis_pass(table_expression_query_info.table_expression); + query_analysis_pass.run(filter_query_tree, query_context); + + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression_query_info.table_expression); + const auto table_expression_names = table_expression_data.getColumnNames(); + NameSet table_expression_required_names_without_filter(table_expression_names.begin(), table_expression_names.end()); + + collectSourceColumns(filter_query_tree, planner_context); + collectSets(filter_query_tree, *planner_context); + + auto filter_actions_dag = std::make_shared(); + + PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); + auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree); + if (expression_nodes.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Filter actions must return single output node. Actual {}", + expression_nodes.size()); + + auto & filter_actions_outputs = filter_actions_dag->getOutputs(); + filter_actions_outputs = std::move(expression_nodes); + + std::string filter_node_name = filter_actions_outputs[0]->result_name; + bool remove_filter_column = true; + + for (const auto & filter_input_node : filter_actions_dag->getInputs()) + if (table_expression_required_names_without_filter.contains(filter_input_node->result_name)) + filter_actions_outputs.push_back(filter_input_node); + + return {std::move(filter_actions_dag), std::move(filter_node_name), remove_filter_column}; +} + FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context) @@ -392,38 +435,38 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - auto row_policy_filter_query_tree = buildQueryTree(row_policy_filter->expression, query_context); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info, planner_context); +} - QueryAnalysisPass query_analysis_pass(table_expression_query_info.table_expression); - query_analysis_pass.run(row_policy_filter_query_tree, query_context); +FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, + SelectQueryInfo & table_expression_query_info, + PlannerContextPtr & planner_context) +{ + const auto & query_context = planner_context->getQueryContext(); + const auto & settings = query_context->getSettingsRef(); - auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression_query_info.table_expression); - const auto table_expression_names = table_expression_data.getColumnNames(); - NameSet table_expression_required_names_without_row_policy(table_expression_names.begin(), table_expression_names.end()); + if (settings.parallel_replicas_count <= 1 || settings.parallel_replicas_custom_key.value.empty()) + return {}; - collectSourceColumns(row_policy_filter_query_tree, planner_context); - collectSets(row_policy_filter_query_tree, *planner_context); + auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *query_context); + if (!custom_key_ast) + throw DB::Exception( + ErrorCodes::BAD_ARGUMENTS, + "Parallel replicas processing iwth custom_key has been requested " + "(setting 'max_parallel_replcias'), but the table does not have custom_key defined for it " + " or it's invalid (setting 'parallel_replicas_custom_key')"); - auto row_policy_actions_dag = std::make_shared(); + LOG_TRACE(&Poco::Logger::get("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); - PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = actions_visitor.visit(row_policy_actions_dag, row_policy_filter_query_tree); - if (expression_nodes.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Row policy filter actions must return single output node. Actual {}", - expression_nodes.size()); + auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + settings.parallel_replicas_custom_key_filter_type, + *storage, + query_context); - auto & row_policy_actions_outputs = row_policy_actions_dag->getOutputs(); - row_policy_actions_outputs = std::move(expression_nodes); - - std::string filter_node_name = row_policy_actions_outputs[0]->result_name; - bool remove_filter_column = true; - - for (const auto & row_policy_input_node : row_policy_actions_dag->getInputs()) - if (table_expression_required_names_without_row_policy.contains(row_policy_input_node->result_name)) - row_policy_actions_outputs.push_back(row_policy_input_node); - - return {std::move(row_policy_actions_dag), std::move(filter_node_name), remove_filter_column}; + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info, planner_context); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, @@ -596,11 +639,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres updatePrewhereOutputsIfNeeded(table_expression_query_info, table_expression_data.getColumnNames(), storage_snapshot); - auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); - bool moved_row_policy_to_prewhere = false; + const auto & columns_names = table_expression_data.getColumnNames(); - if (row_policy_filter_info.actions) + std::vector> where_filters; + const auto add_filter = [&](const FilterDAGInfo & filter_info, std::string description) { + if (!filter_info.actions) + return; + bool is_final = table_expression_query_info.table_expression_modifiers && table_expression_query_info.table_expression_modifiers->hasFinal(); bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); @@ -612,36 +658,62 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!table_expression_query_info.prewhere_info->prewhere_actions) { - table_expression_query_info.prewhere_info->prewhere_actions = row_policy_filter_info.actions; - table_expression_query_info.prewhere_info->prewhere_column_name = row_policy_filter_info.column_name; - table_expression_query_info.prewhere_info->remove_prewhere_column = row_policy_filter_info.do_remove_column; + table_expression_query_info.prewhere_info->prewhere_actions = filter_info.actions; + table_expression_query_info.prewhere_info->prewhere_column_name = filter_info.column_name; + table_expression_query_info.prewhere_info->remove_prewhere_column = filter_info.do_remove_column; } else { - table_expression_query_info.prewhere_info->row_level_filter = row_policy_filter_info.actions; - table_expression_query_info.prewhere_info->row_level_column_name = row_policy_filter_info.column_name; + table_expression_query_info.prewhere_info->row_level_filter = filter_info.actions; + table_expression_query_info.prewhere_info->row_level_column_name = filter_info.column_name; } table_expression_query_info.prewhere_info->need_filter = true; - moved_row_policy_to_prewhere = true; + } + else + { + where_filters.emplace_back(filter_info, std::move(description)); + } + }; + + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); + add_filter(row_policy_filter_info, "Row-level security filter"); + + if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) + { + if (settings.parallel_replicas_count > 1) + { + auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); + add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); + } + else + { + if (auto * distributed = dynamic_cast(storage.get()); + distributed && canUseCustomKey(settings, *distributed->getCluster(), *query_context)) + { + table_expression_query_info.use_custom_key = true; + planner_context->getMutableQueryContext()->setSetting("distributed_group_by_no_merge", 2); + } } } - const auto & columns_names = table_expression_data.getColumnNames(); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); - if (query_plan.isInitialized() && - from_stage == QueryProcessingStage::FetchColumns && - row_policy_filter_info.actions && - !moved_row_policy_to_prewhere) + for (const auto & filter_info_and_description : where_filters) { - auto row_level_filter_step = std::make_unique(query_plan.getCurrentDataStream(), - row_policy_filter_info.actions, - row_policy_filter_info.column_name, - row_policy_filter_info.do_remove_column); - row_level_filter_step->setStepDescription("Row-level security filter"); - query_plan.addStep(std::move(row_level_filter_step)); + const auto & [filter_info, description] = filter_info_and_description; + if (query_plan.isInitialized() && + from_stage == QueryProcessingStage::FetchColumns && + filter_info.actions) + { + auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), + filter_info.actions, + filter_info.column_name, + filter_info.do_remove_column); + filter_step->setStepDescription(description); + query_plan.addStep(std::move(filter_step)); + } } if (query_context->hasQueryContext() && !select_query_options.is_internal) From 28182e9489632a09b10fb3dccae05ae0f508e625 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Mar 2023 15:51:42 +0200 Subject: [PATCH 054/233] Fix typo --- src/Planner/PlannerJoinTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a18ebbe8f9e..1741cca17c5 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -452,7 +452,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, if (!custom_key_ast) throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, - "Parallel replicas processing iwth custom_key has been requested " + "Parallel replicas processing with custom_key has been requested " "(setting 'max_parallel_replcias'), but the table does not have custom_key defined for it " " or it's invalid (setting 'parallel_replicas_custom_key')"); From d1a6c1991abc073b9320c3d6b529b596d416b2ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Mar 2023 17:38:52 +0200 Subject: [PATCH 055/233] Only check MV on ALTER when necessary --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- src/Storages/StorageBuffer.cpp | 6 ++++-- src/Storages/StorageDistributed.cpp | 6 ++++-- src/Storages/StorageMerge.cpp | 6 ++++-- src/Storages/StorageNull.cpp | 6 ++++-- .../02697_alter_dependencies.reference | 1 + .../0_stateless/02697_alter_dependencies.sql | 16 ++++++++++++++++ 7 files changed, 38 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02697_alter_dependencies.reference create mode 100644 tests/queries/0_stateless/02697_alter_dependencies.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index acb5ed248c8..52da00ca4a1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2930,7 +2930,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context old_types.emplace(column.name, column.type.get()); NamesAndTypesList columns_to_check_conversion; - auto name_deps = getDependentViewsByColumn(local_context); + + std::optional name_deps{}; for (const AlterCommand & command : commands) { /// Just validate partition expression @@ -3012,7 +3013,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context if (!command.clear) { - const auto & deps_mv = name_deps[command.column_name]; + if (!name_deps) + name_deps = getDependentViewsByColumn(local_context); + const auto & deps_mv = name_deps.value()[command.column_name]; if (!deps_mv.empty()) { throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a69f26203e9..a4cb15d5711 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1016,7 +1016,7 @@ void StorageBuffer::reschedule() void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const { - auto name_deps = getDependentViewsByColumn(local_context); + std::optional name_deps{}; for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN @@ -1027,7 +1027,9 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context if (command.type == AlterCommand::Type::DROP_COLUMN && !command.clear) { - const auto & deps_mv = name_deps[command.column_name]; + if (!name_deps) + name_deps = getDependentViewsByColumn(local_context); + const auto & deps_mv = name_deps.value()[command.column_name]; if (!deps_mv.empty()) { throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0be4ae3a79f..ec2e8fed8ae 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1394,7 +1394,7 @@ std::optional StorageDistributed::distributedWrite(const ASTInser void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const { - auto name_deps = getDependentViewsByColumn(local_context); + std::optional name_deps{}; for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN @@ -1406,7 +1406,9 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co if (command.type == AlterCommand::DROP_COLUMN && !command.clear) { - const auto & deps_mv = name_deps[command.column_name]; + if (!name_deps) + name_deps = getDependentViewsByColumn(local_context); + const auto & deps_mv = name_deps.value()[command.column_name]; if (!deps_mv.empty()) { throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0f7b47255f6..96c40fc28d4 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -919,7 +919,7 @@ StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(Context void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const { - auto name_deps = getDependentViewsByColumn(local_context); + std::optional name_deps{}; for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN @@ -930,7 +930,9 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP if (command.type == AlterCommand::Type::DROP_COLUMN && !command.clear) { - const auto & deps_mv = name_deps[command.column_name]; + if (!name_deps) + name_deps = getDependentViewsByColumn(local_context); + const auto & deps_mv = name_deps.value()[command.column_name]; if (!deps_mv.empty()) { throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index aa462e1a40c..0ced128c8ef 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -37,7 +37,7 @@ void registerStorageNull(StorageFactory & factory) void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const { - auto name_deps = getDependentViewsByColumn(context); + std::optional name_deps{}; for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN @@ -50,7 +50,9 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt if (command.type == AlterCommand::DROP_COLUMN && !command.clear) { - const auto & deps_mv = name_deps[command.column_name]; + if (!name_deps) + name_deps = getDependentViewsByColumn(context); + const auto & deps_mv = name_deps.value()[command.column_name]; if (!deps_mv.empty()) { throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, diff --git a/tests/queries/0_stateless/02697_alter_dependencies.reference b/tests/queries/0_stateless/02697_alter_dependencies.reference new file mode 100644 index 00000000000..d05b1f927f4 --- /dev/null +++ b/tests/queries/0_stateless/02697_alter_dependencies.reference @@ -0,0 +1 @@ +0 0 diff --git a/tests/queries/0_stateless/02697_alter_dependencies.sql b/tests/queries/0_stateless/02697_alter_dependencies.sql new file mode 100644 index 00000000000..cf9b7551b5f --- /dev/null +++ b/tests/queries/0_stateless/02697_alter_dependencies.sql @@ -0,0 +1,16 @@ +CREATE TABLE mv_source (a Int64, insert_time DateTime) ENGINE = MergeTree() ORDER BY insert_time; +CREATE TABLE mv_target (a Int64, insert_time DateTime) ENGINE = MergeTree() ORDER BY insert_time; +CREATE MATERIALIZED VIEW source_to_target to mv_target as Select * from mv_source where a not in (Select sleepEachRow(0.1) from numbers(50)); + +ALTER TABLE mv_source MODIFY TTL insert_time + toIntervalDay(1); +SYSTEM FLUSH LOGS; +-- This is a fancy way to check that the MV hasn't been called (no functions executed by ALTER) +SELECT + ProfileEvents['FunctionExecute'], + ProfileEvents['TableFunctionExecute'] +FROM system.query_log +WHERE + type = 'QueryFinish' AND + query like '%ALTER TABLE mv_source%' AND + current_database = currentDatabase() AND + event_time > now() - INTERVAL 10 minute; From b5fe1ada0be41d70d6b0a823af605616cae6aa6d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Mar 2023 01:39:36 +0200 Subject: [PATCH 056/233] more options for sync replica --- docs/en/sql-reference/statements/system.md | 8 ++- docs/ru/sql-reference/statements/system.md | 8 ++- docs/zh/sql-reference/statements/system.md | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Parsers/ASTSystemQuery.cpp | 5 +- src/Parsers/ASTSystemQuery.h | 3 +- src/Parsers/ParserSystemQuery.cpp | 11 +++- src/Parsers/SyncReplicaMode.h | 12 ++++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 49 +++++++++++------ .../MergeTree/ReplicatedMergeTreeQueue.h | 10 +++- src/Storages/StorageReplicatedMergeTree.cpp | 55 +++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 3 +- ...ed_storage_definition_syntax_zookeeper.sql | 2 +- ...ptimize_on_nonleader_replica_zookeeper.sql | 2 +- .../02370_lost_part_intersecting_merges.sh | 4 +- .../02438_sync_replica_lightweight.reference | 15 +++++ .../02438_sync_replica_lightweight.sql | 39 +++++++++++++ tests/queries/0_stateless/replication.lib | 4 +- 18 files changed, 173 insertions(+), 61 deletions(-) create mode 100644 src/Parsers/SyncReplicaMode.h create mode 100644 tests/queries/0_stateless/02438_sync_replica_lightweight.reference create mode 100644 tests/queries/0_stateless/02438_sync_replica_lightweight.sql diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index d069ae8413a..c2be8aa4485 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -283,10 +283,14 @@ SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a cluster, but no more than `receive_timeout` seconds. ``` sql -SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT] +SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT | PULL] ``` -After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. If a `STRICT` modifier was specified then the query waits for the replication queue to become empty. The `STRICT` version may never succeed if new entries constantly appear in the replication queue. +After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. The following modifiers are supported: + + - If a `STRICT` modifier was specified then the query waits for the replication queue to become empty. The `STRICT` version may never succeed if new entries constantly appear in the replication queue. + - If a `LIGHTWEIGHT` modifier was specified then the query waits only for `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` and `DROP_PART` entries to be processed. + - If a `PULL` modifier was specified then the query pulls new replication queue entries from ZooKeeper, but does not wait for anything to be processed. ### RESTART REPLICA diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 847f60ca35c..22a74648eab 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -272,10 +272,14 @@ SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] Ждет когда таблица семейства `ReplicatedMergeTree` будет синхронизирована с другими репликами в кластере, но не более `receive_timeout` секунд: ``` sql -SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT] +SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT | PULL] ``` -После выполнения этого запроса таблица `[db.]replicated_merge_tree_family_table_name` загружает команды из общего реплицированного лога в свою собственную очередь репликации. Затем запрос ждет, пока реплика не обработает все загруженные команды. Если указан модификатор `STRICT`, то запрос ждёт когда очередь репликации станет пустой. Строгий вариант запроса может никогда не завершиться успешно, если в очереди репликации постоянно появляются новые записи. +После выполнения этого запроса таблица `[db.]replicated_merge_tree_family_table_name` загружает команды из общего реплицированного лога в свою собственную очередь репликации. Затем запрос ждет, пока реплика не обработает все загруженные команды. Поддерживаются следующие модификаторы: + + - Если указан модификатор `STRICT`, то запрос ждёт когда очередь репликации станет пустой. Строгий вариант запроса может никогда не завершиться успешно, если в очереди репликации постоянно появляются новые записи. + - Если указан модификатор `LIGHTWEIGHT`, то запрос ждёт когда будут обработаны записи `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` и `DROP_PART`. + - Если указан модификатор `PULL`, то запрос только загружает записи очереди репликации из ZooKeeper и не ждёт выполнения чего-либо. ### RESTART REPLICA {#query_language-system-restart-replica} diff --git a/docs/zh/sql-reference/statements/system.md b/docs/zh/sql-reference/statements/system.md index 1942d6fb79a..8fd2dd74d26 100644 --- a/docs/zh/sql-reference/statements/system.md +++ b/docs/zh/sql-reference/statements/system.md @@ -240,7 +240,7 @@ SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] ``` sql -SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT | PULL] ``` ### RESTART REPLICA {#query_language-system-restart-replica} diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index fb6b1635f28..90fd184948d 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -888,7 +888,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) { LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for current last entry to be processed"); auto sync_timeout = getContext()->getSettingsRef().receive_timeout.totalMilliseconds(); - if (!storage_replicated->waitForProcessingQueue(sync_timeout, query.strict_sync)) + if (!storage_replicated->waitForProcessingQueue(sync_timeout, query.sync_replica_mode)) { LOG_ERROR(log, "SYNC REPLICA {}: Timed out!", table_id.getNameForLogs()); throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "SYNC REPLICA {}: command timed out. " \ diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index c38604690ca..4bd5cdb5ebe 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -182,8 +182,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, else if (!disk.empty()) print_identifier(disk); - if (strict_sync) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " STRICT" << (settings.hilite ? hilite_none : ""); + if (sync_replica_mode != SyncReplicaMode::DEFAULT) + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << magic_enum::enum_name(sync_replica_mode) + << (settings.hilite ? hilite_none : ""); } else if (type == Type::SYNC_DATABASE_REPLICA) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 2c7d42edebe..e5824911645 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -2,6 +2,7 @@ #include #include +#include #include "config.h" @@ -108,7 +109,7 @@ public: String schema_cache_storage; - bool strict_sync = false; + SyncReplicaMode sync_replica_mode = SyncReplicaMode::DEFAULT; String getID(char) const override { return "SYSTEM query"; } diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 7c8d42b6bce..26819f0ee6c 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -259,8 +259,15 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return false; if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table)) return false; - if (res->type == Type::SYNC_REPLICA && ParserKeyword{"STRICT"}.ignore(pos, expected)) - res->strict_sync = true; + if (res->type == Type::SYNC_REPLICA) + { + if (ParserKeyword{"STRICT"}.ignore(pos, expected)) + res->sync_replica_mode = SyncReplicaMode::STRICT; + else if (ParserKeyword{"LIGHTWEIGHT"}.ignore(pos, expected)) + res->sync_replica_mode = SyncReplicaMode::LIGHTWEIGHT; + else if (ParserKeyword{"PULL"}.ignore(pos, expected)) + res->sync_replica_mode = SyncReplicaMode::PULL; + } break; } diff --git a/src/Parsers/SyncReplicaMode.h b/src/Parsers/SyncReplicaMode.h new file mode 100644 index 00000000000..a98e1cace50 --- /dev/null +++ b/src/Parsers/SyncReplicaMode.h @@ -0,0 +1,12 @@ +#pragma once + +namespace DB +{ +enum class SyncReplicaMode +{ + DEFAULT, + STRICT, + LIGHTWEIGHT, + PULL, +}; +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c0aac96dd31..7cc26c00098 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -544,7 +544,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep if (!found && need_remove_from_zk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't find {} in the memory queue. It is a bug. Entry: {}", entry->znode_name, entry->toString()); - notifySubscribers(queue_size, entry->znode_name); + notifySubscribers(queue_size, &(entry->znode_name)); if (!need_remove_from_zk) return; @@ -2465,38 +2465,55 @@ String ReplicatedMergeTreeMergePredicate::getCoveringVirtualPart(const String & ReplicatedMergeTreeQueue::SubscriberHandler ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback) { - std::lock_guard lock(state_mutex); - std::unordered_set result; - result.reserve(queue.size()); - for (const auto & entry : queue) - result.insert(entry->znode_name); - + std::lock_guard lock(state_mutex); std::lock_guard lock_subscribers(subscribers_mutex); auto it = subscribers.emplace(subscribers.end(), std::move(callback)); - /// Notify queue size & log entry ids to avoid waiting for removed entries - (*it)(result.size(), result, std::nullopt); + /// Atomically notify about current size + (*it)(queue.size(), nullptr); return SubscriberHandler(it, *this); } +std::unordered_set ReplicatedMergeTreeQueue::getEntryNamesSet(bool lightweight_entries_only) +{ + std::lock_guard lock(state_mutex); + std::unordered_set result; + result.reserve(queue.size()); + for (const auto & entry : queue) + { + bool is_lightweight = entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART + || entry->type == LogEntry::DROP_RANGE || entry->type == LogEntry::REPLACE_RANGE || entry->type == LogEntry::DROP_PART; + if (!lightweight_entries_only || is_lightweight) + result.insert(entry->znode_name); + } + return result; +} + +void ReplicatedMergeTreeQueue::notifySubscribersOnPartialShutdown() +{ + size_t queue_size; + { + std::lock_guard lock(state_mutex); + queue_size = queue.size(); + } + std::lock_guard lock_subscribers(subscribers_mutex); + for (auto & subscriber_callback : subscribers) + subscriber_callback(queue_size, nullptr); +} + ReplicatedMergeTreeQueue::SubscriberHandler::~SubscriberHandler() { std::lock_guard lock(queue.subscribers_mutex); queue.subscribers.erase(it); } -void ReplicatedMergeTreeQueue::notifySubscribers(size_t new_queue_size, std::optional removed_log_entry_id) +void ReplicatedMergeTreeQueue::notifySubscribers(size_t new_queue_size, const String * removed_log_entry_id) { std::lock_guard lock_subscribers(subscribers_mutex); for (auto & subscriber_callback : subscribers) - subscriber_callback(new_queue_size, {}, removed_log_entry_id); -} - -ReplicatedMergeTreeQueue::~ReplicatedMergeTreeQueue() -{ - notifySubscribers(0, std::nullopt); + subscriber_callback(new_queue_size, removed_log_entry_id); } String padIndex(Int64 index) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 36552129690..9b29c6cef5b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -164,7 +164,7 @@ private: /// A subscriber callback is called when an entry queue is deleted mutable std::mutex subscribers_mutex; - using SubscriberCallBack = std::function /*wait_for_ids*/, std::optional /* removed_log_entry_id */)>; + using SubscriberCallBack = std::function; using Subscribers = std::list; using SubscriberIterator = Subscribers::iterator; @@ -182,7 +182,7 @@ private: Subscribers subscribers; /// Notify subscribers about queue change (new queue size and entry that was removed) - void notifySubscribers(size_t new_queue_size, std::optional removed_log_entry_id); + void notifySubscribers(size_t new_queue_size, const String * removed_log_entry_id); /// Check that entry_ptr is REPLACE_RANGE entry and can be removed from queue because current entry covers it bool checkReplaceRangeCanBeRemoved( @@ -287,7 +287,7 @@ private: public: ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeMergeStrategyPicker & merge_strategy_picker_); - ~ReplicatedMergeTreeQueue(); + ~ReplicatedMergeTreeQueue() = default; /// Clears queue state void clear(); @@ -427,6 +427,10 @@ public: /// Adds a subscriber SubscriberHandler addSubscriber(SubscriberCallBack && callback); + std::unordered_set getEntryNamesSet(bool lightweight_entries_only); + + void notifySubscribersOnPartialShutdown(); + struct Status { /// TODO: consider using UInt64 here diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 76170032ca5..1450b08a4fb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4415,6 +4415,7 @@ void StorageReplicatedMergeTree::partialShutdown() partial_shutdown_called = true; partial_shutdown_event.set(); + queue.notifySubscribersOnPartialShutdown(); replica_is_active_node = nullptr; LOG_TRACE(log, "Waiting for threads to finish"); @@ -7578,53 +7579,59 @@ void StorageReplicatedMergeTree::onActionLockRemove(StorageActionBlockType actio background_moves_assignee.trigger(); } -bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_milliseconds, bool strict) +bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_milliseconds, SyncReplicaMode sync_mode) { - Stopwatch watch; - /// Let's fetch new log entries firstly queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); + + if (sync_mode == SyncReplicaMode::PULL) + return true; + /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. background_operations_assignee.trigger(); std::unordered_set wait_for_ids; - bool set_ids_to_wait = true; + bool was_interrupted = false; + if (sync_mode == SyncReplicaMode::DEFAULT) + wait_for_ids = queue.getEntryNamesSet(/* lightweight_entries_only */ false); + else if (sync_mode == SyncReplicaMode::LIGHTWEIGHT) + wait_for_ids = queue.getEntryNamesSet(/* lightweight_entries_only */ true); Poco::Event target_entry_event; - auto callback = [&target_entry_event, &wait_for_ids, &set_ids_to_wait, strict] - (size_t new_queue_size, std::unordered_set log_entry_ids, std::optional removed_log_entry_id) + auto callback = [this, &target_entry_event, &wait_for_ids, &was_interrupted, sync_mode] + (size_t new_queue_size, const String * removed_log_entry_id) { - if (strict) + if (partial_shutdown_called) { - /// In strict mode we wait for queue to become empty + was_interrupted = true; + target_entry_event.set(); + return; + } + + if (sync_mode == SyncReplicaMode::STRICT) + { + /// Wait for queue to become empty if (new_queue_size == 0) target_entry_event.set(); return; } - if (set_ids_to_wait) - { - wait_for_ids = log_entry_ids; - set_ids_to_wait = false; - } + if (removed_log_entry_id) + wait_for_ids.erase(*removed_log_entry_id); - if (removed_log_entry_id.has_value()) - wait_for_ids.erase(removed_log_entry_id.value()); - - if (wait_for_ids.empty() || new_queue_size == 0) + chassert(new_queue_size || wait_for_ids.empty()); + if (wait_for_ids.empty()) target_entry_event.set(); }; const auto handler = queue.addSubscriber(std::move(callback)); - while (!target_entry_event.tryWait(50)) - { - if (max_wait_milliseconds && watch.elapsedMilliseconds() > max_wait_milliseconds) - return false; + if (!target_entry_event.tryWait(max_wait_milliseconds)) + return false; + + if (was_interrupted) + throw Exception(ErrorCodes::ABORTED, "Shutdown is called for table"); - if (partial_shutdown_called) - throw Exception(ErrorCodes::ABORTED, "Shutdown is called for table"); - } return true; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 270d4eb68b2..d410350af31 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB @@ -181,7 +182,7 @@ public: /// Wait till replication queue's current last entry is processed or till size becomes 0 /// If timeout is exceeded returns false - bool waitForProcessingQueue(UInt64 max_wait_milliseconds, bool strict); + bool waitForProcessingQueue(UInt64 max_wait_milliseconds, SyncReplicaMode sync_mode); /// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK. void getStatus(ReplicatedTableStatus & res, bool with_zk_fields = true); diff --git a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql index 0aa692d9667..06fada1ccdb 100644 --- a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql +++ b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql @@ -44,7 +44,7 @@ CREATE TABLE replicated_collapsing(d Date, x UInt32, sign Int8) INSERT INTO replicated_collapsing VALUES ('2017-10-23', 1, 1); INSERT INTO replicated_collapsing VALUES ('2017-10-23', 1, -1), ('2017-10-23', 2, 1); -SYSTEM SYNC REPLICA replicated_collapsing; +SYSTEM SYNC REPLICA replicated_collapsing PULL; OPTIMIZE TABLE replicated_collapsing PARTITION 201710 FINAL; SELECT * FROM replicated_collapsing; diff --git a/tests/queries/0_stateless/00620_optimize_on_nonleader_replica_zookeeper.sql b/tests/queries/0_stateless/00620_optimize_on_nonleader_replica_zookeeper.sql index f6c4287b9fc..6c5eb6556ea 100644 --- a/tests/queries/0_stateless/00620_optimize_on_nonleader_replica_zookeeper.sql +++ b/tests/queries/0_stateless/00620_optimize_on_nonleader_replica_zookeeper.sql @@ -12,7 +12,7 @@ CREATE TABLE rename2 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMe INSERT INTO rename1 VALUES (0, 1, 0); INSERT INTO rename1 VALUES (0, 1, 1); -SYSTEM SYNC REPLICA rename1; -- Avoid "Cannot select parts for optimization: Entry for part 0_1_1_0 hasn't been read from the replication log yet" +SYSTEM SYNC REPLICA rename1 PULL; -- Avoid "Cannot select parts for optimization: Entry for part 0_1_1_0 hasn't been read from the replication log yet" SYSTEM SYNC REPLICA rename2; OPTIMIZE TABLE rename1 FINAL; OPTIMIZE TABLE rename2 FINAL; diff --git a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh index db9bd2dd430..bc297cbb963 100755 --- a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh @@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (1);" $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" -$CLICKHOUSE_CLIENT --receive_timeout=3 -q "system sync replica rmt1;" 2>/dev/null 1>/dev/null +$CLICKHOUSE_CLIENT -q "system sync replica rmt1 pull;" # There's a stupid effect from "zero copy replication": # MERGE_PARTS all_1_2_1 can be executed by rmt2 even if it was assigned by rmt1 @@ -46,7 +46,7 @@ $CLICKHOUSE_CLIENT -q "detach table rmt1;" $CLICKHOUSE_CLIENT -q "attach table rmt1;" $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (3);" -$CLICKHOUSE_CLIENT -q "system sync replica rmt1;" +$CLICKHOUSE_CLIENT -q "system sync replica rmt1 pull;" $CLICKHOUSE_CLIENT -q "optimize table rmt1 final;" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" diff --git a/tests/queries/0_stateless/02438_sync_replica_lightweight.reference b/tests/queries/0_stateless/02438_sync_replica_lightweight.reference new file mode 100644 index 00000000000..25abaad13e2 --- /dev/null +++ b/tests/queries/0_stateless/02438_sync_replica_lightweight.reference @@ -0,0 +1,15 @@ +GET_PART all_0_0_0 +GET_PART all_1_1_0 +1 1 all_0_0_0 +1 2 all_1_1_0 +MERGE_PARTS all_0_1_1 +3 1 all_0_1_1 +3 2 all_0_1_1 +4 1 all_0_1_1 +4 2 all_0_1_1 +5 1 all_0_2_2 +5 2 all_0_2_2 +5 3 all_0_2_2 +6 1 all_0_2_2 +6 2 all_0_2_2 +6 3 all_0_2_2 diff --git a/tests/queries/0_stateless/02438_sync_replica_lightweight.sql b/tests/queries/0_stateless/02438_sync_replica_lightweight.sql new file mode 100644 index 00000000000..207571da147 --- /dev/null +++ b/tests/queries/0_stateless/02438_sync_replica_lightweight.sql @@ -0,0 +1,39 @@ +-- Tags: no-replicated-database +-- Tag no-replicated-database: different number of replicas + +create table rmt1 (n int) engine=ReplicatedMergeTree('/test/{database}/02438/', '1') order by tuple(); +create table rmt2 (n int) engine=ReplicatedMergeTree('/test/{database}/02438/', '2') order by tuple(); + +system stop replicated sends rmt1; +system stop merges rmt2; + +insert into rmt1 values (1); +insert into rmt1 values (2); +system sync replica rmt2 pull; -- does not wait +select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2'; +select 1, n, _part from rmt1 order by n; +select 2, n, _part from rmt2 order by n; + +set optimize_throw_if_noop = 1; +system sync replica rmt1 pull; +optimize table rmt1 final; + +system start replicated sends rmt1; +system sync replica rmt2 lightweight; -- waits for fetches, not merges +select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2'; +select 3, n, _part from rmt1 order by n; +select 4, n, _part from rmt2 order by n; + +system start merges rmt2; +system sync replica rmt2; + +insert into rmt2 values (3); +optimize table rmt2 final; + +system sync replica rmt1 strict; + +select 5, n, _part from rmt1 order by n; +select 6, n, _part from rmt2 order by n; + +drop table rmt1; +drop table rmt2; diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 6784fee6395..1805b56f8dc 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -81,9 +81,9 @@ function check_replication_consistency() # Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") - $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||: some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") - $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||: # Forcefully cancel mutations to avoid waiting for them to finish ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null From 71b498061b9540d82ec586770645722da26ce245 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 28 Mar 2023 01:48:25 +0000 Subject: [PATCH 057/233] Fix a bug, review suggestions --- docs/en/sql-reference/functions/ulid-functions.md | 2 +- src/Functions/FunctionsCodingULID.cpp | 13 ++++++++++--- .../0_stateless/02668_ulid_decoding.reference | 2 ++ tests/queries/0_stateless/02668_ulid_decoding.sql | 7 +++++++ 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/ulid-functions.md b/docs/en/sql-reference/functions/ulid-functions.md index 2bff5da1c1a..3d32a851df2 100644 --- a/docs/en/sql-reference/functions/ulid-functions.md +++ b/docs/en/sql-reference/functions/ulid-functions.md @@ -60,7 +60,7 @@ ULIDStringToDateTime(ulid[, timezone]) **Arguments** -- `ulid` — Input UUID. [String](/docs/en/sql-reference/data-types/string.md) or [FixedString(26)](/docs/en/sql-reference/data-types/fixedstring.md). +- `ulid` — Input ULID. [String](/docs/en/sql-reference/data-types/string.md) or [FixedString(26)](/docs/en/sql-reference/data-types/fixedstring.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md). **Returned value** diff --git a/src/Functions/FunctionsCodingULID.cpp b/src/Functions/FunctionsCodingULID.cpp index 07be6f57cac..bc62b2d9aca 100644 --- a/src/Functions/FunctionsCodingULID.cpp +++ b/src/Functions/FunctionsCodingULID.cpp @@ -68,7 +68,8 @@ public: String timezone; if (arguments.size() == 2) { - timezone = extractTimeZoneNameFromColumn(*arguments[1].column); + if (arguments[1].column) + timezone = extractTimeZoneNameFromColumn(*arguments[1].column); if (timezone.empty()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -120,8 +121,14 @@ public: DateTime64 time = 0; size_t string_size = offsets_src[i] - src_offset; - if (string_size == ULID_LENGTH + 1) - time = decode(vec_src.data() + src_offset); + if (string_size != ULID_LENGTH + 1) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}, ULID must be {} characters long", + arguments[0].name, getName(), ULID_LENGTH + ); + + time = decode(vec_src.data() + src_offset); src_offset += string_size; vec_res[i] = time; diff --git a/tests/queries/0_stateless/02668_ulid_decoding.reference b/tests/queries/0_stateless/02668_ulid_decoding.reference index d00491fd7e5..57af48199b4 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.reference +++ b/tests/queries/0_stateless/02668_ulid_decoding.reference @@ -1 +1,3 @@ 1 +2023-03-28 01:16:44.000 +2023-03-27 19:16:44.000 diff --git a/tests/queries/0_stateless/02668_ulid_decoding.sql b/tests/queries/0_stateless/02668_ulid_decoding.sql index 042c8e52b36..62c4a7d4dbe 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.sql +++ b/tests/queries/0_stateless/02668_ulid_decoding.sql @@ -1,3 +1,10 @@ -- Tags: no-fasttest SELECT dateDiff('s', ULIDStringToDateTime(generateULID()), now()) = 0; +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E'); +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Rica'); +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9', 'America/Costa_Rica'); -- { serverError ILLEGAL_COLUMN } +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Ric'); -- { serverError POCO_EXCEPTION } +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E0'); -- { serverError ILLEGAL_COLUMN } +SELECT ULIDStringToDateTime(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT ULIDStringToDateTime(1, 2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 160aa186bbfb066f1bf3beefdce4131578f50b7f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Mar 2023 14:44:34 +0000 Subject: [PATCH 058/233] Add support for NOSIGN keyword and no_sign_request config --- src/Backups/BackupIO_S3.cpp | 17 +++-- src/Coordination/KeeperSnapshotManagerS3.cpp | 10 ++- src/Disks/ObjectStorages/S3/diskSettings.cpp | 10 ++- src/IO/S3/Client.cpp | 8 +- src/IO/S3/Client.h | 4 +- src/IO/S3/Credentials.cpp | 14 ++-- src/IO/S3/Credentials.h | 12 ++- src/IO/S3Common.cpp | 10 ++- src/IO/S3Common.h | 1 + src/Storages/StorageS3.cpp | 72 ++++++++++++++---- src/TableFunctions/TableFunctionS3.cpp | 76 +++++++++++++++---- .../test_invalid_env_credentials.py | 34 +++++---- 12 files changed, 191 insertions(+), 77 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 1ebc7cb3bb9..d1f4a15b552 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -66,12 +66,17 @@ namespace credentials.GetAWSSecretKey(), settings.auth_settings.server_side_encryption_customer_key_base64, std::move(headers), - settings.auth_settings.use_environment_credentials.value_or( - context->getConfigRef().getBool("s3.use_environment_credentials", false)), - settings.auth_settings.use_insecure_imds_request.value_or( - context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - settings.auth_settings.expiration_window_seconds.value_or( - context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); + S3::CredentialsConfiguration + { + settings.auth_settings.use_environment_credentials.value_or( + context->getConfigRef().getBool("s3.use_environment_credentials", false)), + settings.auth_settings.use_insecure_imds_request.value_or( + context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + settings.auth_settings.expiration_window_seconds.value_or( + context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + settings.auth_settings.no_sign_request.value_or( + context->getConfigRef().getBool("s3.no_sign_request", false)), + }); } Aws::Vector listObjects(S3::Client & client, const S3::URI & s3_uri, const String & file_name) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index cabeb13e2f8..db3ff235b67 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -103,9 +103,13 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo credentials.GetAWSSecretKey(), auth_settings.server_side_encryption_customer_key_base64, std::move(headers), - auth_settings.use_environment_credentials.value_or(false), - auth_settings.use_insecure_imds_request.value_or(false), - auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)); + S3::CredentialsConfiguration + { + auth_settings.use_environment_credentials.value_or(false), + auth_settings.use_insecure_imds_request.value_or(false), + auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS), + auth_settings.no_sign_request.value_or(false), + }); auto new_client = std::make_shared(std::move(new_uri), std::move(auth_settings), std::move(client)); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 3abeb7c70c4..09c7185b007 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -152,9 +152,13 @@ std::unique_ptr getClient( config.getString(config_prefix + ".secret_access_key", ""), config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), {}, - config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)), - config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)), - config.getUInt64(config_prefix + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); + S3::CredentialsConfiguration + { + config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)), + config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)), + config.getUInt64(config_prefix + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + config.getBool(config_prefix + ".no_sign_request", config.getBool("s3.no_sign_request", false)) + }); } } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index e80e58314c7..500af4c58b7 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -563,9 +563,7 @@ std::unique_ptr ClientFactory::create( // NOLINT const String & secret_access_key, const String & server_side_encryption_customer_key_base64, HTTPHeaderEntries headers, - bool use_environment_credentials, - bool use_insecure_imds_request, - uint64_t expiration_window_seconds) + CredentialsConfiguration credentials_configuration) { PocoHTTPClientConfiguration client_configuration = cfg_; client_configuration.updateSchemeAndRegion(); @@ -592,9 +590,7 @@ std::unique_ptr ClientFactory::create( // NOLINT auto credentials_provider = std::make_shared( client_configuration, std::move(credentials), - use_environment_credentials, - use_insecure_imds_request, - expiration_window_seconds); + credentials_configuration); client_configuration.retryStrategy = std::make_shared(std::move(client_configuration.retryStrategy)); return Client::create( diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 0e102a1859d..5c68fca6f04 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -228,9 +228,7 @@ public: const String & secret_access_key, const String & server_side_encryption_customer_key_base64, HTTPHeaderEntries headers, - bool use_environment_credentials, - bool use_insecure_imds_request, - uint64_t expiration_window_seconds = DEFAULT_EXPIRATION_WINDOW_SECONDS); + CredentialsConfiguration credentials_configuration); PocoHTTPClientConfiguration createClientConfiguration( const String & force_region, diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index f6675961ddc..b6b264e274e 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -418,12 +418,14 @@ void AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::refreshIfExpired() S3CredentialsProviderChain::S3CredentialsProviderChain( const DB::S3::PocoHTTPClientConfiguration & configuration, const Aws::Auth::AWSCredentials & credentials, - bool use_environment_credentials, - bool use_insecure_imds_request, - uint64_t expiration_window_seconds) + CredentialsConfiguration credentials_configuration) { auto * logger = &Poco::Logger::get("S3CredentialsProviderChain"); + /// we don't provide any credentials to avoid signing + if (credentials_configuration.no_sign_request) + return; + /// add explicit credentials to the front of the chain /// because it's manually defined by the user if (!credentials.IsEmpty()) @@ -432,7 +434,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( return; } - if (use_environment_credentials) + if (credentials_configuration.use_environment_credentials) { static const char AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI[] = "AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"; static const char AWS_ECS_CONTAINER_CREDENTIALS_FULL_URI[] = "AWS_CONTAINER_CREDENTIALS_FULL_URI"; @@ -453,7 +455,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.for_disk_s3, configuration.get_request_throttler, configuration.put_request_throttler); - AddProvider(std::make_shared(aws_client_configuration, expiration_window_seconds)); + AddProvider(std::make_shared(aws_client_configuration, credentials_configuration.expiration_window_seconds)); } AddProvider(std::make_shared()); @@ -519,7 +521,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( aws_client_configuration.retryStrategy = std::make_shared(1, 1000); auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); - auto config_loader = std::make_shared(ec2_metadata_client, !use_insecure_imds_request); + auto config_loader = std::make_shared(ec2_metadata_client, !credentials_configuration.use_insecure_imds_request); AddProvider(std::make_shared(config_loader)); LOG_INFO(logger, "Added EC2 metadata service credentials provider to the provider chain."); diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index d6214c5e2fa..cd9072f9765 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -121,15 +121,21 @@ private: uint64_t expiration_window_seconds; }; +struct CredentialsConfiguration +{ + bool use_environment_credentials = false; + bool use_insecure_imds_request = false; + uint64_t expiration_window_seconds = DEFAULT_EXPIRATION_WINDOW_SECONDS; + bool no_sign_request = false; +}; + class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain { public: S3CredentialsProviderChain( const DB::S3::PocoHTTPClientConfiguration & configuration, const Aws::Auth::AWSCredentials & credentials, - bool use_environment_credentials, - bool use_insecure_imds_request, - uint64_t expiration_window_seconds); + CredentialsConfiguration credentials_configuration); }; } diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 4acc31ca472..20984b69463 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -89,6 +89,10 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const if (config.has(config_elem + ".expiration_window_seconds")) expiration_window_seconds = config.getUInt64(config_elem + ".expiration_window_seconds"); + std::optional no_sign_request; + if (config.has(config_elem + ".no_sign_request")) + no_sign_request = config.getBool(config_elem + ".no_sign_request"); + HTTPHeaderEntries headers; Poco::Util::AbstractConfiguration::Keys subconfig_keys; config.keys(config_elem, subconfig_keys); @@ -112,7 +116,8 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const std::move(headers), use_environment_credentials, use_insecure_imds_request, - expiration_window_seconds + expiration_window_seconds, + no_sign_request }; } @@ -133,6 +138,9 @@ void AuthSettings::updateFrom(const AuthSettings & from) use_environment_credentials = from.use_environment_credentials; use_insecure_imds_request = from.use_insecure_imds_request; expiration_window_seconds = from.expiration_window_seconds; + + if (from.no_sign_request.has_value()) + no_sign_request = *from.no_sign_request; } } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index ff948c065f8..0b3ab97b2d8 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -85,6 +85,7 @@ struct AuthSettings std::optional use_environment_credentials; std::optional use_insecure_imds_request; std::optional expiration_window_seconds; + std::optional no_sign_request; bool operator==(const AuthSettings & other) const = default; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e17860af288..f2a9b3a3955 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -62,6 +62,8 @@ #include #include +#include + namespace fs = std::filesystem; @@ -1265,9 +1267,13 @@ void StorageS3::updateConfiguration(ContextPtr ctx, StorageS3::Configuration & u credentials.GetAWSSecretKey(), upd.auth_settings.server_side_encryption_customer_key_base64, std::move(headers), - upd.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), - upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - upd.auth_settings.expiration_window_seconds.value_or(ctx->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); + S3::CredentialsConfiguration{ + upd.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), + upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + upd.auth_settings.expiration_window_seconds.value_or( + ctx->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + upd.auth_settings.no_sign_request.value_or(ctx->getConfigRef().getBool("s3.no_sign_request", false)), + }); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) @@ -1306,6 +1312,9 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context /// S3('url') /// S3('url', 'format') /// S3('url', 'format', 'compression') + /// S3('url', NOSIGN) + /// S3('url', NOSIGN, 'format') + /// S3('url', NOSIGN, 'format', 'compression') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') @@ -1314,7 +1323,7 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context if (engine_args.empty() || engine_args.size() > 5) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage S3 requires 1 to 5 arguments: " - "url, [access_key_id, secret_access_key], name of used format and [compression_method]"); + "url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]"); auto * header_it = StorageURL::collectHeaders(engine_args, configuration.headers_from_ast, local_context); if (header_it != engine_args.end()) @@ -1327,24 +1336,57 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context static std::unordered_map> size_to_engine_args { {1, {{}}}, - {2, {{"format", 1}}}, - {4, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}}, {5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"compression_method", 4}}} }; std::unordered_map engine_args_to_idx; - /// For 3 arguments we support 2 possible variants: - /// s3(source, format, compression_method) and s3(source, access_key_id, access_key_id) - /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. - if (engine_args.size() == 3) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/access_key_id"); - if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - engine_args_to_idx = {{"format", 1}, {"compression_method", 2}}; + bool no_sign_request = false; + /// For 2 arguments we support 2 possible variants: + /// - s3(source, format) + /// - s3(source, NOSIGN) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. + if (engine_args.size() == 2) + { + auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + no_sign_request = true; + else + engine_args_to_idx = {{"format", 1}}; + } + /// For 3 arguments we support 2 possible variants: + /// - s3(source, format, compression_method) + /// - s3(source, access_key_id, access_key_id) + /// - s3(source, NOSIGN, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name. + else if (engine_args.size() == 3) + { + auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + engine_args_to_idx = {{"format", 2}}; + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + engine_args_to_idx = {{"format", 1}, {"compression_method", 2}}; else engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; } + /// For 4 arguments we support 2 possible variants: + /// - s3(source, access_key_id, secret_access_key, format) + /// - s3(source, NOSIGN, format, compression_method) + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not. + else if (engine_args.size() == 4) + { + auto second_arg = checkAndGetLiteralArgument(engine_args[1], "access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + engine_args_to_idx = {{"format", 2}, {"compression_method", 3}}; + } + else + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + } else { engine_args_to_idx = size_to_engine_args[engine_args.size()]; @@ -1364,6 +1406,8 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context if (engine_args_to_idx.contains("secret_access_key")) configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); + + configuration.auth_settings.no_sign_request = no_sign_request; } configuration.static_configuration = !configuration.auth_settings.access_key_id.empty(); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index f082b192ee0..9f5d992e4c9 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -18,6 +18,8 @@ #include "registerTableFunctions.h" #include +#include + namespace DB { @@ -52,36 +54,76 @@ void TableFunctionS3::parseArgumentsImpl( static std::unordered_map> size_to_args { {1, {{}}}, - {2, {{"format", 1}}}, - {5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}}, {6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}} }; std::unordered_map args_to_idx; - /// For 4 arguments we support 2 possible variants: - /// s3(source, format, structure, compression_method) and s3(source, access_key_id, access_key_id, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. - if (args.size() == 4) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); - if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + bool no_sign_request = false; + + /// For 2 arguments we support 2 possible variants: + /// - s3(source, format) + /// - s3(source, NOSIGN) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. + if (args.size() == 2) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + no_sign_request = true; else - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + args_to_idx = {{"format", 1}}; } - /// For 3 arguments we support 2 possible variants: - /// s3(source, format, structure) and s3(source, access_key_id, access_key_id) + /// For 3 arguments we support 3 possible variants: + /// - s3(source, format, structure) + /// - s3(source, access_key_id, access_key_id) + /// - s3(source, NOSIGN, format) /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. else if (args.size() == 3) { - - auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); - if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + args_to_idx = {{"format", 2}}; + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) args_to_idx = {{"format", 1}, {"structure", 2}}; else args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; } + /// For 4 arguments we support 3 possible variants: + /// - s3(source, format, structure, compression_method), + /// - s3(source, access_key_id, access_key_id, format) + /// - s3(source, NOSIGN, format, structure) + /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + else if (args.size() == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + args_to_idx = {{"format", 2}, {"structure", 3}}; + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + else + args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + } + /// For 5 arguments we support 2 possible variants: + /// - s3(source, access_key_id, access_key_id, format, structure) + /// - s3(source, NOSIGN, format, structure, compression_method) + /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + else if (args.size() == 5) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "NOSIGN/access_key_id"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + args_to_idx = {{"format", 2}, {"structure", 3}, {"compression_method", 4}}; + } + else + args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}; + } else { args_to_idx = size_to_args[args.size()]; @@ -104,6 +146,8 @@ void TableFunctionS3::parseArgumentsImpl( if (args_to_idx.contains("secret_access_key")) s3_configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); + + s3_configuration.auth_settings.no_sign_request = no_sign_request; } /// For DataLake table functions, we should specify default format. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index aa6479a2ed3..a599855cae1 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -105,25 +105,27 @@ def started_cluster(): def test_with_invalid_environment_credentials(started_cluster): - auth = "'minio','minio123'" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["s3_with_invalid_environment_credentials"] - instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth}) select * from numbers(100) settings s3_truncate_on_insert=1" - ) - with pytest.raises(helpers.client.QueryRuntimeException) as ei: + for (bucket, auth) in [ + (started_cluster.minio_restricted_bucket, "'minio', 'minio123'"), + (started_cluster.minio_bucket, "NOSIGN"), + ]: instance.query( - f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl')" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth}) select * from numbers(100) settings s3_truncate_on_insert=1" ) - assert ei.value.returncode == 243 - assert "HTTP response code: 403" in ei.value.stderr + with pytest.raises(helpers.client.QueryRuntimeException) as ei: + instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl')" + ) - assert ( - "100" - == instance.query( - f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth})" - ).strip() - ) + assert ei.value.returncode == 243 + assert "HTTP response code: 403" in ei.value.stderr + + assert ( + "100" + == instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth})" + ).strip() + ) From 4c61ec2faddb6b0e5ada9b1fa19ee2bc7469a0d5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Mar 2023 07:19:08 +0000 Subject: [PATCH 059/233] update docs --- .../engines/table-engines/integrations/s3.md | 21 ++++++++++++++++++- docs/en/sql-reference/table-functions/s3.md | 18 +++++++++++++++- 2 files changed, 37 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index dd843945e10..c539340d332 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -12,7 +12,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec ``` sql CREATE TABLE s3_engine_table (name String, value UInt32) - ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression]) + ENGINE = S3(path, [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression]) [PARTITION BY expr] [SETTINGS ...] ``` @@ -20,6 +20,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) **Engine parameters** - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). +- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. - `format` — The [format](../../../interfaces/formats.md#formats) of the file. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). - `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. @@ -151,6 +152,7 @@ The following settings can be specified in configuration file for given endpoint - `region` — Specifies S3 region name. Optional. - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`. - `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`. +- `no_sign_request` - Ignore all the credentials so requests are not signed. Useful for accessing public buckets. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. - `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. @@ -168,6 +170,7 @@ The following settings can be specified in configuration file for given endpoint + @@ -175,6 +178,22 @@ The following settings can be specified in configuration file for given endpoint ``` +## Accessing public buckets + +ClickHouse tries to fetch credentials from many different types of sources. +Sometimes, it can produce problems when accessing some buckets that are public causing the client to return `403` error code. +This issue can be avoided by using `NOSIGN` keyword, forcing the client to ignore all the credentials, and not sign the requests. + +``` sql +SELECT * +FROM s3( + 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', + NOSIGN, + 'CSVWithNames' +) +LIMIT 5; +``` + ## See also - [s3 table function](../../../sql-reference/table-functions/s3.md) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 99b7832394d..023d5229a66 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -12,7 +12,7 @@ Provides a table-like interface to select/insert files in [Amazon S3](https://aw **Syntax** ``` sql -s3(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +s3(path [, NOSIGN | aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) ``` :::tip GCS @@ -33,6 +33,7 @@ For GCS, substitute your HMAC key and HMAC secret where you see `aws_access_key_ and not ~~https://storage.cloud.google.com~~. ::: +- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. - `compression` — Parameter is optional. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. @@ -185,6 +186,21 @@ INSERT INTO TABLE FUNCTION ``` As a result, the data is written into three files in different buckets: `my_bucket_1/file.csv`, `my_bucket_10/file.csv`, and `my_bucket_20/file.csv`. +## Accessing public buckets + +ClickHouse tries to fetch credentials from many different types of sources. +Sometimes, it can produce problems when accessing some buckets that are public causing the client to return `403` error code. +This issue can be avoided by using `NOSIGN` keyword, forcing the client to ignore all the credentials, and not sign the requests. + +``` sql +SELECT * +FROM s3( + 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', + NOSIGN, + 'CSVWithNames' +) +LIMIT 5; +``` **See Also** From a0b6cd63bba2cf1cdee72b79b641f6a1a4880b25 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Mar 2023 07:22:31 +0000 Subject: [PATCH 060/233] fix build --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 7cd85db931b..dc00d9a66f1 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -91,7 +91,7 @@ void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfig if (auto session_timeout_key = coordination_key + ".session_timeout_ms"; config.has(session_timeout_key)) - session_timeout_key = config.getInt(session_timeout_key); + session_timeout_ms = config.getInt(session_timeout_key); } Poco::Util::AbstractConfiguration::Keys keys; From 6bf48796a9e25fb26555b89f3fee1b942d49bf0e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 28 Mar 2023 07:29:36 +0000 Subject: [PATCH 061/233] Automatic style fix --- .../integration/test_storage_s3/test_invalid_env_credentials.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index a599855cae1..06d49d8e828 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -107,7 +107,7 @@ def started_cluster(): def test_with_invalid_environment_credentials(started_cluster): instance = started_cluster.instances["s3_with_invalid_environment_credentials"] - for (bucket, auth) in [ + for bucket, auth in [ (started_cluster.minio_restricted_bucket, "'minio', 'minio123'"), (started_cluster.minio_bucket, "NOSIGN"), ]: From ac8dd527dcfd3028fb440d98ad5eebbb7eced5cc Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Mar 2023 07:47:37 +0000 Subject: [PATCH 062/233] Address PR comments --- src/Planner/Planner.cpp | 5 ----- src/Planner/PlannerJoinTree.cpp | 7 +++---- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e38f460e7c5..a8e0d80ce8c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -38,7 +38,6 @@ #include #include -#include #include #include #include @@ -70,10 +69,6 @@ #include #include -#include - -#include - namespace DB { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1741cca17c5..0479170eba1 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,7 +1,6 @@ #include #include -#include "Storages/SelectQueryInfo.h" #include @@ -439,8 +438,8 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, - SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + SelectQueryInfo & table_expression_query_info, + PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -688,7 +687,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } else { - if (auto * distributed = dynamic_cast(storage.get()); + if (auto * distributed = typeid_cast(storage.get()); distributed && canUseCustomKey(settings, *distributed->getCluster(), *query_context)) { table_expression_query_info.use_custom_key = true; From 1605afe64788529cb9ab39d59adf0c251ba47a21 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Mar 2023 08:12:21 +0000 Subject: [PATCH 063/233] Fix test --- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 2 +- ...2_constraints_where_optimization.reference | 17 +++++++++++ .../01622_constraints_where_optimization.sql | 2 ++ .../01623_constraints_column_swap.reference | 29 +++++++++++++++++++ .../01623_constraints_column_swap.sql | 2 ++ 5 files changed, 51 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 5998237f3ce..4d32c96b845 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -705,7 +705,7 @@ public: if (filter_node == nullptr) return; - optimizeNode(query_node->getWhere(), table_expressions, context); + optimizeNode(filter_node, table_expressions, context); has_filter = true; }; diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index 52aca371a6a..9bb42ed1c27 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -56,6 +56,23 @@ QUERY id: 0 CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 SELECT count() FROM t_constraints_where +PREWHERE (b > 20) OR (b < 8) +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.t_constraints_where + PREWHERE + FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 +SELECT count() +FROM t_constraints_where QUERY id: 0 PROJECTION COLUMNS count() UInt64 diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index 33fa62368b0..562c8705a51 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -15,6 +15,8 @@ EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumptio EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 520bd16ae25..124b5d06bed 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -27,6 +27,35 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 +SELECT + cityHash64(a) + 10, + b + 3 +FROM column_swap_test_test +PREWHERE cityHash64(a) = 1 +QUERY id: 0 + PROJECTION COLUMNS + plus(cityHash64(a), 10) UInt64 + plus(b, 3) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 + FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.column_swap_test_test + PREWHERE + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 + CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 97e014d9c25..6d70b78194d 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -14,6 +14,8 @@ INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', t EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; From 6f1e50598a472388d8267f38bcb860633bd44f19 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Mar 2023 09:32:23 +0000 Subject: [PATCH 064/233] fix test --- src/IO/S3/tests/gtest_aws_s3_client.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index bbbf6a430ab..3e0ed21d01c 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -1,5 +1,6 @@ #include +#include "IO/S3/Credentials.h" #include "config.h" @@ -109,8 +110,11 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) secret_access_key, server_side_encryption_customer_key_base64, headers, - use_environment_credentials, - use_insecure_imds_request + DB::S3::CredentialsConfiguration + { + .use_environment_credentials = use_environment_credentials, + .use_insecure_imds_request = use_insecure_imds_request + } ); ASSERT_TRUE(client); From e01743a5cdd996d96f711ad5a9056b89ec474c60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 28 Mar 2023 13:34:23 +0200 Subject: [PATCH 065/233] Try to fix unrelated flakiness --- tests/integration/test_version_update_after_mutation/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 6b27c69462a..67f7ce47451 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -91,8 +91,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("OPTIMIZE TABLE mt FINAL") - assert node1.query("SELECT id FROM mt") == "1\n4\n" - assert node2.query("SELECT id FROM mt") == "1\n4\n" + assert node1.query("SELECT id FROM mt ORDER BY id") == "1\n4\n" + assert node2.query("SELECT id FROM mt ORDER BY id") == "1\n4\n" for node in [node1, node2]: node.query("DROP TABLE mt") From d1e874e95bb9062d6ca8be5190d061efa39c76ba Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Mar 2023 18:22:55 +0100 Subject: [PATCH 066/233] Add a test for ClientInfo initial_query_start_time in inter-server mode Signed-off-by: Azat Khuzhin --- ...nt_info_initial_query_start_time.reference | 8 +++ ...de_client_info_initial_query_start_time.sh | 67 +++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference create mode 100755 tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh diff --git a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference new file mode 100644 index 00000000000..fbce8ae2026 --- /dev/null +++ b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference @@ -0,0 +1,8 @@ +SELECT +3 0 0 +3 0 0 +INSERT +CHECK +1 +2 +6 0 2 diff --git a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh new file mode 100755 index 00000000000..5da643bd17b --- /dev/null +++ b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh @@ -0,0 +1,67 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: interserver mode requires SSL +# +# Test that checks that some of ClientInfo correctly passed in inter-server mode. +# NOTE: we need .sh test (.sql is not enough) because queries on remote nodes does not have current_database = currentDatabase() +# +# Check-style suppression: select * from system.query_log where current_database = currentDatabase(); + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function get_query_id() { random_str 10; } + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists buf; + drop table if exists dist; + drop table if exists data; + + create table data (key Int) engine=Memory(); + create table dist as data engine=Distributed(test_cluster_interserver_secret, currentDatabase(), data, key); + create table dist_dist as data engine=Distributed(test_cluster_interserver_secret, currentDatabase(), dist, key); + system stop distributed sends dist; +" + +echo "SELECT" +query_id="$(get_query_id)" +# initialize connection, but actually if there are other tables that uses this +# cluster then, it will be created long time ago, but this is OK for this +# test, since we care about the difference between NOW() and there should +# not be any significant difference. +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -q "select * from dist" +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + system flush logs; + select count(), countIf(initial_query_start_time_microseconds != query_start_time_microseconds), countIf(event_time - initial_query_start_time > 3) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; +" + +sleep 6 + +query_id="$(get_query_id)" +# this query (and all subsequent) should reuse the previous connection (at least most of the time) +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -q "select * from dist" + +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + system flush logs; + select count(), countIf(initial_query_start_time_microseconds != query_start_time_microseconds), countIf(event_time - initial_query_start_time > 3) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; +" + +echo "INSERT" +query_id="$(get_query_id)" +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -nm -q " + insert into dist_dist values (1),(2); + select * from data; +" + +sleep 3 +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "system flush distributed dist_dist" +sleep 1 +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "system flush distributed dist" + +echo "CHECK" +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + select * from data order by key; + system flush logs; + select count(), countIf(initial_query_start_time_microseconds != query_start_time_microseconds), countIf(event_time - initial_query_start_time > 3) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; +" From cea631a4c267e96a90c97baa8710a18232b66fa6 Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Tue, 28 Mar 2023 23:24:07 +0800 Subject: [PATCH 067/233] throw exception while non-parametric functions having parameters --- .../UserDefined/UserDefinedSQLFunctionVisitor.cpp | 7 +++++++ src/Interpreters/ActionsVisitor.cpp | 7 +++++++ .../0_stateless/25403_non_parametric_function.reference | 0 .../queries/0_stateless/25403_non_parametric_function.sql | 5 +++++ 4 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/25403_non_parametric_function.reference create mode 100644 tests/queries/0_stateless/25403_non_parametric_function.sql diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp index d78a8623a18..57cc45cc75d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; + extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; } void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) @@ -132,6 +133,12 @@ ASTPtr UserDefinedSQLFunctionVisitor::tryToReplaceFunction(const ASTFunction & f if (!user_defined_function) return nullptr; + /// All UDFs are not parametric for now. + if (function.parameters) + { + throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", function.name); + } + const auto & function_arguments_list = function.children.at(0)->as(); auto & function_arguments = function_arguments_list->children; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 8a5ea1205e7..1f4969a7f9a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -75,6 +75,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; } static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) @@ -1109,6 +1110,12 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & } } + /// Normal functions are not parametric for now. + if (node.parameters) + { + throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", node.name); + } + Names argument_names; DataTypes argument_types; bool arguments_present = true; diff --git a/tests/queries/0_stateless/25403_non_parametric_function.reference b/tests/queries/0_stateless/25403_non_parametric_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/25403_non_parametric_function.sql b/tests/queries/0_stateless/25403_non_parametric_function.sql new file mode 100644 index 00000000000..9b7e2ae7a04 --- /dev/null +++ b/tests/queries/0_stateless/25403_non_parametric_function.sql @@ -0,0 +1,5 @@ +SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } + +CREATE FUNCTION sum_udf as (x, y) -> (x + y); + +SELECT sum_udf(1)(1, 2); -- { serverError 309 } \ No newline at end of file From 6d05968a0f7433f3d6dd0fde5d720f5159e6a7d1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 28 Mar 2023 16:23:32 +0000 Subject: [PATCH 068/233] optimize function mapUpdate --- src/Functions/map.cpp | 136 ++++++++++++++++++++++-------------------- 1 file changed, 72 insertions(+), 64 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 549de200bea..f4f2e1fbfea 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -18,6 +18,7 @@ #include "array/arrayIndex.h" #include "Functions/like.h" #include "Functions/FunctionsStringSearch.h" +#include namespace DB @@ -616,103 +617,110 @@ public: "Number of arguments for function {} doesn't match: passed {}, should be 2", getName(), arguments.size()); - const DataTypeMap * left = checkAndGetDataType(arguments[0].type.get()); - const DataTypeMap * right = checkAndGetDataType(arguments[1].type.get()); + const auto * left = checkAndGetDataType(arguments[0].type.get()); + const auto * right = checkAndGetDataType(arguments[1].type.get()); if (!left || !right) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The two arguments for function {} must be both Map type", - getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The two arguments for function {} must be both Map type", getName()); + if (!left->getKeyType()->equals(*right->getKeyType()) || !left->getValueType()->equals(*right->getValueType())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The Key And Value type of Map for function {} must be the same", - getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The Key And Value type of Map for function {} must be the same", getName()); return std::make_shared(left->getKeyType(), left->getValueType()); } bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const ColumnMap * col_map_left = typeid_cast(arguments[0].column.get()); - const auto * col_const_map_left = checkAndGetColumnConst(arguments[0].column.get()); - bool col_const_map_left_flag = false; - if (col_const_map_left) - { - col_const_map_left_flag = true; - col_map_left = typeid_cast(&col_const_map_left->getDataColumn()); - } - if (!col_map_left) - return nullptr; + bool is_left_const = isColumnConst(*arguments[0].column); + bool is_right_const = isColumnConst(*arguments[1].column); - const ColumnMap * col_map_right = typeid_cast(arguments[1].column.get()); - const auto * col_const_map_right = checkAndGetColumnConst(arguments[1].column.get()); - bool col_const_map_right_flag = false; - if (col_const_map_right) - { - col_const_map_right_flag = true; - col_map_right = typeid_cast(&col_const_map_right->getDataColumn()); - } - if (!col_map_right) - return nullptr; + const auto * map_column_left = is_left_const + ? checkAndGetColumnConstData(arguments[0].column.get()) + : checkAndGetColumn(arguments[0].column.get()); - const auto & nested_column_left = col_map_left->getNestedColumn(); - const auto & keys_data_left = col_map_left->getNestedData().getColumn(0); - const auto & values_data_left = col_map_left->getNestedData().getColumn(1); + const auto * map_column_right = is_right_const + ? checkAndGetColumnConstData(arguments[1].column.get()) + : checkAndGetColumn(arguments[1].column.get()); + + if (!map_column_left || !map_column_right) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Arguments for function {} must be maps, got {} and {} instead", + getName(), arguments[0].column->getName(), arguments[1].column->getName()); + + const auto & nested_column_left = map_column_left->getNestedColumn(); + const auto & keys_data_left = map_column_left->getNestedData().getColumn(0); + const auto & values_data_left = map_column_left->getNestedData().getColumn(1); const auto & offsets_left = nested_column_left.getOffsets(); - const auto & nested_column_right = col_map_right->getNestedColumn(); - const auto & keys_data_right = col_map_right->getNestedData().getColumn(0); - const auto & values_data_right = col_map_right->getNestedData().getColumn(1); + const auto & nested_column_right = map_column_right->getNestedColumn(); + const auto & keys_data_right = map_column_right->getNestedData().getColumn(0); + const auto & values_data_right = map_column_right->getNestedData().getColumn(1); const auto & offsets_right = nested_column_right.getOffsets(); - const auto & result_type_map = static_cast(*result_type); - const DataTypePtr & key_type = result_type_map.getKeyType(); - const DataTypePtr & value_type = result_type_map.getValueType(); - MutableColumnPtr keys_data = key_type->createColumn(); - MutableColumnPtr values_data = value_type->createColumn(); - MutableColumnPtr offsets = DataTypeNumber().createColumn(); + auto result_keys = keys_data_left.cloneEmpty(); + auto result_values = values_data_left.cloneEmpty(); + auto result_offsets = ColumnVector::create(input_rows_count); + auto & result_offsets_data = result_offsets->getData(); + + using Set = HashSetWithStackMemory; + + Set right_keys_const; + if (is_right_const) + { + for (size_t i = 0; i < keys_data_right.size(); ++i) + right_keys_const.insert(keys_data_right.getDataAt(i)); + } IColumn::Offset current_offset = 0; for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { - size_t left_it_begin = col_const_map_left_flag ? 0 : offsets_left[row_idx - 1]; - size_t left_it_end = col_const_map_left_flag ? offsets_left.size() : offsets_left[row_idx]; - size_t right_it_begin = col_const_map_right_flag ? 0 : offsets_right[row_idx - 1]; - size_t right_it_end = col_const_map_right_flag ? offsets_right.size() : offsets_right[row_idx]; + size_t left_from = is_left_const ? 0 : offsets_left[row_idx - 1]; + size_t left_to = is_left_const ? offsets_left[0] : offsets_left[row_idx]; - for (size_t i = left_it_begin; i < left_it_end; ++i) + size_t right_from = is_right_const ? 0 : offsets_right[row_idx - 1]; + size_t right_to = is_right_const ? offsets_right[0] : offsets_right[row_idx]; + + auto execute_row = [&](const auto & set) { - bool matched = false; - auto key = keys_data_left.getDataAt(i); - for (size_t j = right_it_begin; j < right_it_end; ++j) + for (size_t i = left_from; i < left_to; ++i) { - if (keys_data_right.getDataAt(j).toString() == key.toString()) + if (!set.find(keys_data_left.getDataAt(i))) { - matched = true; - break; + result_keys->insertFrom(keys_data_left, i); + result_values->insertFrom(values_data_left, i); + ++current_offset; } } - if (!matched) - { - keys_data->insertFrom(keys_data_left, i); - values_data->insertFrom(values_data_left, i); - ++current_offset; - } - } + }; - for (size_t j = right_it_begin; j < right_it_end; ++j) + if (is_right_const) { - keys_data->insertFrom(keys_data_right, j); - values_data->insertFrom(values_data_right, j); - ++current_offset; + execute_row(right_keys_const); + } + else + { + Set right_keys; + for (size_t i = right_from; i < right_to; ++i) + right_keys.insert(keys_data_right.getDataAt(i)); + + execute_row(right_keys); } - offsets->insert(current_offset); + size_t right_map_size = right_to - right_from; + result_keys->insertRangeFrom(keys_data_right, right_from, right_map_size); + result_values->insertRangeFrom(values_data_right, right_from, right_map_size); + + current_offset += right_map_size; + result_offsets_data[row_idx] = current_offset; } auto nested_column = ColumnArray::create( - ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}), - std::move(offsets)); + ColumnTuple::create(Columns{std::move(result_keys), std::move(result_values)}), + std::move(result_offsets)); return ColumnMap::create(nested_column); } From 1e79245b942bf21419daa35719008299bdeccf81 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 28 Mar 2023 17:17:42 +0000 Subject: [PATCH 069/233] add tests --- src/Functions/map.cpp | 6 +++++ tests/performance/map_update.xml | 22 +++++++++++++++++++ .../0_stateless/02169_map_functions.reference | 4 ++++ .../0_stateless/02169_map_functions.sql | 5 +++++ 4 files changed, 37 insertions(+) create mode 100644 tests/performance/map_update.xml diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index f4f2e1fbfea..c7125d85c58 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -663,6 +663,12 @@ public: auto result_keys = keys_data_left.cloneEmpty(); auto result_values = values_data_left.cloneEmpty(); + + size_t size_to_reserve = keys_data_right.size() + (keys_data_left.size() - keys_data_right.size()); + + result_keys->reserve(size_to_reserve); + result_values->reserve(size_to_reserve); + auto result_offsets = ColumnVector::create(input_rows_count); auto & result_offsets_data = result_offsets->getData(); diff --git a/tests/performance/map_update.xml b/tests/performance/map_update.xml new file mode 100644 index 00000000000..2c5417ffe28 --- /dev/null +++ b/tests/performance/map_update.xml @@ -0,0 +1,22 @@ + + + WITH (range(10), range(10))::Map(String, UInt64) AS m1, (range(3), range(3))::Map(String, UInt64) AS m2 + SELECT count() FROM numbers(500000) + WHERE NOT ignore(mapUpdate(materialize(m1), materialize(m2))) + + + WITH (range(10), range(10))::Map(String, UInt64) AS m1, (range(3), range(3))::Map(String, UInt64) AS m2 + SELECT count() FROM numbers(500000) + WHERE NOT ignore(mapUpdate(materialize(m1), m2)) + + + WITH (range(100), range(100))::Map(String, UInt64) AS m1, (range(30), range(30))::Map(String, UInt64) AS m2 + SELECT count() FROM numbers(50000) + WHERE NOT ignore(mapUpdate(materialize(m1), materialize(m2))) + + + WITH (range(100), range(100))::Map(String, UInt64) AS m1, (range(30), range(30))::Map(String, UInt64) AS m2 + SELECT count() FROM numbers(50000) + WHERE NOT ignore(mapUpdate(materialize(m1), m2)) + + diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 160aebbc852..fc7cd1a3c4b 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -31,3 +31,7 @@ {1:2,2:3} {'x':'y','x':'y'} {'x':'y','x':'y'} +{'k1':11,'k2':22} +{'k1':11,'k2':22} +{'k1':11,'k2':22} +{'k1':11,'k2':22} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 4cccaa56722..31112e18b58 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -15,6 +15,11 @@ SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1, 0, 2, 0))); SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0))); +SELECT mapUpdate(map('k1', 1, 'k2', 2), map('k1', 11, 'k2', 22)); +SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), map('k1', 11, 'k2', 22)); +SELECT mapUpdate(map('k1', 1, 'k2', 2), materialize(map('k1', 11, 'k2', 22))); +SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), materialize(map('k1', 11, 'k2', 22))); + SELECT mapApply(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 202f52dc3aa8a7f08905cfbd1ce8688f51fb44a8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 Mar 2023 14:36:38 +0000 Subject: [PATCH 070/233] Implement SHOW COLUMNS Modeled after (*) Fixes #46437 (*) https://dev.mysql.com/doc/refman/8.0/en/show-columns.html --- docs/en/sql-reference/statements/show.md | 89 +++++++++++---- src/Interpreters/InterpreterFactory.cpp | 6 + .../InterpreterShowColumnsQuery.cpp | 108 ++++++++++++++++++ .../InterpreterShowColumnsQuery.h | 32 ++++++ .../InterpreterShowTablesQuery.cpp | 5 +- src/Interpreters/InterpreterShowTablesQuery.h | 3 +- src/Parsers/ASTIdentifier_fwd.h | 4 + src/Parsers/ASTShowColumnsQuery.cpp | 53 +++++++++ src/Parsers/ASTShowColumnsQuery.h | 34 ++++++ src/Parsers/ASTShowTablesQuery.h | 25 ++-- src/Parsers/ParserQueryWithOutput.cpp | 3 + src/Parsers/ParserShowColumnsQuery.cpp | 80 +++++++++++++ src/Parsers/ParserShowColumnsQuery.h | 19 +++ src/Parsers/ParserShowTablesQuery.cpp | 6 - .../0_stateless/25402_show_columns.reference | 38 ++++++ .../0_stateless/25402_show_columns.sql | 80 +++++++++++++ 16 files changed, 541 insertions(+), 44 deletions(-) create mode 100644 src/Interpreters/InterpreterShowColumnsQuery.cpp create mode 100644 src/Interpreters/InterpreterShowColumnsQuery.h create mode 100644 src/Parsers/ASTShowColumnsQuery.cpp create mode 100644 src/Parsers/ASTShowColumnsQuery.h create mode 100644 src/Parsers/ParserShowColumnsQuery.cpp create mode 100644 src/Parsers/ParserShowColumnsQuery.h create mode 100644 tests/queries/0_stateless/25402_show_columns.reference create mode 100644 tests/queries/0_stateless/25402_show_columns.sql diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 544c556d4b3..c0ab51ae444 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -30,7 +30,7 @@ This statement is identical to the query: SELECT name FROM system.databases [WHERE name [NOT] LIKE | ILIKE ''] [LIMIT ] [INTO OUTFILE filename] [FORMAT format] ``` -### Examples +**Examples** Getting database names, containing the symbols sequence 'de' in their names: @@ -92,7 +92,7 @@ Result: └────────────────────────────────┘ ``` -### See Also +**See also** - [CREATE DATABASE](https://clickhouse.com/docs/en/sql-reference/statements/create/database/#query-language-create-database) @@ -128,7 +128,7 @@ This statement is identical to the query: SELECT name FROM system.tables [WHERE name [NOT] LIKE | ILIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] ``` -### Examples +**Examples** Getting table names, containing the symbols sequence 'user' in their names: @@ -191,11 +191,59 @@ Result: └────────────────────────────────┘ ``` -### See Also +**See also** - [Create Tables](https://clickhouse.com/docs/en/getting-started/tutorial/#create-tables) - [SHOW CREATE TABLE](https://clickhouse.com/docs/en/sql-reference/statements/show/#show-create-table) +## SHOW COLUMNS + +Displays a list of columns + +```sql +SHOW [EXTENDED] [FULL] COLUMNS {FROM | IN} [{FROM | IN} ] [{[NOT] {LIKE | ILIKE} '' | WHERE }] [LIMIT ] [INTO +OUTFILE ] [FORMAT ] +``` + +The database and table name can be specified in abbreviated form as `.
`, i.e. `FROM tab FROM db` and `FROM db.tab` are +equivalent. If no database is specified, the query returns the list of columns from the current database. + +The optional keyword `EXTENDED` currently has no effect, it only exists for MySQL compatibility. + +The optional keyword `FULL` causes the output to include the collation, comment and privilege columns. + +`SHOW COLUMNS` produces a result table with the following structure: +- field - The name of the column (String) +- type - The column data type (String) +- null - If the column data type is Nullable (UInt8) +- key - `PRI` if the column is part of the primary key, `SOR` if the column is part of the sorting key, empty otherwise (String) +- default - Default expression of the column if it is of type `ALIAS`, `DEFAULT`, or `MATERIALIZED`, otherwise `NULL`. (Nullable(String)) +- extra - Additional information, currently unused (String) +- collation - Collation of the column, always `NULL` because ClickHouse has no per-column collations, only if `FULL` keyword was specified (Nullable(String)) +- comment - Comment on the column, only if `FULL` keyword was specified (String) +- privilege - The privilege you have on this column, currently not available, only if `FULL` keyword was specified (String) + +**Examples** + +Getting information about all columns in table 'order' starting with 'delivery_': + +```sql +SHOW COLUMNS FROM 'orders' LIKE 'delivery_%' +``` + +Result: + +``` text +┌─field───────────┬─type─────┬─null─┬─key─────┬─default─┬─extra─┐ +│ delivery_date │ DateTime │ 0 │ PRI SOR │ ᴺᵁᴸᴸ │ │ +│ delivery_status │ Bool │ 0 │ │ ᴺᵁᴸᴸ │ │ +└─────────────────┴──────────┴──────┴─────────┴─────────┴───────┘ +``` + +**See also** + +- [system.columns](https://clickhouse.com/docs/en/operations/system-tables/columns) + ## SHOW DICTIONARIES Displays a list of [Dictionaries](../../sql-reference/dictionaries/index.md). @@ -212,7 +260,7 @@ You can get the same results as the `SHOW DICTIONARIES` query in the following w SELECT name FROM system.dictionaries WHERE database = [AND name LIKE ] [LIMIT ] [INTO OUTFILE ] [FORMAT ] ``` -**Example** +**Examples** The following query selects the first two rows from the list of tables in the `system` database, whose names contain `reg`. @@ -231,7 +279,7 @@ SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2 Shows privileges for a user. -### Syntax +**Syntax** ``` sql SHOW GRANTS [FOR user1 [, user2 ...]] @@ -245,7 +293,7 @@ Shows parameters that were used at a [user creation](../../sql-reference/stateme `SHOW CREATE USER` does not output user passwords. -### Syntax +**Syntax** ``` sql SHOW CREATE USER [name1 [, name2 ...] | CURRENT_USER] @@ -255,7 +303,7 @@ SHOW CREATE USER [name1 [, name2 ...] | CURRENT_USER] Shows parameters that were used at a [role creation](../../sql-reference/statements/create/role.md). -### Syntax +**Syntax** ``` sql SHOW CREATE ROLE name1 [, name2 ...] @@ -265,7 +313,7 @@ SHOW CREATE ROLE name1 [, name2 ...] Shows parameters that were used at a [row policy creation](../../sql-reference/statements/create/row-policy.md). -### Syntax +**Syntax** ``` sql SHOW CREATE [ROW] POLICY name ON [database1.]table1 [, [database2.]table2 ...] @@ -275,7 +323,7 @@ SHOW CREATE [ROW] POLICY name ON [database1.]table1 [, [database2.]table2 ...] Shows parameters that were used at a [quota creation](../../sql-reference/statements/create/quota.md). -### Syntax +**Syntax** ``` sql SHOW CREATE QUOTA [name1 [, name2 ...] | CURRENT] @@ -285,7 +333,7 @@ SHOW CREATE QUOTA [name1 [, name2 ...] | CURRENT] Shows parameters that were used at a [settings profile creation](../../sql-reference/statements/create/settings-profile.md). -### Syntax +**Syntax** ``` sql SHOW CREATE [SETTINGS] PROFILE name1 [, name2 ...] @@ -295,7 +343,7 @@ SHOW CREATE [SETTINGS] PROFILE name1 [, name2 ...] Returns a list of [user account](../../guides/sre/user-management/index.md#user-account-management) names. To view user accounts parameters, see the system table [system.users](../../operations/system-tables/users.md#system_tables-users). -### Syntax +**Syntax** ``` sql SHOW USERS @@ -305,7 +353,7 @@ SHOW USERS Returns a list of [roles](../../guides/sre/user-management/index.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). -### Syntax +**Syntax** ``` sql SHOW [CURRENT|ENABLED] ROLES @@ -314,7 +362,7 @@ SHOW [CURRENT|ENABLED] ROLES Returns a list of [setting profiles](../../guides/sre/user-management/index.md#settings-profiles-management). To view user accounts parameters, see the system table [settings_profiles](../../operations/system-tables/settings_profiles.md#system_tables-settings_profiles). -### Syntax +**Syntax** ``` sql SHOW [SETTINGS] PROFILES @@ -324,7 +372,7 @@ SHOW [SETTINGS] PROFILES Returns a list of [row policies](../../guides/sre/user-management/index.md#row-policy-management) for the specified table. To view user accounts parameters, see the system table [system.row_policies](../../operations/system-tables/row_policies.md#system_tables-row_policies). -### Syntax +**Syntax** ``` sql SHOW [ROW] POLICIES [ON [db.]table] @@ -334,7 +382,7 @@ SHOW [ROW] POLICIES [ON [db.]table] Returns a list of [quotas](../../guides/sre/user-management/index.md#quotas-management). To view quotas parameters, see the system table [system.quotas](../../operations/system-tables/quotas.md#system_tables-quotas). -### Syntax +**Syntax** ``` sql SHOW QUOTAS @@ -344,7 +392,7 @@ SHOW QUOTAS Returns a [quota](../../operations/quotas.md) consumption for all users or for current user. To view another parameters, see system tables [system.quotas_usage](../../operations/system-tables/quotas_usage.md#system_tables-quotas_usage) and [system.quota_usage](../../operations/system-tables/quota_usage.md#system_tables-quota_usage). -### Syntax +**Syntax** ``` sql SHOW [CURRENT] QUOTA @@ -353,7 +401,7 @@ SHOW [CURRENT] QUOTA Shows all [users](../../guides/sre/user-management/index.md#user-account-management), [roles](../../guides/sre/user-management/index.md#role-management), [profiles](../../guides/sre/user-management/index.md#settings-profiles-management), etc. and all their [grants](../../sql-reference/statements/grant.md#grant-privileges). -### Syntax +**Syntax** ``` sql SHOW ACCESS @@ -366,13 +414,14 @@ Returns a list of clusters. All available clusters are listed in the [system.clu `SHOW CLUSTER name` query displays the contents of system.clusters table for this cluster. ::: -### Syntax +**Syntax** ``` sql SHOW CLUSTER '' SHOW CLUSTERS [[NOT] LIKE|ILIKE ''] [LIMIT ] ``` -### Examples + +**Examples** Query: diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 502de459156..fb013ce4569 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -77,6 +78,7 @@ #include #include #include +#include #include #include #include @@ -169,6 +171,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp new file mode 100644 index 00000000000..95530ad7c53 --- /dev/null +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -0,0 +1,108 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + + +InterpreterShowColumnsQuery::InterpreterShowColumnsQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_) + , query_ptr(query_ptr_) +{ +} + + +String InterpreterShowColumnsQuery::getRewrittenQuery() +{ + const auto & query = query_ptr->as(); + + WriteBufferFromOwnString rewritten_query; + + rewritten_query << "SELECT name AS field, type AS type, startsWith(type, 'Nullable') AS null, trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra "; + + // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see + // IStorage::getVirtuals(). We can't easily do that via SQL. + + if (query.full) + { + /// "Full" mode is mostly for MySQL compat + /// - collation: no such thing in ClickHouse + /// - comment + /// - privileges: + rewritten_query << ", NULL AS collation, comment, '' AS privileges "; + } + + rewritten_query << "FROM system.columns WHERE "; + + String database; + String table; + if (query.from_table.contains(".")) + { + /// FROM .
(abbreviated form) + chassert(query.from_database.empty()); + std::vector splitted; + boost::split(splitted, query.from_table, boost::is_any_of(".")); + chassert(splitted.size() == 2); + database = splitted[0]; + table = splitted[1]; + } + else if (query.from_database.empty()) + { + /// FROM
+ chassert(!query.from_table.empty()); + database = getContext()->getCurrentDatabase(); + table = query.from_table; + } + else + { + /// FROM FROM
+ chassert(!query.from_database.empty()); + chassert(!query.from_table.empty()); + database = query.from_database; + table = query.from_table; + } + rewritten_query << "database = " << DB::quote << database; + rewritten_query << " AND table = " << DB::quote << table; + + if (!query.like.empty()) + rewritten_query + << " AND name " + << (query.not_like ? "NOT " : "") + << (query.case_insensitive_like ? "ILIKE " : "LIKE ") + << DB::quote << query.like; + else if (query.where_expression) + rewritten_query << " AND (" << query.where_expression << ")"; + + /// Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to + /// sort the output of SHOW COLUMNS otherwise (SELECT * FROM (SHOW COLUMNS ...) ORDER BY ...) is rejected) and 3. some + /// SQL tests can take advantage of this. + rewritten_query << " ORDER BY field, type, null, key, default, extra"; + + if (query.limit_length) + rewritten_query << " LIMIT " << query.limit_length; + + return rewritten_query.str(); + +} + + +BlockIO InterpreterShowColumnsQuery::execute() +{ + return executeQuery(getRewrittenQuery(), getContext(), true); +} + + +} diff --git a/src/Interpreters/InterpreterShowColumnsQuery.h b/src/Interpreters/InterpreterShowColumnsQuery.h new file mode 100644 index 00000000000..ee6dcabd97b --- /dev/null +++ b/src/Interpreters/InterpreterShowColumnsQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/// Returns a list of columns which meet some conditions. +class InterpreterShowColumnsQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterShowColumnsQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); + + BlockIO execute() override; + + /// Ignore quota and limits here because execute() produces a SELECT query which checks quotas/limits by itself. + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + ASTPtr query_ptr; + + String getRewrittenQuery(); +}; + + +} diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 4e0dfdc9236..026057a8309 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -24,7 +24,8 @@ namespace ErrorCodes InterpreterShowTablesQuery::InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_) + : WithMutableContext(context_) + , query_ptr(query_ptr_) { } diff --git a/src/Interpreters/InterpreterShowTablesQuery.h b/src/Interpreters/InterpreterShowTablesQuery.h index 16fc9ef2cf4..2693e5b08ed 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.h +++ b/src/Interpreters/InterpreterShowTablesQuery.h @@ -20,8 +20,7 @@ public: BlockIO execute() override; - /// We ignore the quota and limits here because execute() will rewrite a show query as a SELECT query and then - /// the SELECT query will checks the quota and limits. + /// Ignore quota and limits here because execute() produces a SELECT query which checks quotas/limits by itself. bool ignoreQuota() const override { return true; } bool ignoreLimits() const override { return true; } diff --git a/src/Parsers/ASTIdentifier_fwd.h b/src/Parsers/ASTIdentifier_fwd.h index c4434477fdb..01cfbb8fec7 100644 --- a/src/Parsers/ASTIdentifier_fwd.h +++ b/src/Parsers/ASTIdentifier_fwd.h @@ -17,17 +17,21 @@ class ASTTableIdentifier; void setIdentifierSpecial(ASTPtr & ast); String getIdentifierName(const IAST * ast); + std::optional tryGetIdentifierName(const IAST * ast); + bool tryGetIdentifierNameInto(const IAST * ast, String & name); inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); } + inline std::optional tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); } + inline bool tryGetIdentifierNameInto(const ASTPtr & ast, String & name) { return tryGetIdentifierNameInto(ast.get(), name); diff --git a/src/Parsers/ASTShowColumnsQuery.cpp b/src/Parsers/ASTShowColumnsQuery.cpp new file mode 100644 index 00000000000..113029aacd7 --- /dev/null +++ b/src/Parsers/ASTShowColumnsQuery.cpp @@ -0,0 +1,53 @@ +#include + +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTShowColumnsQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + cloneOutputOptions(*res); + return res; +} + +void ASTShowColumnsQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "SHOW " + << (extended ? "EXTENDED " : "") + << (full ? "FULL " : "") + << "COLUMNS" + << (settings.hilite ? hilite_none : ""); + + if (from_database.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from_table); + else + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from_database) << "." << backQuoteIfNeed(from_table); + + + if (!like.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") + << (not_like ? " NOT " : "") + << (case_insensitive_like ? " ILIKE " : " LIKE" ) + << (settings.hilite ? hilite_none : "") + << DB::quote << like; + + if (where_expression) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); + where_expression->formatImpl(settings, state, frame); + } + + if (limit_length) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIMIT " << (settings.hilite ? hilite_none : ""); + limit_length->formatImpl(settings, state, frame); + } +} + +} diff --git a/src/Parsers/ASTShowColumnsQuery.h b/src/Parsers/ASTShowColumnsQuery.h new file mode 100644 index 00000000000..79e8a67b558 --- /dev/null +++ b/src/Parsers/ASTShowColumnsQuery.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Query SHOW COLUMNS +class ASTShowColumnsQuery : public ASTQueryWithOutput +{ +public: + bool extended = false; + bool full = false; + bool not_like = false; + bool case_insensitive_like = false; + + ASTPtr where_expression; + ASTPtr limit_length; + + String from_database; + String from_table; + + String like; + + String getID(char) const override { return "ShowColumns"; } + ASTPtr clone() const override; + QueryKind getQueryKind() const override { return QueryKind::Show; } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/src/Parsers/ASTShowTablesQuery.h b/src/Parsers/ASTShowTablesQuery.h index b58d65e37ab..2878df54bcc 100644 --- a/src/Parsers/ASTShowTablesQuery.h +++ b/src/Parsers/ASTShowTablesQuery.h @@ -14,31 +14,28 @@ namespace DB class ASTShowTablesQuery : public ASTQueryWithOutput { public: - bool databases{false}; - bool clusters{false}; - bool cluster{false}; - bool dictionaries{false}; - bool m_settings{false}; - bool changed{false}; - bool temporary{false}; - bool caches{false}; - bool full{false}; + bool databases = false; + bool clusters = false; + bool cluster = false; + bool dictionaries = false; + bool m_settings = false; + bool changed = false; + bool temporary = false; + bool caches = false; + bool full = false; String cluster_str; String from; String like; - bool not_like{false}; - bool case_insensitive_like{false}; + bool not_like = false; + bool case_insensitive_like = false; ASTPtr where_expression; ASTPtr limit_length; - /** Get the text that identifies this element. */ String getID(char) const override { return "ShowTables"; } - ASTPtr clone() const override; - QueryKind getQueryKind() const override { return QueryKind::Show; } protected: diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 7024d8cbe11..518f0e7e50e 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -34,6 +35,7 @@ namespace DB bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; + ParserShowColumnsQuery show_columns_p; ParserShowEnginesQuery show_engine_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; @@ -62,6 +64,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || select_p.parse(pos, query, expected) || show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p` || show_tables_p.parse(pos, query, expected) + || show_columns_p.parse(pos, query, expected) || show_engine_p.parse(pos, query, expected) || table_p.parse(pos, query, expected) || describe_cache_p.parse(pos, query, expected) diff --git a/src/Parsers/ParserShowColumnsQuery.cpp b/src/Parsers/ParserShowColumnsQuery.cpp new file mode 100644 index 00000000000..1db31601437 --- /dev/null +++ b/src/Parsers/ParserShowColumnsQuery.cpp @@ -0,0 +1,80 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr like; + ASTPtr from_db; + ASTPtr from_table; + + auto query = std::make_shared(); + + if (!ParserKeyword("SHOW").ignore(pos, expected)) + return false; + + if (ParserKeyword("EXTENDED").ignore(pos, expected)) + query->extended = true; + + if (ParserKeyword("FULL").ignore(pos, expected)) + query->full = true; + + if (!ParserKeyword("COLUMNS").ignore(pos, expected) || ParserKeyword("FIELDS").ignore(pos, expected)) + return false; + + if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected)) + { + if (!ParserCompoundIdentifier().parse(pos, from_table, expected)) + return false; + } + else + return false; + + tryGetIdentifierNameInto(from_table, query->from_table); + bool abbreviated_form = query->from_table.contains("."); /// FROM .
+ + if (!abbreviated_form) + if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected)) + if (!ParserIdentifier().parse(pos, from_db, expected)) + return false; + + tryGetIdentifierNameInto(from_db, query->from_database); + + if (ParserKeyword("NOT").ignore(pos, expected)) + query->not_like = true; + + if (bool insensitive = ParserKeyword("ILIKE").ignore(pos, expected); insensitive || ParserKeyword("LIKE").ignore(pos, expected)) + { + if (insensitive) + query->case_insensitive_like = true; + + if (!ParserStringLiteral().parse(pos, like, expected)) + return false; + } + else if (query->not_like) + return false; + else if (ParserKeyword("WHERE").ignore(pos, expected)) + if (!ParserExpressionWithOptionalAlias(false).parse(pos, query->where_expression, expected)) + return false; + + if (ParserKeyword("LIMIT").ignore(pos, expected)) + if (!ParserExpressionWithOptionalAlias(false).parse(pos, query->limit_length, expected)) + return false; + + if (like) + query->like = like->as().value.safeGet(); + + node = query; + + return true; +} + +} diff --git a/src/Parsers/ParserShowColumnsQuery.h b/src/Parsers/ParserShowColumnsQuery.h new file mode 100644 index 00000000000..999acf722af --- /dev/null +++ b/src/Parsers/ParserShowColumnsQuery.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +namespace DB +{ + +/** Parses queries of the form + * SHOW [EXTENDED] [FULL] COLUMNS (FROM|IN) tbl [(FROM|IN) db] [(([NOT] (LIKE|ILIKE) expr) | (WHERE expr))] [LIMIT n] + */ +class ParserShowColumnsQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW COLUMNS query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserShowTablesQuery.cpp b/src/Parsers/ParserShowTablesQuery.cpp index 1647dd9a5b4..3540a6d3fc8 100644 --- a/src/Parsers/ParserShowTablesQuery.cpp +++ b/src/Parsers/ParserShowTablesQuery.cpp @@ -149,10 +149,8 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } if (s_from.ignore(pos, expected) || s_in.ignore(pos, expected)) - { if (!name_p.parse(pos, database, expected)) return false; - } if (s_not.ignore(pos, expected)) query->not_like = true; @@ -168,16 +166,12 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec else if (query->not_like) return false; else if (s_where.ignore(pos, expected)) - { if (!exp_elem.parse(pos, query->where_expression, expected)) return false; - } if (s_limit.ignore(pos, expected)) - { if (!exp_elem.parse(pos, query->limit_length, expected)) return false; - } } tryGetIdentifierNameInto(database, query->from); diff --git a/tests/queries/0_stateless/25402_show_columns.reference b/tests/queries/0_stateless/25402_show_columns.reference new file mode 100644 index 00000000000..98438bd6595 --- /dev/null +++ b/tests/queries/0_stateless/25402_show_columns.reference @@ -0,0 +1,38 @@ +int32 Nullable(Int32) 1 \N +str String 0 SOR \N +uint64 UInt64 0 PRI SOR \N +--- +int32 Nullable(Int32) 1 \N +str String 0 SOR \N +uint64 UInt64 0 PRI SOR \N +--- +int32 Nullable(Int32) 1 \N \N example comment +str String 0 SOR \N \N +uint64 UInt64 0 PRI SOR \N \N +--- +int32 Nullable(Int32) 1 \N +uint64 UInt64 0 PRI SOR \N +--- +str String 0 SOR \N +--- +int32 Nullable(Int32) 1 \N +uint64 UInt64 0 PRI SOR \N +--- +str String 0 SOR \N +--- +int32 Nullable(Int32) 1 \N +uint64 UInt64 0 PRI SOR \N +--- +int32 Nullable(Int32) 1 \N +--- +int32 Nullable(Int32) 1 \N +str String 0 SOR \N +uint64 UInt64 0 PRI SOR \N +--- +int32 Int32 0 \N +str String 0 \N +uint64 UInt64 0 PRI SOR \N +--- +int32 Int32 0 \N +str String 0 \N +uint64 UInt64 0 PRI SOR \N diff --git a/tests/queries/0_stateless/25402_show_columns.sql b/tests/queries/0_stateless/25402_show_columns.sql new file mode 100644 index 00000000000..baafecd7f68 --- /dev/null +++ b/tests/queries/0_stateless/25402_show_columns.sql @@ -0,0 +1,80 @@ +-- Tags: no-parallel +-- no-parallel: creates a custom database schema and expects to use it exclusively + +-- Create a test table and verify that the output of SHOW COLUMNS is sane. +-- The matching of actual/expected results relies on the fact that the output of SHOW COLUMNS is sorted. +CREATE OR REPLACE TABLE tab +( + `uint64` UInt64, + `int32` Nullable(Int32) COMMENT 'example comment', + `str` String, + INDEX idx str TYPE set(1000) +) +ENGINE = MergeTree +PRIMARY KEY (uint64) +ORDER BY (uint64, str); + +SHOW COLUMNS FROM tab; + +SELECT '---'; + +SHOW EXTENDED COLUMNS FROM tab; + +SELECT '---'; + +SHOW FULL COLUMNS FROM tab; + +SELECT '---'; + +SHOW COLUMNS FROM tab LIKE '%int%'; + +SELECT '---'; + +SHOW COLUMNS FROM tab NOT LIKE '%int%'; + +SELECT '---'; + +SHOW COLUMNS FROM tab ILIKE '%INT%'; + +SELECT '---'; + +SHOW COLUMNS FROM tab NOT ILIKE '%INT%'; + +SELECT '---'; + +SHOW COLUMNS FROM tab WHERE field LIKE '%int%'; + +SELECT '---'; + +SHOW COLUMNS FROM tab LIMIT 1; + +SELECT '---'; + + +-- Create a table in a different database. Intentionally useing the same table/column names as above so +-- we notice if something is buggy in the implementation of SHOW COLUMNS. +DROP DATABASE database_123456789abcde; +CREATE DATABASE database_123456789abcde; -- pseudo-random database name + +CREATE OR REPLACE TABLE database_123456789abcde.tab +( + `uint64` UInt64, + `int32` Int32, + `str` String +) +ENGINE = MergeTree +ORDER BY uint64; + +SHOW COLUMNS FROM tab; + +SELECT '---'; + +SHOW COLUMNS FROM tab FROM database_123456789abcde; + +SELECT '---'; + +SHOW COLUMNS FROM database_123456789abcde.tab; + +DROP DATABASE database_123456789abc; + +DROP TABLE tab; From a0fcf81abfed85bf10caa4eae28fe36a2f164a4d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 28 Mar 2023 18:25:52 +0000 Subject: [PATCH 071/233] Support more ClickHouse types in MsgPack format --- docs/en/interfaces/formats.md | 38 +++--- .../Formats/Impl/MsgPackRowInputFormat.cpp | 128 +++++++++++++++--- .../Formats/Impl/MsgPackRowInputFormat.h | 4 +- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 64 ++++++++- .../02594_msgpack_more_types.reference | 2 + .../0_stateless/02594_msgpack_more_types.sh | 11 ++ 6 files changed, 208 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/02594_msgpack_more_types.reference create mode 100755 tests/queries/0_stateless/02594_msgpack_more_types.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 8430946a6c6..d82f7c4ea3f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2281,22 +2281,28 @@ ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data ### Data Types Matching {#data-types-matching-msgpack} -| MessagePack data type (`INSERT`) | ClickHouse data type | MessagePack data type (`SELECT`) | -|--------------------------------------------------------------------|-----------------------------------------------------------------|------------------------------------| -| `uint N`, `positive fixint` | [UIntN](/docs/en/sql-reference/data-types/int-uint.md) | `uint N` | -| `int N`, `negative fixint` | [IntN](/docs/en/sql-reference/data-types/int-uint.md) | `int N` | -| `bool` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `uint 8` | -| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8`, `bin 16`, `bin 32` | -| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `bin 8`, `bin 16`, `bin 32` | -| `float 32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float 32` | -| `float 64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `float 64` | -| `uint 16` | [Date](/docs/en/sql-reference/data-types/date.md) | `uint 16` | -| `uint 32` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `uint 32` | -| `uint 64` | [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `uint 64` | -| `fixarray`, `array 16`, `array 32` | [Array](/docs/en/sql-reference/data-types/array.md) | `fixarray`, `array 16`, `array 32` | -| `fixmap`, `map 16`, `map 32` | [Map](/docs/en/sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` | -| `uint 32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `uint 32` | -| `bin 8` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8` | +| MessagePack data type (`INSERT`) | ClickHouse data type | MessagePack data type (`SELECT`) | +|--------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------|----------------------------------| +| `uint N`, `positive fixint` | [UIntN](/docs/en/sql-reference/data-types/int-uint.md) | `uint N` | +| `int N`, `negative fixint` | [IntN](/docs/en/sql-reference/data-types/int-uint.md) | `int N` | +| `bool` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `uint 8` | +| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8`, `bin 16`, `bin 32` | +| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `bin 8`, `bin 16`, `bin 32` | +| `float 32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float 32` | +| `float 64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `float 64` | +| `uint 16` | [Date](/docs/en/sql-reference/data-types/date.md) | `uint 16` | +| `int 32` | [Date32](/docs/en/sql-reference/data-types/date32.md) | `int 32` | +| `uint 32` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `uint 32` | +| `uint 64` | [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `uint 64` | +| `fixarray`, `array 16`, `array 32` | [Array](/docs/en/sql-reference/data-types/array.md)/[Tuple](/docs/en/sql-reference/data-types/tuple.md) | `fixarray`, `array 16`, `array 32` | +| `fixmap`, `map 16`, `map 32` | [Map](/docs/en/sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` | +| `uint 32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `uint 32` | +| `bin 8` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8` | +| `int 8` | [Enum8](/docs/en/sql-reference/data-types/enum.md) | `int 8` | +| `bin 8` | [(U)Int128/(U)Int256](/docs/en/sql-reference/data-types/int-uint.md) | `bin 8` | +| `int 32` | [Decimal32](/docs/en/sql-reference/data-types/decimal.md) | `int 32` | +| `int 64` | [Decimal64](/docs/en/sql-reference/data-types/decimal.md) | `int 64` | +| `bin 8` | [Decimal128/Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `bin 8 ` | Example: diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index bc41b512f79..7ce58b9991d 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -25,6 +26,7 @@ #include #include +#include #include #include #include @@ -64,7 +66,7 @@ void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type, UInt8 & read) { info_stack.pop(); } - info_stack.push(Info{column, type, &read}); + info_stack.push(Info{column, type, false, std::nullopt, &read}); } void MsgPackVisitor::reset() @@ -137,16 +139,19 @@ static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value) assert_cast(column).insertValue(value); break; } + case TypeIndex::Enum8: [[fallthrough]]; case TypeIndex::Int8: { assert_cast(column).insertValue(value); break; } + case TypeIndex::Enum16: [[fallthrough]]; case TypeIndex::Int16: { assert_cast(column).insertValue(value); break; } + case TypeIndex::Date32: [[fallthrough]]; case TypeIndex::Int32: { assert_cast(column).insertValue(static_cast(value)); @@ -167,11 +172,30 @@ static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value) assert_cast(column).insertValue(IPv4(static_cast(value))); break; } + case TypeIndex::Decimal32: + { + assert_cast &>(column).insertValue(static_cast(value)); + break; + } + case TypeIndex::Decimal64: + { + assert_cast &>(column).insertValue(value); + break; + } default: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack integer into column with type {}.", type->getName()); } } +template +static void insertFromBinaryRepresentation(IColumn & column, DataTypePtr type, const char * value, size_t size) +{ + if (size != sizeof(typename ColumnType::ValueType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", type->getName(), size); + + assert_cast(column).insertData(value, size); +} + static void insertString(IColumn & column, DataTypePtr type, const char * value, size_t size, bool bin) { auto insert_func = [&](IColumn & column_, DataTypePtr type_) @@ -195,10 +219,33 @@ static void insertString(IColumn & column, DataTypePtr type, const char * value, return; } - if (isIPv6(type) && bin) + if (bin) { - assert_cast(column).insertData(value, size); - return; + switch (type->getTypeId()) + { + case TypeIndex::IPv6: + insertFromBinaryRepresentation(column, type, value, size); + return; + case TypeIndex::Int128: + insertFromBinaryRepresentation(column, type, value, size); + return; + case TypeIndex::UInt128: + insertFromBinaryRepresentation(column, type, value, size); + return; + case TypeIndex::Int256: + insertFromBinaryRepresentation(column, type, value, size); + return; + case TypeIndex::UInt256: + insertFromBinaryRepresentation(column, type, value, size); + return; + case TypeIndex::Decimal128: + insertFromBinaryRepresentation>(column, type, value, size); + return; + case TypeIndex::Decimal256: + insertFromBinaryRepresentation>(column, type, value, size); + return; + default:; + } } if (!isStringOrFixedString(type)) @@ -328,21 +375,47 @@ bool MsgPackVisitor::visit_boolean(bool value) bool MsgPackVisitor::start_array(size_t size) // NOLINT { - if (!isArray(info_stack.top().type)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack array into column with type {}.", info_stack.top().type->getName()); + if (isArray(info_stack.top().type)) + { + auto nested_type = assert_cast(*info_stack.top().type).getNestedType(); + ColumnArray & column_array = assert_cast(info_stack.top().column); + ColumnArray::Offsets & offsets = column_array.getOffsets(); + IColumn & nested_column = column_array.getData(); + offsets.push_back(offsets.back() + size); + if (size > 0) + info_stack.push(Info{nested_column, nested_type, false, size, nullptr}); + } + else if (isTuple(info_stack.top().type)) + { + const auto & tuple_type = assert_cast(*info_stack.top().type); + const auto & nested_types = tuple_type.getElements(); + if (size != nested_types.size()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack array with size {} into Tuple column with {} elements", size, nested_types.size()); + + ColumnTuple & column_tuple = assert_cast(info_stack.top().column); + /// Push nested columns into stack in reverse order. + for (ssize_t i = nested_types.size() - 1; i >= 0; --i) + info_stack.push(Info{column_tuple.getColumn(i), nested_types[i], true, std::nullopt, nullptr}); + } + else + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack array into column with type {}", info_stack.top().type->getName()); + } - auto nested_type = assert_cast(*info_stack.top().type).getNestedType(); - ColumnArray & column_array = assert_cast(info_stack.top().column); - ColumnArray::Offsets & offsets = column_array.getOffsets(); - IColumn & nested_column = column_array.getData(); - offsets.push_back(offsets.back() + size); - info_stack.push(Info{nested_column, nested_type, nullptr}); return true; } -bool MsgPackVisitor::end_array() // NOLINT + +bool MsgPackVisitor::end_array_item() // NOLINT { - info_stack.pop(); + if (info_stack.top().is_tuple_element) + info_stack.pop(); + else + { + --(*info_stack.top().array_size); + if (*info_stack.top().array_size == 0) + info_stack.pop(); + } return true; } @@ -360,7 +433,7 @@ bool MsgPackVisitor::start_map_key() // NOLINT { auto key_column = assert_cast(info_stack.top().column).getNestedData().getColumns()[0]; auto key_type = assert_cast(*info_stack.top().type).getKeyType(); - info_stack.push(Info{*key_column, key_type, nullptr}); + info_stack.push(Info{*key_column, key_type, false, std::nullopt, nullptr}); return true; } @@ -374,7 +447,7 @@ bool MsgPackVisitor::start_map_value() // NOLINT { auto value_column = assert_cast(info_stack.top().column).getNestedData().getColumns()[1]; auto value_type = assert_cast(*info_stack.top().type).getValueType(); - info_stack.push(Info{*value_column, value_type, nullptr}); + info_stack.push(Info{*value_column, value_type, false, std::nullopt, nullptr}); return true; } @@ -513,13 +586,26 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) case msgpack::type::object_type::ARRAY: { msgpack::object_array object_array = object.via.array; - if (object_array.size) + if (!object_array.size) + return nullptr; + + DataTypes nested_types; + nested_types.reserve(object_array.size); + bool nested_types_are_equal = true; + for (size_t i = 0; i != object_array.size; ++i) { - auto nested_type = getDataType(object_array.ptr[0]); - if (nested_type) - return std::make_shared(getDataType(object_array.ptr[0])); + auto nested_type = getDataType(object_array.ptr[i]); + if (!nested_type) + return nullptr; + + nested_types.push_back(nested_type); + nested_types_are_equal &= nested_type->equals(*nested_types[0]); } - return nullptr; + + if (nested_types_are_equal) + return std::make_shared(nested_types[0]); + + return std::make_shared(std::move(nested_types)); } case msgpack::type::object_type::MAP: { diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index 5eaa3719d0c..0b485d3b97c 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -25,6 +25,8 @@ public: { IColumn & column; DataTypePtr type; + bool is_tuple_element; + std::optional array_size; UInt8 * read; }; @@ -37,7 +39,7 @@ public: bool visit_bin(const char * value, size_t size); bool visit_boolean(bool value); bool start_array(size_t size); - bool end_array(); + bool end_array_item(); bool visit_nil(); bool start_map(uint32_t size); bool start_map_key(); diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index 07951d42bc6..9c601492217 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -9,12 +9,14 @@ #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -66,16 +68,19 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_uint64(assert_cast(column).getElement(row_num)); return; } + case TypeIndex::Enum8: [[fallthrough]]; case TypeIndex::Int8: { packer.pack_int8(assert_cast(column).getElement(row_num)); return; } + case TypeIndex::Enum16: [[fallthrough]]; case TypeIndex::Int16: { packer.pack_int16(assert_cast(column).getElement(row_num)); return; } + case TypeIndex::Date32: [[fallthrough]]; case TypeIndex::Int32: { packer.pack_int32(assert_cast(column).getElement(row_num)); @@ -86,6 +91,30 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_int64(assert_cast(column).getElement(row_num)); return; } + case TypeIndex::Int128: + { + packer.pack_bin(static_cast(sizeof(Int128))); + packer.pack_bin_body(column.getDataAt(row_num).data, sizeof(Int128)); + return; + } + case TypeIndex::UInt128: + { + packer.pack_bin(static_cast(sizeof(UInt128))); + packer.pack_bin_body(column.getDataAt(row_num).data, sizeof(UInt128)); + return; + } + case TypeIndex::Int256: + { + packer.pack_bin(static_cast(sizeof(Int256))); + packer.pack_bin_body(column.getDataAt(row_num).data, sizeof(Int256)); + return; + } + case TypeIndex::UInt256: + { + packer.pack_bin(static_cast(sizeof(UInt256))); + packer.pack_bin_body(column.getDataAt(row_num).data, sizeof(UInt256)); + return; + } case TypeIndex::Float32: { packer.pack_float(assert_cast(column).getElement(row_num)); @@ -101,6 +130,28 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_uint64(assert_cast(column).getElement(row_num)); return; } + case TypeIndex::Decimal32: + { + packer.pack_int32(assert_cast &>(column).getElement(row_num)); + return; + } + case TypeIndex::Decimal64: + { + packer.pack_int64(assert_cast &>(column).getElement(row_num)); + return; + } + case TypeIndex::Decimal128: + { + packer.pack_bin(static_cast(sizeof(Decimal128))); + packer.pack_bin_body(column.getDataAt(row_num).data, sizeof(Decimal128)); + return; + } + case TypeIndex::Decimal256: + { + packer.pack_bin(static_cast(sizeof(Decimal256))); + packer.pack_bin_body(column.getDataAt(row_num).data, sizeof(Decimal256)); + return; + } case TypeIndex::String: { const std::string_view & string = assert_cast(column).getDataAt(row_num).toView(); @@ -136,7 +187,18 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr serializeField(nested_column, nested_type, offset + i); } return; - } + } + case TypeIndex::Tuple: + { + const auto & tuple_type = assert_cast(*data_type); + const auto & nested_types = tuple_type.getElements(); + const ColumnTuple & column_tuple = assert_cast(column); + const auto & nested_columns = column_tuple.getColumns(); + packer.pack_array(static_cast(nested_types.size())); + for (size_t i = 0; i < nested_types.size(); ++i) + serializeField(*nested_columns[i], nested_types[i], row_num); + return; + } case TypeIndex::Nullable: { auto nested_type = removeNullable(data_type); diff --git a/tests/queries/0_stateless/02594_msgpack_more_types.reference b/tests/queries/0_stateless/02594_msgpack_more_types.reference new file mode 100644 index 00000000000..8ccf11ccdb4 --- /dev/null +++ b/tests/queries/0_stateless/02594_msgpack_more_types.reference @@ -0,0 +1,2 @@ +a b 2020-01-01 42 42 42 42 42.42 42.42 42.42 42.42 +(42,'Hello') ({42:[1,2,3]},[([(1,2),(1,2)],'Hello',[1,2,3]),([],'World',[1])]) diff --git a/tests/queries/0_stateless/02594_msgpack_more_types.sh b/tests/queries/0_stateless/02594_msgpack_more_types.sh new file mode 100755 index 00000000000..bddfb5ad829 --- /dev/null +++ b/tests/queries/0_stateless/02594_msgpack_more_types.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select 'a'::Enum8('a' = 1) as c1, 'b'::Enum16('b' = 1) as c2, '2020-01-01'::Date32 as c3, 42::Int128 as c4, 42::UInt128 as c5, 42::Int256 as c6, 42::UInt256 as c7, 42.42::Decimal32(2) as c8, 42.42::Decimal64(2) as c9, 42.42::Decimal128(2) as c10, 42.42::Decimal256(2) as c11 format MsgPack" | $CLICKHOUSE_LOCAL --input-format MsgPack --structure="c1 Enum8('a' = 1), c2 Enum16('b' = 1), c3 Date32, c4 Int128, c5 UInt128, c6 Int256, c7 UInt256, c8 Decimal32(2), c9 Decimal64(2), c10 Decimal128(2), c11 Decimal256(2)" -q "select * from table" + +$CLICKHOUSE_LOCAL -q "select tuple(42, 'Hello') as c1, tuple(map(42, [1, 2, 3]), [tuple([tuple(1, 2), tuple(1, 2)], 'Hello', [1, 2, 3]), tuple([], 'World', [1])]) as c2 format MsgPack" | $CLICKHOUSE_LOCAL --input-format MsgPack --structure="c1 Tuple(UInt32, String), c2 Tuple(Map(UInt32, Array(UInt32)), Array(Tuple(Array(Tuple(UInt32, UInt32)), String, Array(UInt32))))" -q "select * from table" + From 0f241f71a3b761fa2d85bd821c9ff4d8b2ca050e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Mar 2023 20:03:55 +0000 Subject: [PATCH 072/233] Fix fasttest --- tests/queries/0_stateless/25402_show_columns.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/25402_show_columns.sql b/tests/queries/0_stateless/25402_show_columns.sql index baafecd7f68..6cf2a599f2c 100644 --- a/tests/queries/0_stateless/25402_show_columns.sql +++ b/tests/queries/0_stateless/25402_show_columns.sql @@ -53,7 +53,7 @@ SELECT '---'; -- Create a table in a different database. Intentionally useing the same table/column names as above so -- we notice if something is buggy in the implementation of SHOW COLUMNS. -DROP DATABASE database_123456789abcde; +DROP DATABASE IF EXISTS database_123456789abcde; CREATE DATABASE database_123456789abcde; -- pseudo-random database name CREATE OR REPLACE TABLE database_123456789abcde.tab @@ -75,6 +75,6 @@ SELECT '---'; SHOW COLUMNS FROM database_123456789abcde.tab; -DROP DATABASE database_123456789abc; +DROP DATABASE database_123456789abcde; DROP TABLE tab; From e743f840f02b5d0a210ab95de3fe2f7e0f225100 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Mar 2023 20:05:17 +0000 Subject: [PATCH 073/233] Fix taipos --- src/Interpreters/InterpreterShowColumnsQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 95530ad7c53..c5392ccdb15 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -53,11 +53,11 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() { /// FROM .
(abbreviated form) chassert(query.from_database.empty()); - std::vector splitted; - boost::split(splitted, query.from_table, boost::is_any_of(".")); - chassert(splitted.size() == 2); - database = splitted[0]; - table = splitted[1]; + std::vector split; + boost::split(split, query.from_table, boost::is_any_of(".")); + chassert(split.size() == 2); + database = split[0]; + table = split[1]; } else if (query.from_database.empty()) { From 6c84eabb79f0813be0c333ecb0effaa89f3600c7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Mar 2023 20:07:26 +0000 Subject: [PATCH 074/233] Fix stylecheck --- src/Interpreters/InterpreterShowColumnsQuery.cpp | 5 ----- src/Parsers/ASTShowColumnsQuery.cpp | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index c5392ccdb15..4474be21d8b 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -12,11 +12,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; -} - InterpreterShowColumnsQuery::InterpreterShowColumnsQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_) diff --git a/src/Parsers/ASTShowColumnsQuery.cpp b/src/Parsers/ASTShowColumnsQuery.cpp index 113029aacd7..2b3e1a58c71 100644 --- a/src/Parsers/ASTShowColumnsQuery.cpp +++ b/src/Parsers/ASTShowColumnsQuery.cpp @@ -32,7 +32,7 @@ void ASTShowColumnsQuery::formatQueryImpl(const FormatSettings & settings, Forma if (!like.empty()) settings.ostr << (settings.hilite ? hilite_keyword : "") - << (not_like ? " NOT " : "") + << (not_like ? " NOT " : "") << (case_insensitive_like ? " ILIKE " : " LIKE" ) << (settings.hilite ? hilite_none : "") << DB::quote << like; From 12559236eeb3de493ddda1da035064aa9591e158 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Mar 2023 21:01:17 +0000 Subject: [PATCH 075/233] Another style fix --- src/Parsers/ASTShowColumnsQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTShowColumnsQuery.cpp b/src/Parsers/ASTShowColumnsQuery.cpp index 2b3e1a58c71..d14cbdc9b84 100644 --- a/src/Parsers/ASTShowColumnsQuery.cpp +++ b/src/Parsers/ASTShowColumnsQuery.cpp @@ -33,7 +33,7 @@ void ASTShowColumnsQuery::formatQueryImpl(const FormatSettings & settings, Forma if (!like.empty()) settings.ostr << (settings.hilite ? hilite_keyword : "") << (not_like ? " NOT " : "") - << (case_insensitive_like ? " ILIKE " : " LIKE" ) + << (case_insensitive_like ? " ILIKE " : " LIKE") << (settings.hilite ? hilite_none : "") << DB::quote << like; From 4b0b5301beac23a22bb20e192a286700b0b93a8c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 00:55:03 +0200 Subject: [PATCH 076/233] improve script for updating clickhouse-docs --- docs/get-clickhouse-docs.sh | 22 ++++++++++++++++++++++ docs/pull-clickhouse-docs-hook.sh | 27 +++++++++++++++++++++++++++ 2 files changed, 49 insertions(+) mode change 100644 => 100755 docs/get-clickhouse-docs.sh create mode 100755 docs/pull-clickhouse-docs-hook.sh diff --git a/docs/get-clickhouse-docs.sh b/docs/get-clickhouse-docs.sh old mode 100644 new mode 100755 index 1ba0dae9844..7868f281bd0 --- a/docs/get-clickhouse-docs.sh +++ b/docs/get-clickhouse-docs.sh @@ -27,5 +27,27 @@ else exit 1 ;; esac + + if [ -n "$2" ]; then + set_git_hook="$2" + elif [ ! -n "$1" ]; then + read -rp "Would you like to setup git hook for automatic update? (y|n): " set_git_hook + fi + + if [ "$set_git_hook" = "y" ]; then + hook_command="$(pwd)/pull-clickhouse-docs-hook.sh 24" + hook_file=$(realpath "$(pwd)/../.git/hooks/post-checkout") + already_have=$(grep -Faq "pull-clickhouse-docs-hook.sh" "$hook_file" 2>/dev/null && echo 1 || echo 0) + if [ $already_have -eq 0 ]; then + echo "Appending '$hook_command' to $hook_file" + echo "$hook_command" >> "$hook_file" + chmod u+x "$hook_file" # Just in case it did not exist before append + else + echo "Looks like the update hook already exists, will not add another one" + fi + elif [ ! "$set_git_hook" = "n" ]; then + echo "Expected 'y' or 'n', got '$set_git_hook', will not setup git hook" + fi + git clone "$git_url" "clickhouse-docs" fi diff --git a/docs/pull-clickhouse-docs-hook.sh b/docs/pull-clickhouse-docs-hook.sh new file mode 100755 index 00000000000..bd93a1d3997 --- /dev/null +++ b/docs/pull-clickhouse-docs-hook.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +set -e +# The script to update user-guides documentation repo +# https://github.com/ClickHouse/clickhouse-docs + +WORKDIR=$(dirname "$0") +WORKDIR=$(readlink -f "${WORKDIR}") +cd "$WORKDIR" + +UPDATE_PERIOD_HOURS="${1:-24}" # By default update once per 24 hours; 0 means "always update" + +if [ ! -d "clickhouse-docs" ]; then + echo "There's no clickhouse-docs/ dir, run get-clickhouse-docs.sh first to clone the repo" + exit 1 +fi + +# Do not update it too often +LAST_FETCH_TS=$(stat -c %Y clickhouse-docs/.git/FETCH_HEAD 2>/dev/null || echo 0) +CURRENT_TS=$(date +%s) +HOURS_SINCE_LAST_FETCH=$(( (CURRENT_TS - LAST_FETCH_TS) / 60 / 60 )) + +if [ "$HOURS_SINCE_LAST_FETCH" -lt "$UPDATE_PERIOD_HOURS" ]; then + exit 0; +fi + +echo "Updating clickhouse-docs..." +git -C clickhouse-docs pull From 81af0b6deb0b4e477c61dab704368d5ac619258d Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 29 Mar 2023 00:29:21 +0000 Subject: [PATCH 077/233] avoid counters updates if not initialized --- src/Common/ThreadStatus.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 16ce73cda20..1b783aa9ec4 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -219,6 +219,9 @@ void ThreadStatus::updatePerformanceCounters() void ThreadStatus::updatePerformanceCountersIfNeeded() { + if (last_rusage->thread_id == 0) + return; // Performance counters are not initialized, so there is no need to update them + constexpr UInt64 performance_counters_update_period_microseconds = 10 * 1000; // 10 milliseconds UInt64 total_elapsed_microseconds = stopwatch.elapsedMicroseconds(); if (last_performance_counters_update_time + performance_counters_update_period_microseconds < total_elapsed_microseconds) From 2b84fb3fffe2c409407506bab719ff03620f6c91 Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Wed, 29 Mar 2023 11:21:30 +0800 Subject: [PATCH 078/233] fix tests --- ...function.reference => 02701_non_parametric_function.reference} | 0 ..._parametric_function.sql => 02701_non_parametric_function.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25403_non_parametric_function.reference => 02701_non_parametric_function.reference} (100%) rename tests/queries/0_stateless/{25403_non_parametric_function.sql => 02701_non_parametric_function.sql} (100%) diff --git a/tests/queries/0_stateless/25403_non_parametric_function.reference b/tests/queries/0_stateless/02701_non_parametric_function.reference similarity index 100% rename from tests/queries/0_stateless/25403_non_parametric_function.reference rename to tests/queries/0_stateless/02701_non_parametric_function.reference diff --git a/tests/queries/0_stateless/25403_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql similarity index 100% rename from tests/queries/0_stateless/25403_non_parametric_function.sql rename to tests/queries/0_stateless/02701_non_parametric_function.sql From 7078576ee52f6834228be3d1bdb4e67d92afd745 Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Wed, 29 Mar 2023 12:58:35 +0800 Subject: [PATCH 079/233] fix tests --- tests/queries/0_stateless/02701_non_parametric_function.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql index 9b7e2ae7a04..9946b50a8c8 100644 --- a/tests/queries/0_stateless/02701_non_parametric_function.sql +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -2,4 +2,6 @@ SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { CREATE FUNCTION sum_udf as (x, y) -> (x + y); -SELECT sum_udf(1)(1, 2); -- { serverError 309 } \ No newline at end of file +SELECT sum_udf(1)(1, 2); -- { serverError 309 } + +DROP FUNCTION sum_udf; \ No newline at end of file From 3507c4998e3a0c2eef6c88c89e06cf66202de1fd Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Wed, 29 Mar 2023 14:07:45 +0800 Subject: [PATCH 080/233] fix tests --- tests/queries/0_stateless/02701_non_parametric_function.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql index 9946b50a8c8..1164a060323 100644 --- a/tests/queries/0_stateless/02701_non_parametric_function.sql +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -1,6 +1,6 @@ SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } -CREATE FUNCTION sum_udf as (x, y) -> (x + y); +CREATE FUNCTION IF NOT EXISTS sum_udf as (x, y) -> (x + y); SELECT sum_udf(1)(1, 2); -- { serverError 309 } From 6738be00298db5095a6b0837b58e4c0337dd1888 Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Wed, 29 Mar 2023 16:14:51 +0800 Subject: [PATCH 081/233] fix tests --- tests/queries/0_stateless/02701_non_parametric_function.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql index 1164a060323..a757343a624 100644 --- a/tests/queries/0_stateless/02701_non_parametric_function.sql +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -4,4 +4,4 @@ CREATE FUNCTION IF NOT EXISTS sum_udf as (x, y) -> (x + y); SELECT sum_udf(1)(1, 2); -- { serverError 309 } -DROP FUNCTION sum_udf; \ No newline at end of file +DROP FUNCTION IF EXISTS sum_udf; \ No newline at end of file From 36f6408ada36c03968a5b86f8a47da7d43f32cb4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Mar 2023 08:33:45 +0100 Subject: [PATCH 082/233] Add sanity checks for writing number in variable length format And just to double check: # var_uint 9223372036854775807 ffffffffffffffff7f ffffffffffffffff7f ffffffffffffffff7f x: 9223372036854775807, y: 9223372036854775807 # var_uint 9223372036854775808 808080808080808080 808080808080808080 808080808080808080 x: 9223372036854775808, y: 0 Signed-off-by: Azat Khuzhin --- src/IO/VarInt.h | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index 0869051034a..d026192cb7d 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -14,7 +15,19 @@ namespace ErrorCodes } -/** Write UInt64 in variable length format (base128) NOTE Only up to 2^63 - 1 are supported. */ +/** Variable-Length Quantity (VLQ) Base-128 compression + * + * NOTE: Due to historical reasons, only up to 1<<63-1 are supported, which + * cannot be changed without breaking the backward compatibility. + * Also some drivers may support full 1<<64 range (i.e. python - + * clickhouse-driver), while others has the same limitations as ClickHouse + * (i.e. Rust - clickhouse-rs). + * So implementing VLQ for the whole 1<<64 range will require different set of + * helpers. + */ +constexpr size_t VAR_UINT_MAX = (1ULL<<63) - 1; + +/** Write UInt64 in variable length format (base128) */ void writeVarUInt(UInt64 x, std::ostream & ostr); void writeVarUInt(UInt64 x, WriteBuffer & ostr); char * writeVarUInt(UInt64 x, char * ostr); @@ -186,6 +199,7 @@ inline const char * readVarUInt(UInt64 & x, const char * istr, size_t size) inline void writeVarUInt(UInt64 x, WriteBuffer & ostr) { + chassert(x <= VAR_UINT_MAX); for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; @@ -205,6 +219,7 @@ inline void writeVarUInt(UInt64 x, WriteBuffer & ostr) inline void writeVarUInt(UInt64 x, std::ostream & ostr) { + chassert(x <= VAR_UINT_MAX); for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; @@ -222,6 +237,7 @@ inline void writeVarUInt(UInt64 x, std::ostream & ostr) inline char * writeVarUInt(UInt64 x, char * ostr) { + chassert(x <= VAR_UINT_MAX); for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; From 9457b1cc46f08aca385eea3c86578e561b80ba38 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Mar 2023 13:10:35 +0100 Subject: [PATCH 083/233] Use VAR_UINT_MAX for unknown_packet_in_send_data Signed-off-by: Azat Khuzhin --- src/Server/TCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 85cdb75977b..941bbe4fffe 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1895,7 +1895,7 @@ void TCPHandler::sendData(const Block & block) { --unknown_packet_in_send_data; if (unknown_packet_in_send_data == 0) - writeVarUInt(UInt64(-1), *out); + writeVarUInt(VAR_UINT_MAX, *out); } writeVarUInt(Protocol::Server::Data, *out); From fc5406d9e2ad2fc4806afc77aa7e5a9fb54e4155 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 29 Mar 2023 09:39:58 +0000 Subject: [PATCH 084/233] Fix 25402_show_columns (hopefully) --- tests/queries/0_stateless/25402_show_columns.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/25402_show_columns.sql b/tests/queries/0_stateless/25402_show_columns.sql index 6cf2a599f2c..28ac54bd193 100644 --- a/tests/queries/0_stateless/25402_show_columns.sql +++ b/tests/queries/0_stateless/25402_show_columns.sql @@ -3,7 +3,8 @@ -- Create a test table and verify that the output of SHOW COLUMNS is sane. -- The matching of actual/expected results relies on the fact that the output of SHOW COLUMNS is sorted. -CREATE OR REPLACE TABLE tab +DROP TABLE IF EXISTS tab; +CREATE TABLE tab ( `uint64` UInt64, `int32` Nullable(Int32) COMMENT 'example comment', @@ -56,7 +57,8 @@ SELECT '---'; DROP DATABASE IF EXISTS database_123456789abcde; CREATE DATABASE database_123456789abcde; -- pseudo-random database name -CREATE OR REPLACE TABLE database_123456789abcde.tab +DROP TABLE IF EXISTS database_123456789abcde.tab; +CREATE TABLE database_123456789abcde.tab ( `uint64` UInt64, `int32` Int32, From d7d9f0cb6b3b21d5116242c1f58ce950dae7bf31 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 29 Mar 2023 11:11:42 +0200 Subject: [PATCH 085/233] Fix overflow of VarUInt format in Progress packets Otherwise query like this, can trigger sanity check: WITH x AS (SELECT [], number AS a FROM numbers(9223372036854775807)), y AS (SELECT arrayLastOrNull(x -> (x >= -inf), []), arrayLastOrNull(x -> (x >= NULL), [1]), number AS a FROM numbers(1.)) SELECT [1023], * FROM x WHERE a IN (SELECT a FROM y) ORDER BY arrayLastOrNull(x -> (x >= 1025), [1048577, 1048576]) DESC NULLS LAST, '0.0000000002' ASC NULLS LAST, a DESC NULLS FIRST CI: https://s3.amazonaws.com/clickhouse-test-reports/0/a9bcd022d5f4a5be530595dbfae3ed177b5c1972/fuzzer_astfuzzermsan/report.html Signed-off-by: Azat Khuzhin --- src/IO/Progress.cpp | 10 +++++----- src/IO/VarInt.h | 8 ++++++++ src/IO/examples/var_uint.cpp | 14 ++++++++++---- 3 files changed, 23 insertions(+), 9 deletions(-) diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index 1069803633c..1dd530b4c81 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -28,13 +28,13 @@ void ProgressValues::read(ReadBuffer & in, UInt64 server_revision) void ProgressValues::write(WriteBuffer & out, UInt64 client_revision) const { - writeVarUInt(read_rows, out); - writeVarUInt(read_bytes, out); - writeVarUInt(total_rows_to_read, out); + writeVarUIntOverflow(read_rows, out); + writeVarUIntOverflow(read_bytes, out); + writeVarUIntOverflow(total_rows_to_read, out); if (client_revision >= DBMS_MIN_REVISION_WITH_CLIENT_WRITE_INFO) { - writeVarUInt(written_rows, out); - writeVarUInt(written_bytes, out); + writeVarUIntOverflow(written_rows, out); + writeVarUIntOverflow(written_bytes, out); } if (client_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS) { diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index d026192cb7d..f95b479df11 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -33,6 +33,14 @@ void writeVarUInt(UInt64 x, WriteBuffer & ostr); char * writeVarUInt(UInt64 x, char * ostr); +/** Write UInt64 in variable length format, wrap the value to VAR_UINT_MAX if it exceed VAR_UINT_MAX (to bypass sanity check) */ +template +auto writeVarUIntOverflow(UInt64 x, Args && ... args) +{ + return writeVarUInt(std::min(x, VAR_UINT_MAX), std::forward(args)...); +} + + /** Read UInt64, written in variable length format (base128) */ void readVarUInt(UInt64 & x, std::istream & istr); void readVarUInt(UInt64 & x, ReadBuffer & istr); diff --git a/src/IO/examples/var_uint.cpp b/src/IO/examples/var_uint.cpp index 65e1f0495d3..06c707b4a0a 100644 --- a/src/IO/examples/var_uint.cpp +++ b/src/IO/examples/var_uint.cpp @@ -18,26 +18,32 @@ int main(int argc, char ** argv) } DB::UInt64 x = DB::parse(argv[1]); + + std::cout << std::hex << std::showbase << "Input: " << x << std::endl; + Poco::HexBinaryEncoder hex(std::cout); - DB::writeVarUInt(x, hex); + std::cout << "writeVarUIntOverflow(std::ostream): 0x"; + DB::writeVarUIntOverflow(x, hex); std::cout << std::endl; std::string s; { DB::WriteBufferFromString wb(s); - DB::writeVarUInt(x, wb); + DB::writeVarUIntOverflow(x, wb); wb.next(); } + std::cout << "writeVarUIntOverflow(WriteBuffer): 0x"; hex << s; std::cout << std::endl; s.clear(); s.resize(9); - s.resize(DB::writeVarUInt(x, s.data()) - s.data()); + s.resize(DB::writeVarUIntOverflow(x, s.data()) - s.data()); + std::cout << "writeVarUIntOverflow(char *): 0x"; hex << s; std::cout << std::endl; @@ -46,7 +52,7 @@ int main(int argc, char ** argv) DB::ReadBufferFromString rb(s); DB::readVarUInt(y, rb); - std::cerr << "x: " << x << ", y: " << y << std::endl; + std::cerr << "Input: " << x << ", readVarUInt(writeVarUIntOverflow()): " << y << std::endl; return 0; } From 7b1ad221b259abf9ceec9d0e1e48840f456bf4c0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Mar 2023 11:08:44 +0000 Subject: [PATCH 086/233] Address PR comments --- .../engines/table-engines/integrations/s3.md | 11 +++----- src/IO/S3Common.cpp | 12 ++++++--- src/Storages/StorageS3.cpp | 4 +++ src/TableFunctions/TableFunctionS3.cpp | 2 +- .../configs/named_collections.xml | 4 +++ .../test_invalid_env_credentials.py | 25 ++++++++++++++++++- 6 files changed, 45 insertions(+), 13 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index c539340d332..7b2df1b98fb 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -12,7 +12,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec ``` sql CREATE TABLE s3_engine_table (name String, value UInt32) - ENGINE = S3(path, [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression]) + ENGINE = S3(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression]) [PARTITION BY expr] [SETTINGS ...] ``` @@ -185,13 +185,8 @@ Sometimes, it can produce problems when accessing some buckets that are public c This issue can be avoided by using `NOSIGN` keyword, forcing the client to ignore all the credentials, and not sign the requests. ``` sql -SELECT * -FROM s3( - 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', - NOSIGN, - 'CSVWithNames' -) -LIMIT 5; +CREATE TABLE big_table (name String, value UInt32) + ENGINE = S3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', NOSIGN, 'CSVWithNames'); ``` ## See also diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 20984b69463..2f99c523429 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -135,9 +135,15 @@ void AuthSettings::updateFrom(const AuthSettings & from) headers = from.headers; region = from.region; server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64; - use_environment_credentials = from.use_environment_credentials; - use_insecure_imds_request = from.use_insecure_imds_request; - expiration_window_seconds = from.expiration_window_seconds; + + if (from.use_environment_credentials) + use_environment_credentials = from.use_environment_credentials; + + if (from.use_insecure_imds_request) + use_insecure_imds_request = from.use_insecure_imds_request; + + if (from.expiration_window_seconds) + expiration_window_seconds = from.expiration_window_seconds; if (from.no_sign_request.has_value()) no_sign_request = *from.no_sign_request; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f2a9b3a3955..b735e090f59 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -96,6 +96,8 @@ static const std::unordered_set optional_configuration_keys = "upload_part_size_multiply_parts_count_threshold", "max_single_part_upload_size", "max_connections", + "expiration_window_seconds", + "no_sign_request" }; namespace ErrorCodes @@ -1289,6 +1291,8 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 0); + configuration.auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", 0); + configuration.auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); configuration.format = collection.getOrDefault("format", "auto"); configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 9f5d992e4c9..9ce1ac439af 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -112,7 +112,7 @@ void TableFunctionS3::parseArgumentsImpl( /// For 5 arguments we support 2 possible variants: /// - s3(source, access_key_id, access_key_id, format, structure) /// - s3(source, NOSIGN, format, structure, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not. else if (args.size() == 5) { auto second_arg = checkAndGetLiteralArgument(args[1], "NOSIGN/access_key_id"); diff --git a/tests/integration/test_storage_s3/configs/named_collections.xml b/tests/integration/test_storage_s3/configs/named_collections.xml index fcc8bcac555..64674e2a3e3 100644 --- a/tests/integration/test_storage_s3/configs/named_collections.xml +++ b/tests/integration/test_storage_s3/configs/named_collections.xml @@ -35,5 +35,9 @@ minio minio123 + + http://minio1:9001/root/test_cache4.jsonl + true + diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 06d49d8e828..0ee679014b1 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -88,7 +88,10 @@ def started_cluster(): "AWS_ACCESS_KEY_ID": "aws", "AWS_SECRET_ACCESS_KEY": "aws123", }, - main_configs=["configs/use_environment_credentials.xml"], + main_configs=[ + "configs/use_environment_credentials.xml", + "configs/named_collections.xml", + ], ) logging.info("Starting cluster...") @@ -129,3 +132,23 @@ def test_with_invalid_environment_credentials(started_cluster): f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth})" ).strip() ) + + +def test_no_sign_named_collections(started_cluster): + instance = started_cluster.instances["s3_with_invalid_environment_credentials"] + + bucket = started_cluster.minio_bucket + + instance.query( + f"insert into function s3(s3_json_no_sign) select * from numbers(100) settings s3_truncate_on_insert=1" + ) + + with pytest.raises(helpers.client.QueryRuntimeException) as ei: + instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl')" + ) + + assert ei.value.returncode == 243 + assert "HTTP response code: 403" in ei.value.stderr + + assert "100" == instance.query(f"select count() from s3(s3_json_no_sign)").strip() From f4b884a5a82de0c0164f8ad53b2f47fd766a3c93 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 29 Mar 2023 13:55:29 +0200 Subject: [PATCH 087/233] Fix test flakiness --- tests/queries/0_stateless/02701_non_parametric_function.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql index a757343a624..5261fa7b082 100644 --- a/tests/queries/0_stateless/02701_non_parametric_function.sql +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -1,7 +1,9 @@ +-- Tags: no-parallel + SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } CREATE FUNCTION IF NOT EXISTS sum_udf as (x, y) -> (x + y); SELECT sum_udf(1)(1, 2); -- { serverError 309 } -DROP FUNCTION IF EXISTS sum_udf; \ No newline at end of file +DROP FUNCTION IF EXISTS sum_udf; From 8b8159d58f868213b99d2669b266a5c717e17ee9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 14:04:25 +0200 Subject: [PATCH 088/233] fix --- docs/get-clickhouse-docs.sh | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/get-clickhouse-docs.sh b/docs/get-clickhouse-docs.sh index 7868f281bd0..e3b70dee665 100755 --- a/docs/get-clickhouse-docs.sh +++ b/docs/get-clickhouse-docs.sh @@ -7,6 +7,8 @@ WORKDIR=$(dirname "$0") WORKDIR=$(readlink -f "${WORKDIR}") cd "$WORKDIR" +UPDATE_PERIOD_HOURS=${UPDATE_PERIOD_HOURS:=24} + if [ -d "clickhouse-docs" ]; then git -C clickhouse-docs pull else @@ -30,20 +32,20 @@ else if [ -n "$2" ]; then set_git_hook="$2" - elif [ ! -n "$1" ]; then + elif [ -z "$1" ]; then read -rp "Would you like to setup git hook for automatic update? (y|n): " set_git_hook fi if [ "$set_git_hook" = "y" ]; then - hook_command="$(pwd)/pull-clickhouse-docs-hook.sh 24" + hook_command="$(pwd)/pull-clickhouse-docs-hook.sh $UPDATE_PERIOD_HOURS" hook_file=$(realpath "$(pwd)/../.git/hooks/post-checkout") already_have=$(grep -Faq "pull-clickhouse-docs-hook.sh" "$hook_file" 2>/dev/null && echo 1 || echo 0) - if [ $already_have -eq 0 ]; then + if grep -Faq "pull-clickhouse-docs-hook.sh" "$hook_file" 2>/dev/null; then + echo "Looks like the update hook already exists, will not add another one" + else echo "Appending '$hook_command' to $hook_file" echo "$hook_command" >> "$hook_file" chmod u+x "$hook_file" # Just in case it did not exist before append - else - echo "Looks like the update hook already exists, will not add another one" fi elif [ ! "$set_git_hook" = "n" ]; then echo "Expected 'y' or 'n', got '$set_git_hook', will not setup git hook" From 4ea9f96b1bde2e62d834b2e8a436433414fbc53a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Mar 2023 12:46:49 +0000 Subject: [PATCH 089/233] Lower number of processes in KeeperMap test --- tests/integration/test_keeper_map/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 3809f046d55..4b940fbf1d1 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -123,7 +123,7 @@ def test_create_drop_keeper_map_concurrent(started_cluster): manager = multiprocessing.Manager() stop_event = manager.Event() results = [] - for i in range(multiprocessing.cpu_count()): + for i in range(8): sleep(0.2) results.append( pool.apply_async( From 80f0251ff209915fe5fb8996d6f5803cf65b841e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Mar 2023 12:54:57 +0000 Subject: [PATCH 090/233] better --- src/IO/S3Common.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 2f99c523429..5c946d59274 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -136,13 +136,13 @@ void AuthSettings::updateFrom(const AuthSettings & from) region = from.region; server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64; - if (from.use_environment_credentials) + if (from.use_environment_credentials.has_value()) use_environment_credentials = from.use_environment_credentials; - if (from.use_insecure_imds_request) + if (from.use_insecure_imds_request.has_value()) use_insecure_imds_request = from.use_insecure_imds_request; - if (from.expiration_window_seconds) + if (from.expiration_window_seconds.has_value()) expiration_window_seconds = from.expiration_window_seconds; if (from.no_sign_request.has_value()) From 482a9690fcf1d95137c8b8cf94e687c277b6dfd7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 17:00:34 +0200 Subject: [PATCH 091/233] fix --- docs/get-clickhouse-docs.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/get-clickhouse-docs.sh b/docs/get-clickhouse-docs.sh index e3b70dee665..9f0bd52362a 100755 --- a/docs/get-clickhouse-docs.sh +++ b/docs/get-clickhouse-docs.sh @@ -39,7 +39,6 @@ else if [ "$set_git_hook" = "y" ]; then hook_command="$(pwd)/pull-clickhouse-docs-hook.sh $UPDATE_PERIOD_HOURS" hook_file=$(realpath "$(pwd)/../.git/hooks/post-checkout") - already_have=$(grep -Faq "pull-clickhouse-docs-hook.sh" "$hook_file" 2>/dev/null && echo 1 || echo 0) if grep -Faq "pull-clickhouse-docs-hook.sh" "$hook_file" 2>/dev/null; then echo "Looks like the update hook already exists, will not add another one" else From 8b965ba895df4edecfeec74a7a5056bc0c464ac0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 18:15:46 +0300 Subject: [PATCH 092/233] Update get-clickhouse-docs.sh --- docs/get-clickhouse-docs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/get-clickhouse-docs.sh b/docs/get-clickhouse-docs.sh index 9f0bd52362a..34db4e74cee 100755 --- a/docs/get-clickhouse-docs.sh +++ b/docs/get-clickhouse-docs.sh @@ -37,7 +37,7 @@ else fi if [ "$set_git_hook" = "y" ]; then - hook_command="$(pwd)/pull-clickhouse-docs-hook.sh $UPDATE_PERIOD_HOURS" + hook_command="$(pwd)/pull-clickhouse-docs-hook.sh $UPDATE_PERIOD_HOURS ||:" hook_file=$(realpath "$(pwd)/../.git/hooks/post-checkout") if grep -Faq "pull-clickhouse-docs-hook.sh" "$hook_file" 2>/dev/null; then echo "Looks like the update hook already exists, will not add another one" From 3174c07a8667b0e33eefef42db5d60d4da5f2135 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 29 Mar 2023 17:43:26 +0200 Subject: [PATCH 093/233] try fix test --- tests/queries/0_stateless/02151_hash_table_sizes_stats.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats.sh b/tests/queries/0_stateless/02151_hash_table_sizes_stats.sh index ccd6f89e752..4a1eea0a238 100755 --- a/tests/queries/0_stateless/02151_hash_table_sizes_stats.sh +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-tsan # shellcheck disable=SC2154 From 4240bb8527142bf630f3435552241c30dada7c85 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 29 Mar 2023 19:41:09 +0000 Subject: [PATCH 094/233] Fix test --- tests/queries/0_stateless/02668_ulid_decoding.reference | 2 +- tests/queries/0_stateless/02668_ulid_decoding.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02668_ulid_decoding.reference b/tests/queries/0_stateless/02668_ulid_decoding.reference index 57af48199b4..b48580d60bb 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.reference +++ b/tests/queries/0_stateless/02668_ulid_decoding.reference @@ -1,3 +1,3 @@ 1 -2023-03-28 01:16:44.000 +2023-03-27 19:16:44.000 2023-03-27 19:16:44.000 diff --git a/tests/queries/0_stateless/02668_ulid_decoding.sql b/tests/queries/0_stateless/02668_ulid_decoding.sql index 62c4a7d4dbe..296d2b7ce32 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.sql +++ b/tests/queries/0_stateless/02668_ulid_decoding.sql @@ -1,7 +1,7 @@ -- Tags: no-fasttest SELECT dateDiff('s', ULIDStringToDateTime(generateULID()), now()) = 0; -SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E'); +SELECT toTimezone(ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E'), 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9', 'America/Costa_Rica'); -- { serverError ILLEGAL_COLUMN } SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Ric'); -- { serverError POCO_EXCEPTION } From 42c2ccb7cc39e8c2812b1dc03368c55180c47266 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 30 Mar 2023 00:11:13 +0200 Subject: [PATCH 095/233] Support BACKUP ALL command. --- src/Parsers/ParserBackupQuery.cpp | 13 ++--- .../test_backup_restore_new/test.py | 47 +++++++++++++++++++ 2 files changed, 52 insertions(+), 8 deletions(-) diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index cead1a98c1a..cbe4567ee90 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -103,7 +103,7 @@ namespace }); } - bool parseElement(IParser::Pos & pos, Expected & expected, bool allow_all, Element & element) + bool parseElement(IParser::Pos & pos, Expected & expected, Element & element) { return IParserBase::wrapParseImpl(pos, [&] { @@ -169,7 +169,7 @@ namespace return true; } - if (allow_all && ParserKeyword{"ALL"}.ignore(pos, expected)) + if (ParserKeyword{"ALL"}.ignore(pos, expected)) { element.type = ElementType::ALL; parseExceptDatabases(pos, expected, element.except_databases); @@ -181,7 +181,7 @@ namespace }); } - bool parseElements(IParser::Pos & pos, Expected & expected, bool allow_all, std::vector & elements) + bool parseElements(IParser::Pos & pos, Expected & expected, std::vector & elements) { return IParserBase::wrapParseImpl(pos, [&] { @@ -190,7 +190,7 @@ namespace auto parse_element = [&] { Element element; - if (parseElement(pos, expected, allow_all, element)) + if (parseElement(pos, expected, element)) { result.emplace_back(std::move(element)); return true; @@ -334,11 +334,8 @@ bool ParserBackupQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) else return false; - /// Disable "ALL" if this is a RESTORE command. - bool allow_all = (kind == Kind::RESTORE); - std::vector elements; - if (!parseElements(pos, expected, allow_all, elements)) + if (!parseElements(pos, expected, elements)) return false; String cluster; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 3f67fe8e5f7..7c3374ffe7d 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1184,6 +1184,53 @@ def test_restore_partition(): ) +def test_backup_all(): + create_and_fill_table() + + session_id = new_session_id() + instance.http_query( + "CREATE TEMPORARY TABLE temp_tbl(s String)", params={"session_id": session_id} + ) + instance.http_query( + "INSERT INTO temp_tbl VALUES ('q'), ('w'), ('e')", + params={"session_id": session_id}, + ) + + instance.query("CREATE FUNCTION two_and_half AS (x) -> x * 2.5") + + instance.query("CREATE USER u1 IDENTIFIED BY 'qwe123' SETTINGS custom_a = 1") + + backup_name = new_backup_name() + instance.http_query( + f"BACKUP ALL TO {backup_name}", + params={"session_id": session_id}, + ) + + instance.query("DROP TABLE test.table") + instance.query("DROP FUNCTION two_and_half") + instance.query("DROP USER u1") + + session_id = new_session_id() + instance.http_query( + f"RESTORE ALL FROM {backup_name}", + params={"session_id": session_id}, + method="POST", + ) + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + assert instance.http_query( + "SELECT * FROM temp_tbl ORDER BY s", params={"session_id": session_id} + ) == TSV([["e"], ["q"], ["w"]]) + + assert instance.query("SELECT two_and_half(6)") == "15\n" + + assert ( + instance.query("SHOW CREATE USER u1") + == "CREATE USER u1 IDENTIFIED WITH sha256_password SETTINGS custom_a = 1\n" + ) + + def test_operation_id(): create_and_fill_table(n=30) From ed29c141fb2f6f9d6cbad1695468018be747dc64 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Mar 2023 22:09:17 +0000 Subject: [PATCH 096/233] fix race in StorageS3 --- src/Storages/IStorageDataLake.h | 2 +- src/Storages/StorageS3.cpp | 122 +++++++----------- src/Storages/StorageS3.h | 28 ++-- .../02703_storage_s3_race.reference | 1 + .../0_stateless/02703_storage_s3_race.sh | 15 +++ 5 files changed, 74 insertions(+), 94 deletions(-) create mode 100644 tests/queries/0_stateless/02703_storage_s3_race.reference create mode 100755 tests/queries/0_stateless/02703_storage_s3_race.sh diff --git a/src/Storages/IStorageDataLake.h b/src/Storages/IStorageDataLake.h index 9e322377fbd..37776294491 100644 --- a/src/Storages/IStorageDataLake.h +++ b/src/Storages/IStorageDataLake.h @@ -59,7 +59,7 @@ public: auto new_configuration = getAdjustedConfiguration(ctx, configuration, &Poco::Logger::get("Storage" + String(name))); - return Storage::getTableStructureFromData(new_configuration, format_settings, ctx, /*object_infos*/ nullptr); + return Storage::getTableStructureFromData(new_configuration, format_settings, ctx); } static Configuration diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e17860af288..0753095874c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -139,15 +139,13 @@ public: ASTPtr & query_, const Block & virtual_header_, ContextPtr context_, - ObjectInfos * object_infos_, - Strings * read_keys_, + KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) : WithContext(context_) , client(S3::Client::create(client_)) , globbed_uri(globbed_uri_) , query(query_) , virtual_header(virtual_header_) - , object_infos(object_infos_) , read_keys(read_keys_) , request_settings(request_settings_) , list_objects_pool(1) @@ -278,9 +276,6 @@ private: .last_modification_time = row.GetLastModified().Millis() / 1000, }; - if (object_infos) - (*object_infos)[fs::path(globbed_uri.bucket) / key] = info; - temp_buffer.emplace_back(std::move(key), std::move(info)); } } @@ -324,11 +319,7 @@ private: buffer_iter = buffer.begin(); if (read_keys) - { - read_keys->reserve(read_keys->size() + buffer.size()); - for (const auto & [key, _] : buffer) - read_keys->push_back(key); - } + read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); } void createFilterAST(const String & any_key) @@ -374,8 +365,7 @@ private: std::unique_ptr matcher; bool recursive{false}; bool is_finished{false}; - ObjectInfos * object_infos; - Strings * read_keys; + KeysWithInfo * read_keys; S3::ListObjectsV2Request request; S3Settings::RequestSettings request_settings; @@ -392,10 +382,9 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, - ObjectInfos * object_infos_, - Strings * read_keys_, + KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_, request_settings_)) + : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { } @@ -421,8 +410,7 @@ public: ASTPtr query_, const Block & virtual_header_, ContextPtr context_, - ObjectInfos * object_infos_, - Strings * read_keys_) + KeysWithInfo * read_keys_) : WithContext(context_) , bucket(bucket_) , query(query_) @@ -460,26 +448,15 @@ public: } } - if (read_keys_) - *read_keys_ = all_keys; - for (auto && key : all_keys) { - std::optional info; - - /// To avoid extra requests update total_size only if object_infos != nullptr - /// (which means we eventually need this info anyway, so it should be ok to do it now) - if (object_infos_) - { - info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); - total_size += info->size; - - String path = fs::path(bucket) / key; - (*object_infos_)[std::move(path)] = *info; - } - + auto info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); + total_size += info.size; keys.emplace_back(std::move(key), std::move(info)); } + + if (read_keys_) + *read_keys_ = keys; } KeyWithInfo next() @@ -516,11 +493,10 @@ StorageS3Source::KeysIterator::KeysIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, - ObjectInfos * object_infos, - Strings * read_keys) + KeysWithInfo * read_keys) : pimpl(std::make_shared( client_, version_id_, keys_, bucket_, request_settings_, - query, virtual_header, context, object_infos, read_keys)) + query, virtual_header, context, read_keys)) { } @@ -963,8 +939,7 @@ StorageS3::StorageS3( compression_method, is_key_with_globs, format_settings, - context_, - &object_infos); + context_); storage_metadata.setColumns(columns); } @@ -993,8 +968,7 @@ std::shared_ptr StorageS3::createFileIterator( ContextPtr local_context, ASTPtr query, const Block & virtual_block, - ObjectInfos * object_infos, - Strings * read_keys) + KeysWithInfo * read_keys) { if (distributed_processing) { @@ -1005,14 +979,14 @@ std::shared_ptr StorageS3::createFileIterator( /// Iterate through disclosed globs and make a source for each file return std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_block, - local_context, object_infos, read_keys, s3_configuration.request_settings); + local_context, read_keys, s3_configuration.request_settings); } else { return std::make_shared( *s3_configuration.client, s3_configuration.url.version_id, keys, - s3_configuration.url.bucket, s3_configuration.request_settings, query, virtual_block, local_context, - object_infos, read_keys); + s3_configuration.url.bucket, s3_configuration.request_settings, query, + virtual_block, local_context, read_keys); } } @@ -1060,8 +1034,7 @@ Pipe StorageS3::read( distributed_processing, local_context, query_info.query, - virtual_block, - &object_infos); + virtual_block); ColumnsDescription columns_description; Block block_for_format; @@ -1377,13 +1350,12 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context ColumnsDescription StorageS3::getTableStructureFromData( StorageS3::Configuration & configuration, const std::optional & format_settings, - ContextPtr ctx, - ObjectInfos * object_infos) + ContextPtr ctx) { updateConfiguration(ctx, configuration); return getTableStructureFromDataImpl( configuration.format, configuration, configuration.compression_method, - configuration.url.key.find_first_of("*?{") != std::string::npos, format_settings, ctx, object_infos); + configuration.url.key.find_first_of("*?{") != std::string::npos, format_settings, ctx); } ColumnsDescription StorageS3::getTableStructureFromDataImpl( @@ -1392,10 +1364,9 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( const String & compression_method, bool is_key_with_globs, const std::optional & format_settings, - ContextPtr ctx, - ObjectInfos * object_infos) + ContextPtr ctx) { - std::vector read_keys; + KeysWithInfo read_keys; auto file_iterator = createFileIterator( s3_configuration, @@ -1403,12 +1374,12 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( is_key_with_globs, false, ctx, nullptr, - {}, object_infos, &read_keys); + {}, &read_keys); std::optional columns_from_cache; size_t prev_read_keys_size = read_keys.size(); if (ctx->getSettingsRef().schema_inference_use_cache_for_s3) - columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), s3_configuration, object_infos, format, format_settings, ctx); + columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), s3_configuration, format, format_settings, ctx); ReadBufferIterator read_buffer_iterator = [&, first = true](ColumnsDescription & cached_columns) mutable -> std::unique_ptr { @@ -1428,7 +1399,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (ctx->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) { - columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), s3_configuration, object_infos, format, format_settings, ctx); + columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), s3_configuration, format, format_settings, ctx); prev_read_keys_size = read_keys.size(); if (columns_from_cache) { @@ -1549,10 +1520,9 @@ SchemaCache & StorageS3::getSchemaCache(const ContextPtr & ctx) } std::optional StorageS3::tryGetColumnsFromCache( - const Strings::const_iterator & begin, - const Strings::const_iterator & end, + const KeysWithInfo::const_iterator & begin, + const KeysWithInfo::const_iterator & end, const Configuration & s3_configuration, - ObjectInfos * object_infos, const String & format_name, const std::optional & format_settings, const ContextPtr & ctx) @@ -1560,31 +1530,33 @@ std::optional StorageS3::tryGetColumnsFromCache( auto & schema_cache = getSchemaCache(ctx); for (auto it = begin; it < end; ++it) { - String path = fs::path(s3_configuration.url.bucket) / *it; - auto get_last_mod_time = [&]() -> std::optional + auto get_last_mod_time = [&] { - S3::ObjectInfo info; - /// Check if we already have information about this object. - /// If no, request it and remember for possible future usage. - if (object_infos && object_infos->contains(path)) - info = (*object_infos)[path]; + time_t last_modification_time = 0; + if (it->info) + { + last_modification_time = it->info->last_modification_time; + } else { /// Note that in case of exception in getObjectInfo returned info will be empty, /// but schema cache will handle this case and won't return columns from cache /// because we can't say that it's valid without last modification time. - info = S3::getObjectInfo(*s3_configuration.client, s3_configuration.url.bucket, *it, s3_configuration.url.version_id, s3_configuration.request_settings, - {}, {}, /* throw_on_error= */ false); - if (object_infos) - (*object_infos)[path] = info; + last_modification_time = S3::getObjectInfo( + *s3_configuration.client, + s3_configuration.url.bucket, + it->key, + s3_configuration.url.version_id, + s3_configuration.request_settings, + /*with_metadata=*/ false, + /*for_disk_s3=*/ false, + /*throw_on_error= */ false).last_modification_time; } - if (info.last_modification_time) - return info.last_modification_time; - - return std::nullopt; + return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; }; + String path = fs::path(s3_configuration.url.bucket) / it->key; String source = fs::path(s3_configuration.url.uri.getHost() + std::to_string(s3_configuration.url.uri.getPort())) / path; auto cache_key = getKeyForSchemaCache(source, format_name, format_settings, ctx); auto columns = schema_cache.tryGet(cache_key, get_last_mod_time); @@ -1596,7 +1568,7 @@ std::optional StorageS3::tryGetColumnsFromCache( } void StorageS3::addColumnsToCache( - const Strings & keys, + const KeysWithInfo & keys, const Configuration & s3_configuration, const ColumnsDescription & columns, const String & format_name, @@ -1606,7 +1578,7 @@ void StorageS3::addColumnsToCache( auto host_and_bucket = fs::path(s3_configuration.url.uri.getHost() + std::to_string(s3_configuration.url.uri.getPort())) / s3_configuration.url.bucket; Strings sources; sources.reserve(keys.size()); - std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const String & key){ return host_and_bucket / key; }); + std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; }); auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx); auto & schema_cache = getSchemaCache(ctx); schema_cache.addMany(cache_keys, columns); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index b4f95d8d10d..a34113b8bae 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -51,7 +51,7 @@ public: }; using KeysWithInfo = std::vector; - using ObjectInfos = std::unordered_map; + class IIterator { public: @@ -71,8 +71,7 @@ public: ASTPtr query, const Block & virtual_header, ContextPtr context, - ObjectInfos * object_infos = nullptr, - Strings * read_keys_ = nullptr, + KeysWithInfo * read_keys_ = nullptr, const S3Settings::RequestSettings & request_settings_ = {}); KeyWithInfo next() override; @@ -96,8 +95,7 @@ public: ASTPtr query, const Block & virtual_header, ContextPtr context, - ObjectInfos * object_infos = nullptr, - Strings * read_keys = nullptr); + KeysWithInfo * read_keys = nullptr); KeyWithInfo next() override; size_t getTotalSize() const override; @@ -288,8 +286,6 @@ public: bool supportsPartitionBy() const override; - using ObjectInfos = StorageS3Source::ObjectInfos; - static void processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection); static SchemaCache & getSchemaCache(const ContextPtr & ctx); @@ -299,8 +295,7 @@ public: static ColumnsDescription getTableStructureFromData( StorageS3::Configuration & configuration, const std::optional & format_settings, - ContextPtr ctx, - ObjectInfos * object_infos = nullptr); + ContextPtr ctx); protected: static StorageS3::Configuration updateConfiguration(ContextPtr local_context, const Configuration & configuration); @@ -323,7 +318,7 @@ private: ASTPtr partition_by; bool is_key_with_globs = false; - ObjectInfos object_infos; + using KeysWithInfo = StorageS3Source::KeysWithInfo; static std::shared_ptr createFileIterator( const Configuration & s3_configuration, @@ -333,8 +328,7 @@ private: ContextPtr local_context, ASTPtr query, const Block & virtual_block, - ObjectInfos * object_infos = nullptr, - Strings * read_keys = nullptr); + KeysWithInfo * read_keys = nullptr); static ColumnsDescription getTableStructureFromDataImpl( const String & format, @@ -342,24 +336,22 @@ private: const String & compression_method, bool is_key_with_globs, const std::optional & format_settings, - ContextPtr ctx, - ObjectInfos * object_infos = nullptr); + ContextPtr ctx); bool supportsSubcolumns() const override; bool supportsSubsetOfColumns() const override; static std::optional tryGetColumnsFromCache( - const Strings::const_iterator & begin, - const Strings::const_iterator & end, + const KeysWithInfo::const_iterator & begin, + const KeysWithInfo::const_iterator & end, const Configuration & s3_configuration, - ObjectInfos * object_infos, const String & format_name, const std::optional & format_settings, const ContextPtr & ctx); static void addColumnsToCache( - const Strings & keys, + const KeysWithInfo & keys, const Configuration & s3_configuration, const ColumnsDescription & columns, const String & format_name, diff --git a/tests/queries/0_stateless/02703_storage_s3_race.reference b/tests/queries/0_stateless/02703_storage_s3_race.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02703_storage_s3_race.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02703_storage_s3_race.sh b/tests/queries/0_stateless/02703_storage_s3_race.sh new file mode 100755 index 00000000000..f4ef8c2426f --- /dev/null +++ b/tests/queries/0_stateless/02703_storage_s3_race.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +filename="test_${CLICKHOUSE_DATABASE}_${RANDOM}" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_s3_race" +$CLICKHOUSE_CLIENT --query "CREATE TABLE test_s3_race (u UInt64) ENGINE = S3(s3_conn, filename='$filename', format='CSV')" +$CLICKHOUSE_CLIENT --s3_truncate_on_insert 1 --query "INSERT INTO test_s3_race VALUES (1)" + +$CLICKHOUSE_BENCHMARK -i 100 -c 4 <<< "SELECT * FROM test_s3_race" >/dev/null 2>&1 +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_s3_race" +echo "OK" From 8aebdd79f615bda1b4433ea2d745b519a7d68378 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 30 Mar 2023 01:37:01 +0200 Subject: [PATCH 097/233] Update 02668_ulid_decoding.sql --- tests/queries/0_stateless/02668_ulid_decoding.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02668_ulid_decoding.sql b/tests/queries/0_stateless/02668_ulid_decoding.sql index 296d2b7ce32..df94025b7b5 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.sql +++ b/tests/queries/0_stateless/02668_ulid_decoding.sql @@ -1,6 +1,6 @@ -- Tags: no-fasttest -SELECT dateDiff('s', ULIDStringToDateTime(generateULID()), now()) = 0; +SELECT dateDiff('minute', ULIDStringToDateTime(generateULID()), now()) = 0; SELECT toTimezone(ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E'), 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9', 'America/Costa_Rica'); -- { serverError ILLEGAL_COLUMN } From e43780326ec2319ea343fd68bbba1a2bbb1925e2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 30 Mar 2023 11:09:07 +0800 Subject: [PATCH 098/233] wip support map args for map_from_arrays --- src/Functions/map.cpp | 56 +++++++++++++++++++++++++------------------ 1 file changed, 33 insertions(+), 23 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 549de200bea..e1dc58eb077 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -173,23 +173,31 @@ public: getName(), arguments.size()); - const auto * keys_type = checkAndGetDataType(arguments[0].get()); - if (!keys_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName()); + /// The first argument should always be Array. + /// Because key type can not be nested type of Map, which is Tuple + DataTypePtr key_type; + if (const auto * keys_type = checkAndGetDataType(arguments[0].get())) + key_type = keys_type->getNestedType(); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be Array or Map", getName()); - const auto * values_type = checkAndGetDataType(arguments[1].get()); - if (!values_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an Array", getName()); + DataTypePtr value_type; + if (const auto * value_array_type = checkAndGetDataType(arguments[1].get())) + value_type = value_array_type->getNestedType(); + else if (const auto * value_map_type = checkAndGetDataType(arguments[1].get())) + value_type = value_map_type->getValueType(); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName()); - DataTypes key_value_types{keys_type->getNestedType(), values_type->getNestedType()}; + DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - ColumnPtr holder_keys; bool is_keys_const = isColumnConst(*arguments[0].column); + ColumnPtr holder_keys; const ColumnArray * col_keys; if (is_keys_const) { @@ -201,24 +209,26 @@ public: col_keys = checkAndGetColumn(arguments[0].column.get()); } - ColumnPtr holder_values; - bool is_values_const = isColumnConst(*arguments[1].column); - const ColumnArray * col_values; - if (is_values_const) - { - holder_values = arguments[1].column->convertToFullColumnIfConst(); - col_values = checkAndGetColumn(holder_values.get()); - } - else - { - col_values = checkAndGetColumn(arguments[1].column.get()); - } + if (!col_keys) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The first argument of function {} must be Array", getName()); - if (!col_keys || !col_values) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Arguments of function {} must be array", getName()); + bool is_values_const = isColumnConst(*arguments[1].column); + ColumnPtr holder_values; + if (is_values_const) + holder_values = arguments[1].column->convertToFullColumnIfConst(); + else + holder_values = arguments[1].column; + + const ColumnArray * col_values; + if (const auto * col_values_array = checkAndGetColumn(holder_values.get())) + col_values = col_values_array; + else if (const auto * col_values_map = checkAndGetColumn(holder_values.get())) + col_values = &col_values_map->getNestedColumn(); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The second arguments of function {} must be Array or Map", getName()); if (!col_keys->hasEqualOffsets(*col_values)) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName()); const auto & data_keys = col_keys->getDataPtr(); const auto & data_values = col_values->getDataPtr(); From 5ca488d70e17e46fc1904abb52a2ad002e2b9bec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Mar 2023 09:14:52 +0300 Subject: [PATCH 099/233] Update src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp Co-authored-by: Antonio Andelic --- src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 7ce58b9991d..eeca14176cc 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -412,8 +412,10 @@ bool MsgPackVisitor::end_array_item() // NOLINT info_stack.pop(); else { - --(*info_stack.top().array_size); - if (*info_stack.top().array_size == 0) + assert(info_stack.top().array_size.has_value()); + auto & current_array_size = *info_stack.top().array_size; + --current_array_size; + if (current_array_size == 0) info_stack.pop(); } return true; From 9db58532f451f91bac7da76709709f0b4f42c623 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 30 Mar 2023 08:41:14 +0200 Subject: [PATCH 100/233] Clang-tidy fix --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b735e090f59..6cdb950b202 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1291,7 +1291,7 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 0); - configuration.auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", 0); + configuration.auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", false); configuration.auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); configuration.format = collection.getOrDefault("format", "auto"); From 8c8c7464514834135a37131567b66257fae84046 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 30 Mar 2023 09:04:36 +0200 Subject: [PATCH 101/233] Update FunctionsCodingULID.cpp --- src/Functions/FunctionsCodingULID.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCodingULID.cpp b/src/Functions/FunctionsCodingULID.cpp index bc62b2d9aca..3201578791a 100644 --- a/src/Functions/FunctionsCodingULID.cpp +++ b/src/Functions/FunctionsCodingULID.cpp @@ -49,7 +49,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 1 || arguments.size() > 2) + if (arguments.empty() || arguments.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: should be 1 or 2", From 202dc90045497212f3e8b95381144e468b041cee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Mar 2023 10:01:52 +0200 Subject: [PATCH 102/233] Randomize JIT settings in tests --- tests/clickhouse-test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a355c2f8e73..4d16fead44f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -558,6 +558,9 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), + "min_count_to_compile_expression": lambda: random.randint(0, 3), + "min_count_to_compile_aggregate_expression": lambda: random.randint(0, 3), + "min_count_to_compile_sort_description": lambda: random.randint(0, 3), } @staticmethod From 990ef56443b270dc11b50a0ab20e5661b9d3aa31 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Mar 2023 10:55:29 +0200 Subject: [PATCH 103/233] Randomize JIT settings in tests --- tests/clickhouse-test | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4d16fead44f..fa88bc19efd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -558,9 +558,9 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), - "min_count_to_compile_expression": lambda: random.randint(0, 3), - "min_count_to_compile_aggregate_expression": lambda: random.randint(0, 3), - "min_count_to_compile_sort_description": lambda: random.randint(0, 3), + "min_count_to_compile_expression": lambda: random.choice([0, 3]), + "min_count_to_compile_aggregate_expression": lambda: random.choice([0, 3]), + "min_count_to_compile_sort_description": lambda: random.choice([0, 3]), } @staticmethod From baabc49f339373f77395d0c8078208cf5f2616ea Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 30 Mar 2023 11:01:51 +0200 Subject: [PATCH 104/233] Update ParserKQLSort.cpp Style --- src/Parsers/Kusto/ParserKQLSort.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/Kusto/ParserKQLSort.cpp b/src/Parsers/Kusto/ParserKQLSort.cpp index f7540d729fd..ef4b84b17c7 100644 --- a/src/Parsers/Kusto/ParserKQLSort.cpp +++ b/src/Parsers/Kusto/ParserKQLSort.cpp @@ -27,7 +27,7 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) while (!new_pos->isEnd() && new_pos->type != TokenType::PipeMark && new_pos->type != TokenType::Semicolon) { String tmp(new_pos->begin, new_pos->end); - if (tmp == "desc" or tmp == "asc") + if (tmp == "desc" || tmp == "asc") has_dir = true; if (new_pos->type == TokenType::Comma) From e3f4089f3b6923d0281c0fe66cf186d759b07a02 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 30 Mar 2023 18:00:38 +0800 Subject: [PATCH 105/233] fix bugs --- src/Functions/map.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index e1dc58eb077..14453de0646 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -185,7 +185,7 @@ public: if (const auto * value_array_type = checkAndGetDataType(arguments[1].get())) value_type = value_array_type->getNestedType(); else if (const auto * value_map_type = checkAndGetDataType(arguments[1].get())) - value_type = value_map_type->getValueType(); + value_type = std::make_shared(value_map_type->getKeyValueTypes()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName()); From 2a35c189732fb36261dcf63b3885ab364600de79 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 30 Mar 2023 18:18:21 +0800 Subject: [PATCH 106/233] add docs and uts --- .../functions/tuple-map-functions.md | 20 +++++++++++-------- .../0_stateless/01651_map_functions.reference | 3 +++ .../0_stateless/01651_map_functions.sql | 5 +++++ 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 34c865e7752..cdecbbcc2e9 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -68,9 +68,9 @@ Result: ## mapFromArrays -Merges an [Array](../../sql-reference/data-types/array.md) of keys and an [Array](../../sql-reference/data-types/array.md) of values into a [Map(key, value)](../../sql-reference/data-types/map.md). +Merges an [Array](../../sql-reference/data-types/array.md) of keys and an [Array](../../sql-reference/data-types/array.md) of values into a [Map(key, value)](../../sql-reference/data-types/map.md). Notice that the second argument could also be a [Map](../../sql-reference/data-types/map.md), thus it is casted to an Array when executing. -The function is a more convenient alternative to `CAST((key_array, value_array), 'Map(key_type, value_type)')`. For example, instead of writing `CAST((['aa', 'bb'], [4, 5]), 'Map(String, UInt32)')`, you can write `mapFromArrays(['aa', 'bb'], [4, 5])`. +The function is a more convenient alternative to `CAST((key_array, value_array_or_map), 'Map(key_type, value_type)')`. For example, instead of writing `CAST((['aa', 'bb'], [4, 5]), 'Map(String, UInt32)')`, you can write `mapFromArrays(['aa', 'bb'], [4, 5])`. **Syntax** @@ -82,11 +82,11 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** - `keys` — Given key array to create a map from. The nested type of array must be: [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md), [LowCardinality](../../sql-reference/data-types/lowcardinality.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [UUID](../../sql-reference/data-types/uuid.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), [Date32](../../sql-reference/data-types/date32.md), [Enum](../../sql-reference/data-types/enum.md) -- `values` - Given value array to create a map from. +- `values` - Given value array or map to create a map from. **Returned value** -- A map whose keys and values are constructed from the key and value arrays +- A map whose keys and values are constructed from the key array and value array/map. **Example** @@ -94,13 +94,17 @@ Query: ```sql select mapFromArrays(['a', 'b', 'c'], [1, 2, 3]) -``` - -```text + ┌─mapFromArrays(['a', 'b', 'c'], [1, 2, 3])─┐ │ {'a':1,'b':2,'c':3} │ └───────────────────────────────────────────┘ -``` + +SELECT mapFromArrays([1, 2, 3], map('a', 1, 'b', 2, 'c', 3)) + +┌─mapFromArrays([1, 2, 3], map('a', 1, 'b', 2, 'c', 3))─┐ +│ {1:('a',1),2:('b',2),3:('c',3)} │ +└───────────────────────────────────────────────────────┘ +``` ## mapAdd diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index f7fd3503327..60f1b6e0d0c 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -33,3 +33,6 @@ {'aa':4,'bb':5} {'aa':4,'bb':5} {'aa':4,'bb':5} +{'aa':('a',4),'bb':('b',5)} +{'aa':('a',4),'bb':('b',5)} +{'aa':('a',4),'bb':('b',5)} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 848b6932fe0..5942bf8b2c2 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -39,3 +39,8 @@ select mapFromArrays(['aa', 'bb'], 5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError BAD_ARGUMENTS } + +select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); +select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); +select mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) from numbers(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } From b22d3e913620a7bd8dc805d520a94f7824910610 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 30 Mar 2023 09:55:01 +0000 Subject: [PATCH 107/233] Remove wrong assert --- .../Passes/LogicalExpressionOptimizerPass.cpp | 1 - ...702_logical_optimizer_with_nulls.reference | 19 +++++++++++++++++++ .../02702_logical_optimizer_with_nulls.sql | 15 +++++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 97669f3924f..13f8025f5ea 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -219,7 +219,6 @@ private: /// we can replace OR with the operand if (or_operands[0]->getResultType()->equals(*function_node.getResultType())) { - assert(!function_node.getResultType()->isNullable()); node = std::move(or_operands[0]); return; } diff --git a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference index 263329e47be..eb79bbc842a 100644 --- a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference +++ b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference @@ -56,3 +56,22 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 9, column_name: a, result_type: Int32, source_id: 3 CONSTANT id: 13, constant_value: Tuple_(UInt64_1, UInt64_3, UInt64_2), constant_value_type: Tuple(UInt8, UInt8, UInt8) +1 test +2 test2 +3 another +QUERY id: 0 + PROJECTION COLUMNS + a Nullable(Int32) + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Nullable(Int32), source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.02702_logical_optimizer_with_null_column + WHERE + FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: Nullable(UInt8) + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Nullable(Int32), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_3, UInt64_2), constant_value_type: Tuple(UInt8, UInt8, UInt8) diff --git a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql index 9a49e31fe81..07d0b170a02 100644 --- a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql +++ b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql @@ -15,3 +15,18 @@ EXPLAIN QUERY TREE SELECT * FROM 02702_logical_optimizer WHERE a = 1 OR 3 = a OR SELECT * FROM 02702_logical_optimizer WHERE a = 1 OR 3 = a OR 2 = a OR a = NULL; EXPLAIN QUERY TREE SELECT * FROM 02702_logical_optimizer WHERE a = 1 OR 3 = a OR 2 = a OR a = NULL; + +DROP TABLE 02702_logical_optimizer; + +DROP TABLE IF EXISTS 02702_logical_optimizer_with_null_column; + +CREATE TABLE 02702_logical_optimizer_with_null_column +(a Nullable(Int32), b LowCardinality(String)) +ENGINE=Memory; + +INSERT INTO 02702_logical_optimizer_with_null_column VALUES (1, 'test'), (2, 'test2'), (3, 'another'); + +SELECT * FROM 02702_logical_optimizer_with_null_column WHERE a = 1 OR 3 = a OR 2 = a; +EXPLAIN QUERY TREE SELECT * FROM 02702_logical_optimizer_with_null_column WHERE a = 1 OR 3 = a OR 2 = a; + +DROP TABLE 02702_logical_optimizer_with_null_column; From 2df32324af0a3dba04fe688b5158ec6bff59cf2e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 30 Mar 2023 12:03:52 +0000 Subject: [PATCH 108/233] MySQL compatibility: Make str_to_date alias case-insensitive MySQL doesn't care about the case --- src/Functions/parseDateTime.cpp | 2 +- tests/queries/0_stateless/02668_parse_datetime.reference | 2 ++ tests/queries/0_stateless/02668_parse_datetime.sql | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 6a7a6010d4b..553e993a806 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1856,7 +1856,7 @@ REGISTER_FUNCTION(ParseDateTime) factory.registerAlias("TO_UNIXTIME", FunctionParseDateTime::name); factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("str_to_date", FunctionParseDateTimeOrNull::name); + factory.registerAlias("str_to_date", FunctionParseDateTimeOrNull::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index f39655c6a41..afa3d0eb962 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -208,5 +208,7 @@ select parseDateTimeOrNull('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', ' 1 select str_to_date('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 +select sTr_To_DaTe('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +1 select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; 1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 757c4fe2efe..51c2fda8428 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -138,6 +138,7 @@ select parseDateTimeOrZero('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', ' select parseDateTimeOrNull('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); select parseDateTimeOrNull('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; select str_to_date('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select sTr_To_DaTe('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; -- { echoOff } From ad246d669e85755bb6bc88169479bd6e50379afb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 30 Mar 2023 12:08:52 +0000 Subject: [PATCH 109/233] Disable AST optimizations for projection analysis. --- .../Optimizations/optimizeUseAggregateProjection.cpp | 6 +++++- .../25402_projection_and_ast_optimizations_bug.reference | 1 + .../25402_projection_and_ast_optimizations_bug.sql | 6 ++++++ 3 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.reference create mode 100644 tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 77b5547207c..21cb112cb14 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -61,11 +61,15 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( /// This is a bad approach. /// We'd better have a separate interpreter for projections. /// Now it's not obvious we didn't miss anything here. + /// + /// Setting ignoreASTOptimizations is used because some of them are invalid for projections. + /// Example: 'SELECT min(c0), max(c0), count() GROUP BY -c0' for minmax_count projection can be rewritten to + /// 'SELECT min(c0), max(c0), count() GROUP BY c0' which is incorrect cause we store a column '-c0' in projection. InterpreterSelectQuery interpreter( projection.query_ast, context, Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), - SelectQueryOptions{QueryProcessingStage::WithMergeableState}); + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations()); const auto & analysis_result = interpreter.getAnalysisResult(); const auto & query_analyzer = interpreter.getQueryAnalyzer(); diff --git a/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.reference b/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.reference new file mode 100644 index 00000000000..9049324c392 --- /dev/null +++ b/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.reference @@ -0,0 +1 @@ +-2.5574077246549023 0.6663667453928805 1 diff --git a/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.sql b/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.sql new file mode 100644 index 00000000000..5589fbeeb9e --- /dev/null +++ b/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.sql @@ -0,0 +1,6 @@ +drop table if exists t1; +CREATE TABLE t1 (c0 Int32) ENGINE = MergeTree() ORDER BY c0 PARTITION BY (- (c0)); +insert into t1 values(1); +SELECT (- ((((tan (t1.c0)))+(t1.c0)))), (cos ((sin (pow(t1.c0,t1.c0))))), ((gcd((- (t1.c0)),((t1.c0)+(t1.c0))))*((- ((- (t1.c0)))))) FROM t1 GROUP BY (sqrt ((- (t1.c0)))), t1.c0, pow((erf ((- (t1.c0)))),t1.c0); +drop table t1; + From 8b2fc8a40a0d7072965714083908dd25e81fe18d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 30 Mar 2023 12:24:52 +0000 Subject: [PATCH 110/233] parseDateTime[InJodaSyntax](): Require format argument Function parseDateTime[InJodaSyntax]() previously accepted an (undocumented) default format string. The problem was that 1. there is no "natural" default format 2. MySQL str_to_date() (= the motivation for parseDateTime[InJodaSyntax]()) also provides no default format. --> get rid of the default format --- src/Functions/parseDateTime.cpp | 16 ++++------------ .../0_stateless/02668_parse_datetime.reference | 3 +++ .../queries/0_stateless/02668_parse_datetime.sql | 4 ++++ ...02668_parse_datetime_in_joda_syntax.reference | 3 +++ .../02668_parse_datetime_in_joda_syntax.sql | 3 +++ 5 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 6a7a6010d4b..785c423f3ea 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -480,10 +480,10 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) + if (arguments.size() != 2 && arguments.size() != 3) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", getName(), arguments.size()); @@ -494,14 +494,14 @@ namespace arguments[0].type->getName(), getName()); - if (arguments.size() > 1 && !isString(arguments[1].type)) + if (!isString(arguments[1].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}. Should be String", arguments[0].type->getName(), getName()); - if (arguments.size() > 2 && !isString(arguments[2].type)) + if (arguments.size() == 3 && !isString(arguments[2].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}. Should be String", @@ -1776,14 +1776,6 @@ namespace String getFormat(const ColumnsWithTypeAndName & arguments) const { - if (arguments.size() < 2) - { - if constexpr (parse_syntax == ParseSyntax::Joda) - return "yyyy-MM-dd HH:mm:ss"; - else - return "%Y-%m-%d %H:%M:%S"; - } - const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); if (!format_column) throw Exception( diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index f39655c6a41..c60f6e6467d 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -210,3 +210,6 @@ select str_to_date('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTi 1 select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; 1 +-- Error handling +select parseDateTime('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTime('12 AM', '%h %p', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 757c4fe2efe..d6ee6cb617c 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -140,4 +140,8 @@ select parseDateTimeOrNull('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', ' select str_to_date('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; +-- Error handling +select parseDateTime('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTime('12 AM', '%h %p', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + -- { echoOff } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference index 124836d6118..9fbf105dc41 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference @@ -353,3 +353,6 @@ select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = 1 select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; 1 +-- Error handling +select parseDateTimeInJodaSyntax('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql index b2c781432d1..f5810d3d4c3 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -238,4 +238,7 @@ select parseDateTimeInJodaSyntaxOrZero('2001 invalid 366 2000', 'yyyy D yyyy', ' select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; +-- Error handling +select parseDateTimeInJodaSyntax('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- { echoOff } From 014db1fbaf787474a4629abfc84f38b30ad3e93a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 30 Mar 2023 15:45:33 +0300 Subject: [PATCH 111/233] Update docs/get-clickhouse-docs.sh Co-authored-by: Mikhail f. Shiryaev --- docs/get-clickhouse-docs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/get-clickhouse-docs.sh b/docs/get-clickhouse-docs.sh index 34db4e74cee..92ba8058dcc 100755 --- a/docs/get-clickhouse-docs.sh +++ b/docs/get-clickhouse-docs.sh @@ -36,6 +36,8 @@ else read -rp "Would you like to setup git hook for automatic update? (y|n): " set_git_hook fi + git clone "$git_url" "clickhouse-docs" + if [ "$set_git_hook" = "y" ]; then hook_command="$(pwd)/pull-clickhouse-docs-hook.sh $UPDATE_PERIOD_HOURS ||:" hook_file=$(realpath "$(pwd)/../.git/hooks/post-checkout") @@ -49,6 +51,4 @@ else elif [ ! "$set_git_hook" = "n" ]; then echo "Expected 'y' or 'n', got '$set_git_hook', will not setup git hook" fi - - git clone "$git_url" "clickhouse-docs" fi From d6c71533fae14635efdbda83ebe20d62226a9497 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 30 Mar 2023 08:20:20 +0000 Subject: [PATCH 112/233] Move keeper map tests to stateless --- tests/integration/test_keeper_map/test.py | 110 ------------------ ...eeper_map_concurrent_create_drop.reference | 1 + ...02703_keeper_map_concurrent_create_drop.sh | 53 +++++++++ .../02704_keeper_map_zk_nodes.reference | 0 .../0_stateless/02704_keeper_map_zk_nodes.sh | 77 ++++++++++++ 5 files changed, 131 insertions(+), 110 deletions(-) create mode 100644 tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.reference create mode 100755 tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh create mode 100644 tests/queries/0_stateless/02704_keeper_map_zk_nodes.reference create mode 100755 tests/queries/0_stateless/02704_keeper_map_zk_nodes.sh diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 4b940fbf1d1..c6ec7103056 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -1,14 +1,6 @@ -import multiprocessing import pytest -from time import sleep -import random -from itertools import count -from sys import stdout - -from multiprocessing.dummy import Pool from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry, assert_logs_contain from helpers.network import PartitionManager test_recover_staled_replica_run = 1 @@ -46,108 +38,6 @@ def remove_children(client, path): client.delete(child_path) -def test_create_keeper_map(started_cluster): - node.query( - "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" - ) - zk_client = get_genuine_zk() - - def assert_children_size(path, expected_size): - children_size = 0 - # 4 secs should be more than enough for replica to sync - for _ in range(10): - children_size = len(zk_client.get_children(path)) - if children_size == expected_size: - return - sleep(0.4) - assert ( - False - ), f"Invalid number of children for '{path}': actual {children_size}, expected {expected_size}" - - def assert_root_children_size(expected_size): - assert_children_size("/test_keeper_map/test1", expected_size) - - def assert_data_children_size(expected_size): - assert_children_size("/test_keeper_map/test1/data", expected_size) - - assert_root_children_size(2) - assert_data_children_size(0) - - node.query("INSERT INTO test_keeper_map VALUES (1, 11)") - assert_data_children_size(1) - - node.query( - "CREATE TABLE test_keeper_map_another (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" - ) - assert_root_children_size(2) - assert_data_children_size(1) - - node.query("INSERT INTO test_keeper_map_another VALUES (1, 11)") - assert_root_children_size(2) - assert_data_children_size(1) - - node.query("INSERT INTO test_keeper_map_another VALUES (2, 22)") - assert_root_children_size(2) - assert_data_children_size(2) - - node.query("DROP TABLE test_keeper_map SYNC") - assert_root_children_size(2) - assert_data_children_size(2) - - node.query("DROP TABLE test_keeper_map_another SYNC") - assert_root_children_size(0) - - zk_client.stop() - - -def create_drop_loop(index, stop_event): - table_name = f"test_keeper_map_{index}" - - for i in count(0, 1): - if stop_event.is_set(): - return - - node.query_with_retry( - f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value UInt64) ENGINE = KeeperMap('/test') PRIMARY KEY(key);" - ) - node.query_with_retry(f"INSERT INTO {table_name} VALUES ({index}, {i})") - result = node.query_with_retry( - f"SELECT value FROM {table_name} WHERE key = {index}" - ) - assert result.strip() == str(i) - node.query_with_retry(f"DROP TABLE IF EXISTS {table_name} SYNC") - - -def test_create_drop_keeper_map_concurrent(started_cluster): - pool = Pool() - manager = multiprocessing.Manager() - stop_event = manager.Event() - results = [] - for i in range(8): - sleep(0.2) - results.append( - pool.apply_async( - create_drop_loop, - args=( - i, - stop_event, - ), - ) - ) - - sleep(60) - stop_event.set() - - for result in results: - result.get() - - pool.close() - - client = get_genuine_zk() - assert len(client.get_children("/test_keeper_map/test")) == 0 - client.stop() - - def test_keeper_map_without_zk(started_cluster): def assert_keeper_exception_after_partition(query): with PartitionManager() as pm: diff --git a/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.reference b/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh b/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh new file mode 100755 index 00000000000..3964427895c --- /dev/null +++ b/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, zookeeper, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function create_drop_loop() +{ + table_name="02703_keeper_map_concurrent_$1" + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $table_name" + for _ in `seq $1` + do + sleep 0.3 + done + + i=0 + while true; + do + $CLICKHOUSE_CLIENT --query="CREATE TABLE IF NOT EXISTS $table_name (key UInt64, value UInt64) ENGINE = KeeperMap('/02703_keeper_map/$CLICKHOUSE_DATABASE') PRIMARY KEY(key)" + $CLICKHOUSE_CLIENT --query="INSERT INTO $table_name VALUES ($1, $i)" + result=$($CLICKHOUSE_CLIENT --query="SELECT value FROM $table_name WHERE key = $1") + + if [ $result != $i ] + then + echo "Got invalid result $result" + exit 1 + fi + + $CLICKHOUSE_CLIENT --query="DROP TABLE $table_name" + + ((++i)) + done +} + +export -f create_drop_loop; + +THREADS=10 +TIMEOUT=30 + +for i in `seq $THREADS` +do + timeout $TIMEOUT bash -c "create_drop_loop $i" 2> /dev/null & +done + +wait + +for i in `seq $THREADS` +do + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS 02703_keeper_map_concurrent_$i" +done + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM system.zookeeper WHERE path = '/test_keeper_map/02703_keeper_map/$CLICKHOUSE_DATABASE'" diff --git a/tests/queries/0_stateless/02704_keeper_map_zk_nodes.reference b/tests/queries/0_stateless/02704_keeper_map_zk_nodes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02704_keeper_map_zk_nodes.sh b/tests/queries/0_stateless/02704_keeper_map_zk_nodes.sh new file mode 100755 index 00000000000..9689d4f5a50 --- /dev/null +++ b/tests/queries/0_stateless/02704_keeper_map_zk_nodes.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, zookeeper, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +table_name="02704_keeper_map_zk_nodes" +table_name_another="02704_keeper_map_zk_nodes_new_table" + +$CLICKHOUSE_CLIENT --multiquery --query=" +DROP TABLE IF EXISTS $table_name; +DROP TABLE IF EXISTS $table_name_another; +CREATE TABLE $table_name (key UInt64, value UInt64) +ENGINE = KeeperMap('/$table_name/$CLICKHOUSE_DATABASE') +PRIMARY KEY(key)" + +function assert_children_size() +{ + for _ in `seq 10` + do + children_size=$($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.zookeeper WHERE path = '$1'") + if [ $children_size == $2 ] + then + return + fi + + sleep 0.4 + done + + echo "Invalid number of children for path '$1': actual $children_size, expected $2" + exit 1 +} + +function assert_root_children_size() +{ + assert_children_size "/test_keeper_map/02704_keeper_map_zk_nodes/$CLICKHOUSE_DATABASE" $1 +} + +function assert_data_children_size() +{ + assert_children_size "/test_keeper_map/02704_keeper_map_zk_nodes/$CLICKHOUSE_DATABASE/data" $1 +} + +assert_root_children_size 2 +assert_data_children_size 0 + +$CLICKHOUSE_CLIENT --query="INSERT INTO $table_name VALUES (1, 11)" + +assert_data_children_size 1 + +$CLICKHOUSE_CLIENT --query=" +CREATE TABLE $table_name_another (key UInt64, value UInt64) +ENGINE = KeeperMap('/$table_name/$CLICKHOUSE_DATABASE') +PRIMARY KEY(key)" + +assert_root_children_size 2 +assert_data_children_size 1 + +$CLICKHOUSE_CLIENT --query="INSERT INTO $table_name_another VALUES (1, 11)" + +assert_root_children_size 2 +assert_data_children_size 1 + +$CLICKHOUSE_CLIENT --query="INSERT INTO $table_name_another VALUES (2, 22)" + +assert_root_children_size 2 +assert_data_children_size 2 + +$CLICKHOUSE_CLIENT --query="DROP TABLE $table_name" + +assert_root_children_size 2 +assert_data_children_size 2 + +$CLICKHOUSE_CLIENT --query="DROP TABLE $table_name_another" + +assert_root_children_size 0 From ee5e1ece1f8d729cc45256fa7709b13c4adfc2fc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 30 Mar 2023 13:43:11 +0000 Subject: [PATCH 113/233] Stabilize 02477_age --- tests/queries/0_stateless/02477_age.reference | 2 +- tests/queries/0_stateless/02477_age.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02477_age.reference b/tests/queries/0_stateless/02477_age.reference index 249c413d923..4afe3df8ad8 100644 --- a/tests/queries/0_stateless/02477_age.reference +++ b/tests/queries/0_stateless/02477_age.reference @@ -37,7 +37,7 @@ Constant and non-constant arguments -524160 1440 Case insensitive --10 +-1 Dependance of timezones 0 0 diff --git a/tests/queries/0_stateless/02477_age.sql b/tests/queries/0_stateless/02477_age.sql index 9b612276b01..72a692f61d0 100644 --- a/tests/queries/0_stateless/02477_age.sql +++ b/tests/queries/0_stateless/02477_age.sql @@ -45,7 +45,7 @@ SELECT age('minute', materialize(toDate('2017-12-31')), materialize(toDate('2018 SELECT 'Case insensitive'; -SELECT age('year', today(), today() - INTERVAL 10 YEAR); +SELECT age('YeAr', toDate('2017-12-31'), toDate('2016-01-01')); SELECT 'Dependance of timezones'; From 886c3636ec826c67f5a2e6ac2263e352a24e18f3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 Mar 2023 15:56:49 +0200 Subject: [PATCH 114/233] Update 02703_storage_s3_race.sh --- tests/queries/0_stateless/02703_storage_s3_race.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02703_storage_s3_race.sh b/tests/queries/0_stateless/02703_storage_s3_race.sh index f4ef8c2426f..65a38e600f7 100755 --- a/tests/queries/0_stateless/02703_storage_s3_race.sh +++ b/tests/queries/0_stateless/02703_storage_s3_race.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f870e6f7a928ec9a926c10d9398677c86f44e74a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 30 Mar 2023 14:00:44 +0000 Subject: [PATCH 115/233] Rename setting stop_reading_on_first_cancel to partial_result_on_first_cancel. --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 4 ++-- src/Client/ClientBase.cpp | 10 +++++----- src/Client/ClientBase.h | 2 +- src/Core/Settings.h | 2 +- src/Server/TCPHandler.cpp | 6 +++--- .../0_stateless/25341_stop_reading_on_first_cancel.sh | 2 +- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f3c0f20f3a6..1f068285fd7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4050,7 +4050,7 @@ Possible values: Default value: `0`. -## stop_reading_on_first_cancel {#stop_reading_on_first_cancel} +## partial_result_on_first_cancel {#partial_result_on_first_cancel} When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterward, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. **Example without setting on Ctrl+C** @@ -4066,7 +4066,7 @@ Query was cancelled. **Example with setting on Ctrl+C** ```sql -SELECT sum(number) FROM numbers(10000000000) SETTINGS stop_reading_on_first_cancel=true +SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true ┌──────sum(number)─┐ │ 1355411451286266 │ diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index d77f7ba6a46..171e9918163 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4085,7 +4085,7 @@ ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; Значение по умолчанию: `''`. -## stop_reading_on_first_cancel {#stop_reading_on_first_cancel} +## partial_result_on_first_cancel {#partial_result_on_first_cancel} Если установлено значение `true` и пользователь хочет прервать запрос (например, с помощью `Ctrl+C` на клиенте), то запрос продолжает выполнение только для данных, которые уже были считаны из таблицы. После этого он вернет частичный результат запроса для той части таблицы, которая была прочитана. Чтобы полностью остановить выполнение запроса без частичного результата, пользователь должен отправить 2 запроса отмены. **Пример с выключенной настройкой при нажатии Ctrl+C** @@ -4101,7 +4101,7 @@ Query was cancelled. **Пример с включенной настройкой при нажатии Ctrl+C** ```sql -SELECT sum(number) FROM numbers(10000000000) SETTINGS stop_reading_on_first_cancel=true +SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true ┌──────sum(number)─┐ │ 1355411451286266 │ diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d3ba1d7e84c..120d273aa62 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -861,7 +861,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } const auto & settings = global_context->getSettingsRef(); - const Int32 signals_before_stop = settings.stop_reading_on_first_cancel ? 2 : 1; + const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; int retries_left = 10; while (retries_left) @@ -885,7 +885,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa if (send_external_tables) sendExternalTables(parsed_query); - receiveResult(parsed_query, signals_before_stop, settings.stop_reading_on_first_cancel); + receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); break; } @@ -910,7 +910,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa /// Receives and processes packets coming from server. /// Also checks if query execution should be cancelled. -void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool stop_reading_on_first_cancel) +void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel) { // TODO: get the poll_interval from commandline. const auto receive_timeout = connection_parameters.timeouts.receive_timeout; @@ -934,11 +934,11 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b /// to avoid losing sync. if (!cancelled) { - if (stop_reading_on_first_cancel && QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) + if (partial_result_on_first_cancel && QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) { connection->sendCancel(); /// First cancel reading request was sent. Next requests will only be with a full cancel - stop_reading_on_first_cancel = false; + partial_result_on_first_cancel = false; } else if (QueryInterruptHandler::cancelled()) { diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index faf3fa8653a..5926f73f51a 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -131,7 +131,7 @@ protected: private: - void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool stop_reading_on_first_cancel); + void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel); bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_); void receiveLogsAndProfileEvents(ASTPtr parsed_query); bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9fa2ba0d32f..89c18165ea2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -281,7 +281,7 @@ class IColumn; \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ - M(Bool, stop_reading_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \ + M(Bool, partial_result_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \ diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 85cdb75977b..7977605db3b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1818,14 +1818,14 @@ void TCPHandler::decreaseCancellationStatus(const std::string & log_message) { auto prev_status = magic_enum::enum_name(state.cancellation_status); - bool stop_reading_on_first_cancel = false; + bool partial_result_on_first_cancel = false; if (query_context) { const auto & settings = query_context->getSettingsRef(); - stop_reading_on_first_cancel = settings.stop_reading_on_first_cancel; + partial_result_on_first_cancel = settings.partial_result_on_first_cancel; } - if (stop_reading_on_first_cancel && state.cancellation_status == CancellationStatus::NOT_CANCELLED) + if (partial_result_on_first_cancel && state.cancellation_status == CancellationStatus::NOT_CANCELLED) { state.cancellation_status = CancellationStatus::READ_CANCELLED; } diff --git a/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh index 9694907d679..09837bff808 100755 --- a/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh +++ b/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS stop_reading_on_first_cancel=true;" & +$CLICKHOUSE_CLIENT -n --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true;" & pid=$! sleep 2 kill -SIGINT $pid From 7b3754102f4eacc54830027bf0b85062beb2a987 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 30 Mar 2023 14:14:46 +0000 Subject: [PATCH 116/233] Address flaky 02346_full_text_search --- .../02346_full_text_search.reference | 14 -- .../0_stateless/02346_full_text_search.sql | 203 +++++++++--------- 2 files changed, 103 insertions(+), 114 deletions(-) diff --git a/tests/queries/0_stateless/02346_full_text_search.reference b/tests/queries/0_stateless/02346_full_text_search.reference index 9cd09110608..d6e510b9375 100644 --- a/tests/queries/0_stateless/02346_full_text_search.reference +++ b/tests/queries/0_stateless/02346_full_text_search.reference @@ -46,17 +46,3 @@ Test inverted(2) on UTF-8 data af inverted 102 clickhouse你好 1 -Test max_digestion_size_per_segment -af inverted -BC614E,05397FB1,6969696969898240,CF3304 -1 -Test density==1 -af inverted -1 -1 -Test density==0.1 -af inverted -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index ed086861f1f..18d1ce0fd96 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -234,105 +234,108 @@ SELECT read_rows==2 from system.query_log AND result_rows==1 LIMIT 1; ----------------------------------------------------- -SELECT 'Test max_digestion_size_per_segment'; -DROP TABLE IF EXISTS tab; +-- Tests with parameter max_digestion_size_per_segment are flaky in CI, not clear why --> comment out for the time being: -CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0)) - Engine=MergeTree - ORDER BY (k) - SETTINGS max_digestion_size_per_segment = 1024, index_granularity = 256 - AS - SELECT - number, - format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s - FROM numbers(10240); - --- check inverted index was created -SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; - --- search inverted index -SELECT s FROM tab WHERE hasToken(s, '6969696969898240'); - --- check the query only read 1 granule (1 row total; each granule has 256 rows) -SYSTEM FLUSH LOGS; -SELECT read_rows==256 from system.query_log - WHERE query_kind ='Select' - AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT s FROM tab WHERE hasToken(s, \'6969696969898240\');') - AND type='QueryFinish' - AND result_rows==1 - LIMIT 1; - ----------------------------------------------------- -SELECT 'Test density==1'; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 1.0)) - Engine=MergeTree - ORDER BY (k) - SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 - AS - SELECT number, if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number))) - FROM numbers(1024); - --- check inverted index was created -SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; - --- search inverted index, no row has 'happy birthday' -SELECT count() == 0 FROM tab WHERE s =='happy birthday'; - --- check the query only skip all granules (0 row total; each granule has 512 rows) -SYSTEM FLUSH LOGS; -SELECT read_rows==0 from system.query_log - WHERE query_kind ='Select' - AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s ==\'happy birthday\';') - AND type='QueryFinish' - AND result_rows==1 - LIMIT 1; - ----------------------------------------------------- -SELECT 'Test density==0.1'; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 0.1)) - Engine=MergeTree - ORDER BY (k) - SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 - AS - SELECT number, if(number==1023, 'happy new year', if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number)))) - FROM numbers(1024); - --- check inverted index was created - -SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; - --- search inverted index, no row has 'happy birthday' -SELECT count() == 0 FROM tab WHERE s == 'happy birthday'; - --- check the query does not skip any of the 2 granules(1024 rows total; each granule has 512 rows) -SYSTEM FLUSH LOGS; -SELECT read_rows==1024 from system.query_log - WHERE query_kind ='Select' - AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s == \'happy birthday\';') - AND type='QueryFinish' - AND result_rows==1 - LIMIT 1; - --- search inverted index, no row has 'happy new year' -SELECT count() == 1 FROM tab WHERE s == 'happy new year'; - --- check the query only read 1 granule because of density (1024 rows total; each granule has 512 rows) -SYSTEM FLUSH LOGS; -SELECT read_rows==512 from system.query_log - WHERE query_kind ='Select' - AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT count() == 1 FROM tab WHERE s == \'happy new year\';') - AND type='QueryFinish' - AND result_rows==1 - LIMIT 1; +-- ---------------------------------------------------- +-- SELECT 'Test max_digestion_size_per_segment'; +-- +-- DROP TABLE IF EXISTS tab; +-- +-- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0)) +-- Engine=MergeTree +-- ORDER BY (k) +-- SETTINGS max_digestion_size_per_segment = 1024, index_granularity = 256 +-- AS +-- SELECT +-- number, +-- format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s +-- FROM numbers(10240); +-- +-- -- check inverted index was created +-- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; +-- +-- -- search inverted index +-- SELECT s FROM tab WHERE hasToken(s, '6969696969898240'); +-- +-- -- check the query only read 1 granule (1 row total; each granule has 256 rows) +-- SYSTEM FLUSH LOGS; +-- SELECT read_rows==256 from system.query_log +-- WHERE query_kind ='Select' +-- AND current_database = currentDatabase() +-- AND endsWith(trimRight(query), 'SELECT s FROM tab WHERE hasToken(s, \'6969696969898240\');') +-- AND type='QueryFinish' +-- AND result_rows==1 +-- LIMIT 1; +-- +-- ---------------------------------------------------- +-- SELECT 'Test density==1'; +-- +-- DROP TABLE IF EXISTS tab; +-- +-- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 1.0)) +-- Engine=MergeTree +-- ORDER BY (k) +-- SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 +-- AS +-- SELECT number, if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number))) +-- FROM numbers(1024); +-- +-- -- check inverted index was created +-- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; +-- +-- -- search inverted index, no row has 'happy birthday' +-- SELECT count() == 0 FROM tab WHERE s =='happy birthday'; +-- +-- -- check the query only skip all granules (0 row total; each granule has 512 rows) +-- SYSTEM FLUSH LOGS; +-- SELECT read_rows==0 from system.query_log +-- WHERE query_kind ='Select' +-- AND current_database = currentDatabase() +-- AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s ==\'happy birthday\';') +-- AND type='QueryFinish' +-- AND result_rows==1 +-- LIMIT 1; +-- +-- ---------------------------------------------------- +-- SELECT 'Test density==0.1'; +-- +-- DROP TABLE IF EXISTS tab; +-- +-- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 0.1)) +-- Engine=MergeTree +-- ORDER BY (k) +-- SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 +-- AS +-- SELECT number, if(number==1023, 'happy new year', if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number)))) +-- FROM numbers(1024); +-- +-- -- check inverted index was created +-- +-- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; +-- +-- -- search inverted index, no row has 'happy birthday' +-- SELECT count() == 0 FROM tab WHERE s == 'happy birthday'; +-- +-- -- check the query does not skip any of the 2 granules(1024 rows total; each granule has 512 rows) +-- SYSTEM FLUSH LOGS; +-- SELECT read_rows==1024 from system.query_log +-- WHERE query_kind ='Select' +-- AND current_database = currentDatabase() +-- AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s == \'happy birthday\';') +-- AND type='QueryFinish' +-- AND result_rows==1 +-- LIMIT 1; +-- +-- -- search inverted index, no row has 'happy new year' +-- SELECT count() == 1 FROM tab WHERE s == 'happy new year'; +-- +-- -- check the query only read 1 granule because of density (1024 rows total; each granule has 512 rows) +-- SYSTEM FLUSH LOGS; +-- SELECT read_rows==512 from system.query_log +-- WHERE query_kind ='Select' +-- AND current_database = currentDatabase() +-- AND endsWith(trimRight(query), 'SELECT count() == 1 FROM tab WHERE s == \'happy new year\';') +-- AND type='QueryFinish' +-- AND result_rows==1 +-- LIMIT 1; From 7f841454d1b20bd9017f508382b0924f64fb4406 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 Mar 2023 14:57:00 +0000 Subject: [PATCH 117/233] add more test cases --- .../0_stateless/02169_map_functions.reference | 18 ++++++++++++++++++ .../0_stateless/02169_map_functions.sql | 14 ++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index fc7cd1a3c4b..ed5394726c0 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -35,3 +35,21 @@ {'k1':11,'k2':22} {'k1':11,'k2':22} {'k1':11,'k2':22} +{'k1':1,'k2':22,'k3':33,'k4':44} +{'k1':1,'k2':22,'k3':33,'k4':44} +{'k1':1,'k2':22,'k3':33,'k4':44} +{'k1':1,'k2':22,'k3':33,'k4':44} +{'k1':1,'k2':2,'k3':33,'k4':44} +{'k1':1,'k2':2,'k3':33,'k4':44} +{'k1':1,'k2':2,'k3':33,'k4':44} +{'k1':1,'k2':2,'k3':33,'k4':44} +{} +{0:0} +{1:1,0:0} +{1:1,0:0,2:4} +{1:1,3:3,0:0,2:4} +{1:1,3:3,0:0,2:4,4:16} +{1:1,3:3,5:5,0:0,2:4,4:16} +{1:1,3:3,5:5,0:0,2:4,4:16,6:36} +{1:1,3:3,5:5,7:7,0:0,2:4,4:16,6:36} +{1:1,3:3,5:5,7:7,0:0,2:4,4:16,6:36,8:64} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 31112e18b58..1f43647c509 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -20,6 +20,20 @@ SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), map('k1', 11, 'k2', 22)); SELECT mapUpdate(map('k1', 1, 'k2', 2), materialize(map('k1', 11, 'k2', 22))); SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), materialize(map('k1', 11, 'k2', 22))); +SELECT mapUpdate(map('k1', 1, 'k2', 2, 'k3', 3), map('k2', 22, 'k3', 33, 'k4', 44)); +SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2, 'k3', 3)), map('k2', 22, 'k3', 33, 'k4', 44)); +SELECT mapUpdate(map('k1', 1, 'k2', 2, 'k3', 3), materialize(map('k2', 22, 'k3', 33, 'k4', 44))); +SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2, 'k3', 3)), materialize(map('k2', 22, 'k3', 33, 'k4', 44))); + +SELECT mapUpdate(map('k1', 1, 'k2', 2), map('k3', 33, 'k4', 44)); +SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), map('k3', 33, 'k4', 44)); +SELECT mapUpdate(map('k1', 1, 'k2', 2), materialize(map('k3', 33, 'k4', 44))); +SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), materialize(map('k3', 33, 'k4', 44))); + +WITH (range(0, number % 10), range(0, number % 10))::Map(UInt64, UInt64) AS m1, + (range(0, number % 10, 2), arrayMap(x -> x * x, range(0, number % 10, 2)))::Map(UInt64, UInt64) AS m2 +SELECT DISTINCT mapUpdate(m1, m2) FROM numbers (100000); + SELECT mapApply(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From c6056d993daf9f9ebabb867aa7dda8f3256c0e1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Mar 2023 18:00:52 +0300 Subject: [PATCH 118/233] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a09e4d4fe24..bb85d8e33ae 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,7 +18,7 @@ #### New Feature * Add new mode for splitting the work on replicas using settings `parallel_replicas_custom_key` and `parallel_replicas_custom_key_filter_type`. If the cluster consists of a single shard with multiple replicas, up to `max_parallel_replicas` will be randomly picked and turned into shards. For each shard, a corresponding filter is added to the query on the initiator before being sent to the shard. If the cluster consists of multiple shards, it will behave the same as `sample_key` but with the possibility to define an arbitrary key. [#45108](https://github.com/ClickHouse/ClickHouse/pull/45108) ([Antonio Andelic](https://github.com/antonio2368)). -* An option to display partial result on cancel: Added query setting `stop_reading_on_first_cancel` allowing the canceled query (e.g. due to Ctrl-C) to return a partial result. [#45689](https://github.com/ClickHouse/ClickHouse/pull/45689) ([Alexey Perevyshin](https://github.com/alexX512)). +* An option to display partial result on cancel: Added query setting `partial_result_on_first_cancel` allowing the canceled query (e.g. due to Ctrl-C) to return a partial result. [#45689](https://github.com/ClickHouse/ClickHouse/pull/45689) ([Alexey Perevyshin](https://github.com/alexX512)). * Added support of arbitrary tables engines for temporary tables (except for Replicated and KeeperMap engines). Close [#31497](https://github.com/ClickHouse/ClickHouse/issues/31497). [#46071](https://github.com/ClickHouse/ClickHouse/pull/46071) ([Roman Vasin](https://github.com/rvasin)). * Add support for replication of user-defined SQL functions using a centralized storage in Keeper. [#46085](https://github.com/ClickHouse/ClickHouse/pull/46085) ([Aleksei Filatov](https://github.com/aalexfvk)). * Implement `system.server_settings` (similar to `system.settings`), which will contain server configurations. [#46550](https://github.com/ClickHouse/ClickHouse/pull/46550) ([pufit](https://github.com/pufit)). From d186d356a6fc3415fe7f3190e92361bed57fab47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Mar 2023 18:01:52 +0300 Subject: [PATCH 119/233] Update CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bb85d8e33ae..5decadf8f8c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ ### ClickHouse release 23.3 LTS, 2023-03-30 #### Upgrade Notes +* Lightweight DELETEs are production ready and enabled by default. The `DELETE` query for MergeTree tables is now available by default. * The behavior of `*domain*RFC` and `netloc` functions is slightly changed: relaxed the set of symbols that are allowed in the URL authority for better conformance. [#46841](https://github.com/ClickHouse/ClickHouse/pull/46841) ([Azat Khuzhin](https://github.com/azat)). * Prohibited creating tables based on KafkaEngine with DEFAULT/EPHEMERAL/ALIAS/MATERIALIZED statements for columns. [#47138](https://github.com/ClickHouse/ClickHouse/pull/47138) ([Aleksandr Musorin](https://github.com/AVMusorin)). * An "asynchronous connection drain" feature is removed. Related settings and metrics are removed as well. It was an internal feature, so the removal should not affect users who had never heard about that feature. [#47486](https://github.com/ClickHouse/ClickHouse/pull/47486) ([Alexander Tokmakov](https://github.com/tavplubix)). From 06bf500c3062449f48b566f6a7b46e301366b534 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 30 Mar 2023 17:36:56 +0200 Subject: [PATCH 120/233] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 533072a2372..43797f593d2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4061,7 +4061,7 @@ Possible values: Default value: `0`. ## partial_result_on_first_cancel {#partial_result_on_first_cancel} -When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterward, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. +When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterwards, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. **Example without setting on Ctrl+C** ```sql From 9593ffc7cb928d56eb03d8a53047cd04716cfdcd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 30 Mar 2023 17:37:31 +0200 Subject: [PATCH 121/233] Determine the hosts order in `SHOW CLUSTER` query --- src/Interpreters/InterpreterShowTablesQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index a631cb72722..158c74d7dc6 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -87,7 +87,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() rewritten_query << " WHERE cluster = " << DB::quote << query.cluster_str; /// (*) - rewritten_query << " ORDER BY cluster"; + rewritten_query << " ORDER BY cluster, shard_num, replica_num, host_name, host_address, port"; return rewritten_query.str(); } From 1c58a911eb77eee5abcc1cdd78d71de9690d3323 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 30 Mar 2023 18:07:40 +0200 Subject: [PATCH 122/233] Push clickhouse-keeper as both w/ and w/o suffix `-alpine` --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- .github/workflows/release.yml | 2 +- .github/workflows/release_branches.yml | 2 +- docker/keeper/Dockerfile | 3 +++ docker/keeper/Dockerfile.ubuntu | 1 + 7 files changed, 9 insertions(+), 5 deletions(-) create mode 120000 docker/keeper/Dockerfile.ubuntu diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 867cca9d037..0d81a7b303c 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -470,7 +470,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_server.py --release-type head --no-push \ --image-repo clickhouse/clickhouse-server --image-path docker/server - python3 docker_server.py --release-type head --no-push --no-ubuntu \ + python3 docker_server.py --release-type head --no-push \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7c5e477ab60..b1ea1641a02 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -862,7 +862,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_server.py --release-type head \ --image-repo clickhouse/clickhouse-server --image-path docker/server - python3 docker_server.py --release-type head --no-ubuntu \ + python3 docker_server.py --release-type head \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 2f2c263df37..ab0cbbb7ec1 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -918,7 +918,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_server.py --release-type head --no-push \ --image-repo clickhouse/clickhouse-server --image-path docker/server - python3 docker_server.py --release-type head --no-push --no-ubuntu \ + python3 docker_server.py --release-type head --no-push \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 73246af6dfc..0742ebfd449 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -55,7 +55,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_server.py --release-type auto --version "$GITHUB_TAG" \ --image-repo clickhouse/clickhouse-server --image-path docker/server - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --no-ubuntu \ + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index e56a1fb58fc..1282dbef50b 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -527,7 +527,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_server.py --release-type head --no-push \ --image-repo clickhouse/clickhouse-server --image-path docker/server - python3 docker_server.py --release-type head --no-push --no-ubuntu \ + python3 docker_server.py --release-type head --no-push \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 34c1406b687..6496a2b2a12 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -1,3 +1,6 @@ +# The Dockerfile.ubuntu exists for the tests/ci/docker_server.py script +# If the image is built from Dockerfile.alpine, then the `-alpine` suffix is added automatically, +# so the only purpose of Dockerfile.ubuntu is to push `latest`, `head` and so on w/o suffixes FROM ubuntu:20.04 AS glibc-donor ARG TARGETARCH diff --git a/docker/keeper/Dockerfile.ubuntu b/docker/keeper/Dockerfile.ubuntu new file mode 120000 index 00000000000..1d1fe94df49 --- /dev/null +++ b/docker/keeper/Dockerfile.ubuntu @@ -0,0 +1 @@ +Dockerfile \ No newline at end of file From 3a3438ebf970cd4eae6de275e04bb4699a9266d5 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Thu, 30 Mar 2023 10:26:33 -0600 Subject: [PATCH 123/233] Update settings-formats.md Default compression method for Parquet changed from snappy to lz4 in 23.3 --- docs/en/operations/settings/settings-formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 40ca914b738..91b67ee8238 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1128,7 +1128,7 @@ Default value: `2.latest`. Compression method used in output Parquet format. Supported codecs: `snappy`, `lz4`, `brotli`, `zstd`, `gzip`, `none` (uncompressed) -Default value: `snappy`. +Default value: `lz4`. ## Hive format settings {#hive-format-settings} From 4675fa82a33dad335a782116e53bc9a76eb1dfd1 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Thu, 30 Mar 2023 10:29:08 -0600 Subject: [PATCH 124/233] Update delete.md Lightweight deletes are no longer experimental in 23.3 --- docs/en/sql-reference/statements/delete.md | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 7d7b8855d51..149e7ab371f 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -21,15 +21,6 @@ DELETE FROM hits WHERE Title LIKE '%hello%'; Lightweight deletes are asynchronous by default. Set `mutations_sync` equal to 1 to wait for one replica to process the statement, and set `mutations_sync` to 2 to wait for all replicas. -:::note -This feature is experimental and requires you to set `allow_experimental_lightweight_delete` to true: - -```sql -SET allow_experimental_lightweight_delete = true; -``` - -::: - :::note `DELETE FROM` requires the `ALTER DELETE` privilege: ```sql @@ -64,6 +55,3 @@ With the described implementation now we can see what can negatively affect 'DEL - Table having a very large number of data parts - Having a lot of data in Compact parts—in a Compact part, all columns are stored in one file. -:::note -This implementation might change in the future. -::: From 191f78022321786d2dd6fa9f3369b2849be4107e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 30 Mar 2023 17:22:20 +0000 Subject: [PATCH 125/233] Sort output of SHOW PROCESSLIST Follow-up to #48127 --- src/Interpreters/InterpreterShowProcesslistQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/src/Interpreters/InterpreterShowProcesslistQuery.cpp index 780ba688a89..1522af4bcbe 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -12,7 +12,7 @@ namespace DB BlockIO InterpreterShowProcesslistQuery::execute() { - return executeQuery("SELECT * FROM system.processes", getContext(), true); + return executeQuery("SELECT * FROM system.processes ORDER BY query_id", getContext(), true); } } From 4938681f87a68cc382d6817048308d5d259b3669 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 30 Mar 2023 17:46:11 +0000 Subject: [PATCH 126/233] Fix macos build --- docs/en/development/build-cross-osx.md | 8 ++++---- src/IO/VarInt.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index 1df88dbb235..e6e5bd6ca4d 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -11,14 +11,14 @@ This is intended for continuous integration checks that run on Linux servers. If The cross-build for macOS is based on the [Build instructions](../development/build.md), follow them first. -## Install Clang-14 +## Install Clang-15 Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup. For example the commands for Bionic are like: ``` bash -sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-14 main" >> /etc/apt/sources.list -sudo apt-get install clang-14 +sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-15 main" >> /etc/apt/sources.list +sudo apt-get install clang-15 ``` ## Install Cross-Compilation Toolset {#install-cross-compilation-toolset} @@ -55,7 +55,7 @@ curl -L 'https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX1 cd ClickHouse mkdir build-darwin cd build-darwin -CC=clang-14 CXX=clang++-14 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/x86_64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. +CC=clang-15 CXX=clang++-15 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/x86_64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. ninja ``` diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index f95b479df11..fa1908cf615 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -25,7 +25,7 @@ namespace ErrorCodes * So implementing VLQ for the whole 1<<64 range will require different set of * helpers. */ -constexpr size_t VAR_UINT_MAX = (1ULL<<63) - 1; +constexpr UInt64 VAR_UINT_MAX = (1ULL<<63) - 1; /** Write UInt64 in variable length format (base128) */ void writeVarUInt(UInt64 x, std::ostream & ostr); From edeeac84cf0792503c2f9efc65390ce0d735648e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 30 Mar 2023 20:08:38 +0200 Subject: [PATCH 127/233] Fix incorrect ThreadPool usage after ThreadPool introspection ``` $ gg 'ThreadPool[^()]([A-Za-z_]\+,' src/ src/Interpreters/Context.cpp: shared->load_marks_threadpool = std::make_unique(pool_size, pool_size, queue_size); src/Interpreters/Context.cpp: shared->prefetch_threadpool = std::make_unique(pool_size, pool_size, queue_size); src/Interpreters/Context.cpp: shared->threadpool_writer = std::make_unique(pool_size, pool_size, queue_size); ``` Fixes: #47880 Signed-off-by: Azat Khuzhin --- src/Common/CurrentMetrics.cpp | 6 ++++++ src/Interpreters/Context.cpp | 15 ++++++++++++--- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 4c773048597..542c48148c8 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -78,6 +78,12 @@ M(BackupsThreadsActive, "Number of threads in thread pool for BACKUP running a task.") \ M(RestoreThreads, "Number of threads in the thread pool for RESTORE.") \ M(RestoreThreadsActive, "Number of threads in the thread pool for RESTORE running a task.") \ + M(MarksLoaderThreads, "Number of threads in thread pool for loading marks.") \ + M(MarksLoaderThreadsActive, "Number of threads in the thread pool for loading marks running a task.") \ + M(IOPrefetchThreads, "Number of threads in the IO prefertch thread pool.") \ + M(IOPrefetchThreadsActive, "Number of threads in the IO prefetch thread pool running a task.") \ + M(IOWriterThreads, "Number of threads in the IO writer thread pool.") \ + M(IOWriterThreadsActive, "Number of threads in the IO writer thread pool running a task.") \ M(IOThreads, "Number of threads in the IO thread pool.") \ M(IOThreadsActive, "Number of threads in the IO thread pool running a task.") \ M(ThreadPoolRemoteFSReaderThreads, "Number of threads in the thread pool for remote_filesystem_read_method=threadpool.") \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 82a8e43e8e2..522107dccc9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -145,6 +145,12 @@ namespace CurrentMetrics extern const Metric BackgroundFetchesPoolSize; extern const Metric BackgroundCommonPoolTask; extern const Metric BackgroundCommonPoolSize; + extern const Metric MarksLoaderThreads; + extern const Metric MarksLoaderThreadsActive; + extern const Metric IOPrefetchThreads; + extern const Metric IOPrefetchThreadsActive; + extern const Metric IOWriterThreads; + extern const Metric IOWriterThreadsActive; } namespace DB @@ -2018,7 +2024,8 @@ ThreadPool & Context::getLoadMarksThreadpool() const { auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); - shared->load_marks_threadpool = std::make_unique(pool_size, pool_size, queue_size); + shared->load_marks_threadpool = std::make_unique( + CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, pool_size, pool_size, queue_size); } return *shared->load_marks_threadpool; } @@ -2043,7 +2050,8 @@ ThreadPool & Context::getPrefetchThreadpool() const { auto pool_size = getPrefetchThreadpoolSize(); auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); - shared->prefetch_threadpool = std::make_unique(pool_size, pool_size, queue_size); + shared->prefetch_threadpool = std::make_unique( + CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, pool_size, pool_size, queue_size); } return *shared->prefetch_threadpool; } @@ -3967,7 +3975,8 @@ ThreadPool & Context::getThreadPoolWriter() const auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); - shared->threadpool_writer = std::make_unique(pool_size, pool_size, queue_size); + shared->threadpool_writer = std::make_unique( + CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); } return *shared->threadpool_writer; From 37213aa6b49b773a81e4810f8d65d7a9210f4e1f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 30 Mar 2023 14:45:01 -0400 Subject: [PATCH 128/233] edits --- CHANGELOG.md | 72 ++++++++++++++++++++++++++-------------------------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5decadf8f8c..47320208f02 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,60 +18,60 @@ * Do not allow const and non-deterministic secondary indices [#46839](https://github.com/ClickHouse/ClickHouse/pull/46839) ([Anton Popov](https://github.com/CurtizJ)). #### New Feature -* Add new mode for splitting the work on replicas using settings `parallel_replicas_custom_key` and `parallel_replicas_custom_key_filter_type`. If the cluster consists of a single shard with multiple replicas, up to `max_parallel_replicas` will be randomly picked and turned into shards. For each shard, a corresponding filter is added to the query on the initiator before being sent to the shard. If the cluster consists of multiple shards, it will behave the same as `sample_key` but with the possibility to define an arbitrary key. [#45108](https://github.com/ClickHouse/ClickHouse/pull/45108) ([Antonio Andelic](https://github.com/antonio2368)). +* Add a new mode for splitting the work on replicas using settings `parallel_replicas_custom_key` and `parallel_replicas_custom_key_filter_type`. If the cluster consists of a single shard with multiple replicas, up to `max_parallel_replicas` will be randomly picked and turned into shards. For each shard, a corresponding filter is added to the query on the initiator before being sent to the shard. If the cluster consists of multiple shards, it will behave the same as `sample_key` but with the possibility to define an arbitrary key. [#45108](https://github.com/ClickHouse/ClickHouse/pull/45108) ([Antonio Andelic](https://github.com/antonio2368)). * An option to display partial result on cancel: Added query setting `partial_result_on_first_cancel` allowing the canceled query (e.g. due to Ctrl-C) to return a partial result. [#45689](https://github.com/ClickHouse/ClickHouse/pull/45689) ([Alexey Perevyshin](https://github.com/alexX512)). * Added support of arbitrary tables engines for temporary tables (except for Replicated and KeeperMap engines). Close [#31497](https://github.com/ClickHouse/ClickHouse/issues/31497). [#46071](https://github.com/ClickHouse/ClickHouse/pull/46071) ([Roman Vasin](https://github.com/rvasin)). -* Add support for replication of user-defined SQL functions using a centralized storage in Keeper. [#46085](https://github.com/ClickHouse/ClickHouse/pull/46085) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Add support for replication of user-defined SQL functions using centralized storage in Keeper. [#46085](https://github.com/ClickHouse/ClickHouse/pull/46085) ([Aleksei Filatov](https://github.com/aalexfvk)). * Implement `system.server_settings` (similar to `system.settings`), which will contain server configurations. [#46550](https://github.com/ClickHouse/ClickHouse/pull/46550) ([pufit](https://github.com/pufit)). * Support for `UNDROP TABLE` query. Closes [#46811](https://github.com/ClickHouse/ClickHouse/issues/46811). [#47241](https://github.com/ClickHouse/ClickHouse/pull/47241) ([chen](https://github.com/xiedeyantu)). -* Allow separate grants for named collections (e.g. to be able to give `SHOW/CREATE/ALTER/DROP named collection` access only to certain collections, instead of all at once). Closes [#40894](https://github.com/ClickHouse/ClickHouse/issues/40894). Add new access type `NAMED_COLLECTION_CONTROL` which is not given to default user unless explicitly added to user config (is required to be able to do `GRANT ALL`), also `show_named_collections` is no longer obligatory to be manually specified for default user to be able to have full access rights as was in 23.2. [#46241](https://github.com/ClickHouse/ClickHouse/pull/46241) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow separate grants for named collections (e.g. to be able to give `SHOW/CREATE/ALTER/DROP named collection` access only to certain collections, instead of all at once). Closes [#40894](https://github.com/ClickHouse/ClickHouse/issues/40894). Add new access type `NAMED_COLLECTION_CONTROL` which is not given to user default unless explicitly added to the user config (is required to be able to do `GRANT ALL`), also `show_named_collections` is no longer obligatory to be manually specified for user default to be able to have full access rights as was in 23.2. [#46241](https://github.com/ClickHouse/ClickHouse/pull/46241) ([Kseniia Sumarokova](https://github.com/kssenii)). * Allow nested custom disks. Previously custom disks supported only flat disk structure. [#47106](https://github.com/ClickHouse/ClickHouse/pull/47106) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Intruduce a function `widthBucket` (with a `WIDTH_BUCKET` alias for compatibility). [#42974](https://github.com/ClickHouse/ClickHouse/issues/42974). [#46790](https://github.com/ClickHouse/ClickHouse/pull/46790) ([avoiderboi](https://github.com/avoiderboi)). -* Add new function `parseDateTime`/`parseDateTimeInJodaSyntax` according to specified format string. parseDateTime parses string to datetime in MySQL syntax, parseDateTimeInJodaSyntax parses in Joda syntax. [#46815](https://github.com/ClickHouse/ClickHouse/pull/46815) ([李扬](https://github.com/taiyang-li)). -* Use `dummy UInt8` for default structure of table function `null`. Closes [#46930](https://github.com/ClickHouse/ClickHouse/issues/46930). [#47006](https://github.com/ClickHouse/ClickHouse/pull/47006) ([flynn](https://github.com/ucasfl)). +* Introduce a function `widthBucket` (with a `WIDTH_BUCKET` alias for compatibility). [#42974](https://github.com/ClickHouse/ClickHouse/issues/42974). [#46790](https://github.com/ClickHouse/ClickHouse/pull/46790) ([avoiderboi](https://github.com/avoiderboi)). +* Add new function `parseDateTime`/`parseDateTimeInJodaSyntax` according to the specified format string. parseDateTime parses String to DateTime in MySQL syntax, parseDateTimeInJodaSyntax parses in Joda syntax. [#46815](https://github.com/ClickHouse/ClickHouse/pull/46815) ([李扬](https://github.com/taiyang-li)). +* Use `dummy UInt8` for the default structure of table function `null`. Closes [#46930](https://github.com/ClickHouse/ClickHouse/issues/46930). [#47006](https://github.com/ClickHouse/ClickHouse/pull/47006) ([flynn](https://github.com/ucasfl)). * Support for date format with a comma, like `Dec 15, 2021` in the `parseDateTimeBestEffort` function. Closes [#46816](https://github.com/ClickHouse/ClickHouse/issues/46816). [#47071](https://github.com/ClickHouse/ClickHouse/pull/47071) ([chen](https://github.com/xiedeyantu)). -* Add settings `http_wait_end_of_query` and `http_response_buffer_size` that corresponds to URL params `wait_end_of_query` and `buffer_size` for HTTP interface. This allows to change these settings in the profiles. [#47108](https://github.com/ClickHouse/ClickHouse/pull/47108) ([Vladimir C](https://github.com/vdimir)). +* Add settings `http_wait_end_of_query` and `http_response_buffer_size` that corresponds to URL params `wait_end_of_query` and `buffer_size` for the HTTP interface. This allows changing these settings in the profiles. [#47108](https://github.com/ClickHouse/ClickHouse/pull/47108) ([Vladimir C](https://github.com/vdimir)). * Add `system.dropped_tables` table that shows tables that were dropped from `Atomic` databases but were not completely removed yet. [#47364](https://github.com/ClickHouse/ClickHouse/pull/47364) ([chen](https://github.com/xiedeyantu)). * Add `INSTR` as alias of `positionCaseInsensitive` for MySQL compatibility. Closes [#47529](https://github.com/ClickHouse/ClickHouse/issues/47529). [#47535](https://github.com/ClickHouse/ClickHouse/pull/47535) ([flynn](https://github.com/ucasfl)). * Added `toDecimalString` function allowing to convert numbers to string with fixed precision. [#47838](https://github.com/ClickHouse/ClickHouse/pull/47838) ([Andrey Zvonov](https://github.com/zvonand)). * Add a merge tree setting `max_number_of_mutations_for_replica`. It limits the number of part mutations per replica to the specified amount. Zero means no limit on the number of mutations per replica (the execution can still be constrained by other settings). [#48047](https://github.com/ClickHouse/ClickHouse/pull/48047) ([Vladimir C](https://github.com/vdimir)). -* Add Map-related function `mapFromArrays`, which allows us to create map from a pair of arrays. [#31125](https://github.com/ClickHouse/ClickHouse/pull/31125) ([李扬](https://github.com/taiyang-li)). -* Allow control compression in Parquet/ORC/Arrow output formats, support more compression for input formats. This closes [#13541](https://github.com/ClickHouse/ClickHouse/issues/13541). [#47114](https://github.com/ClickHouse/ClickHouse/pull/47114) ([Kruglov Pavel](https://github.com/Avogar)). +* Add the Map-related function `mapFromArrays`, which allows the creation of a map from a pair of arrays. [#31125](https://github.com/ClickHouse/ClickHouse/pull/31125) ([李扬](https://github.com/taiyang-li)). +* Allow control of compression in Parquet/ORC/Arrow output formats, adds support for more compression input formats. This closes [#13541](https://github.com/ClickHouse/ClickHouse/issues/13541). [#47114](https://github.com/ClickHouse/ClickHouse/pull/47114) ([Kruglov Pavel](https://github.com/Avogar)). * Add SSL User Certificate authentication to the native protocol. Closes [#47077](https://github.com/ClickHouse/ClickHouse/issues/47077). [#47596](https://github.com/ClickHouse/ClickHouse/pull/47596) ([Nikolay Degterinsky](https://github.com/evillique)). * Add *OrNull() and *OrZero() variants for `parseDateTime`, add alias `str_to_date` for MySQL parity. [#48000](https://github.com/ClickHouse/ClickHouse/pull/48000) ([Robert Schulze](https://github.com/rschu1ze)). * Added operator `REGEXP` (similar to operators "LIKE", "IN", "MOD" etc.) for better compatibility with MySQL [#47869](https://github.com/ClickHouse/ClickHouse/pull/47869) ([Robert Schulze](https://github.com/rschu1ze)). #### Performance Improvement * Marks in memory are now compressed, using 3-6x less memory. [#47290](https://github.com/ClickHouse/ClickHouse/pull/47290) ([Michael Kolupaev](https://github.com/al13n321)). -* Backups for large numbers of files were unbelievably slow in previous versions. Not anymore. Now they are unbelievably fast. [#47251](https://github.com/ClickHouse/ClickHouse/pull/47251) ([Alexey Milovidov](https://github.com/alexey-milovidov)). Introduced a separate thread pool for backup's IO operations. This will allow to scale it independently of other pools and increase performance. [#47174](https://github.com/ClickHouse/ClickHouse/pull/47174) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). Use MultiRead request and retries for collecting metadata at final stage of backup processing. [#47243](https://github.com/ClickHouse/ClickHouse/pull/47243) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). If a backup and restoring data are both in S3 then server-side copy should be used from now on. [#47546](https://github.com/ClickHouse/ClickHouse/pull/47546) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backups for large numbers of files were unbelievably slow in previous versions. Not anymore. Now they are unbelievably fast. [#47251](https://github.com/ClickHouse/ClickHouse/pull/47251) ([Alexey Milovidov](https://github.com/alexey-milovidov)). Introduced a separate thread pool for backup's IO operations. This will allow scaling it independently of other pools and increase performance. [#47174](https://github.com/ClickHouse/ClickHouse/pull/47174) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). Use MultiRead request and retries for collecting metadata at the final stage of backup processing. [#47243](https://github.com/ClickHouse/ClickHouse/pull/47243) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). If a backup and restoring data are both in S3 then server-side copy should be used from now on. [#47546](https://github.com/ClickHouse/ClickHouse/pull/47546) ([Vitaly Baranov](https://github.com/vitlibar)). * Fixed excessive reading in queries with `FINAL`. [#47801](https://github.com/ClickHouse/ClickHouse/pull/47801) ([Nikita Taranov](https://github.com/nickitat)). -* Setting `max_final_threads` would be set to number of cores at server startup (by the same algorithm as we use for `max_threads`). This improves concurrency of `final` execution on servers with high number of CPUs. [#47915](https://github.com/ClickHouse/ClickHouse/pull/47915) ([Nikita Taranov](https://github.com/nickitat)). +* Setting `max_final_threads` would be set to the number of cores at server startup (by the same algorithm as used for `max_threads`). This improves the concurrency of `final` execution on servers with high number of CPUs. [#47915](https://github.com/ClickHouse/ClickHouse/pull/47915) ([Nikita Taranov](https://github.com/nickitat)). * Allow executing reading pipeline for DIRECT dictionary with CLICKHOUSE source in multiple threads. To enable set `dictionary_use_async_executor=1` in `SETTINGS` section for source in `CREATE DICTIONARY` statement. [#47986](https://github.com/ClickHouse/ClickHouse/pull/47986) ([Vladimir C](https://github.com/vdimir)). * Optimize one nullable key aggregate performance. [#45772](https://github.com/ClickHouse/ClickHouse/pull/45772) ([LiuNeng](https://github.com/liuneng1994)). * Implemented lowercase `tokenbf_v1` index utilization for `hasTokenOrNull`, `hasTokenCaseInsensitive` and `hasTokenCaseInsensitiveOrNull`. [#46252](https://github.com/ClickHouse/ClickHouse/pull/46252) ([ltrk2](https://github.com/ltrk2)). * Optimize functions `position` and `LIKE` by searching the first two chars using SIMD. [#46289](https://github.com/ClickHouse/ClickHouse/pull/46289) ([Jiebin Sun](https://github.com/jiebinn)). -* Optimize queries from the `system.detached_parts`, which could be significantly large. Added several sources with respect to the block size limitation; in each block an IO thread pool is used to calculate the part size, i.e. to make syscalls in parallel. [#46624](https://github.com/ClickHouse/ClickHouse/pull/46624) ([Sema Checherinda](https://github.com/CheSema)). +* Optimize queries from the `system.detached_parts`, which could be significantly large. Added several sources with respect to the block size limitation; in each block, an IO thread pool is used to calculate the part size, i.e. to make syscalls in parallel. [#46624](https://github.com/ClickHouse/ClickHouse/pull/46624) ([Sema Checherinda](https://github.com/CheSema)). * Increase the default value of `max_replicated_merges_in_queue` for ReplicatedMergeTree tables from 16 to 1000. It allows faster background merge operation on clusters with a very large number of replicas, such as clusters with shared storage in ClickHouse Cloud. [#47050](https://github.com/ClickHouse/ClickHouse/pull/47050) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Updated `clickhouse-copier` to use `GROUP BY` instead of `DISTINCT` to get list of partitions. For large tables this reduced the select time from over 500s to under 1s. [#47386](https://github.com/ClickHouse/ClickHouse/pull/47386) ([Clayton McClure](https://github.com/cmcclure-twilio)). +* Updated `clickhouse-copier` to use `GROUP BY` instead of `DISTINCT` to get the list of partitions. For large tables, this reduced the select time from over 500s to under 1s. [#47386](https://github.com/ClickHouse/ClickHouse/pull/47386) ([Clayton McClure](https://github.com/cmcclure-twilio)). * Fix performance degradation in `ASOF JOIN`. [#47544](https://github.com/ClickHouse/ClickHouse/pull/47544) ([Ongkong](https://github.com/ongkong)). -* Even more batching in Keeper. Avoid breaking batches on read requests to improve performance. [#47978](https://github.com/ClickHouse/ClickHouse/pull/47978) ([Antonio Andelic](https://github.com/antonio2368)). -* Allow PREWHERE for Merge with different DEFAULT expression for column. [#46831](https://github.com/ClickHouse/ClickHouse/pull/46831) ([Azat Khuzhin](https://github.com/azat)). +* Even more batching in Keeper. Improve performance by avoiding breaking batches on read requests. [#47978](https://github.com/ClickHouse/ClickHouse/pull/47978) ([Antonio Andelic](https://github.com/antonio2368)). +* Allow PREWHERE for Merge with different DEFAULT expressions for columns. [#46831](https://github.com/ClickHouse/ClickHouse/pull/46831) ([Azat Khuzhin](https://github.com/azat)). #### Experimental Feature -* Parallel replicas: Improved the overall performance by better utilizing local replica. And forbid reading with parallel replicas from non-replicated MergeTree by default. [#47858](https://github.com/ClickHouse/ClickHouse/pull/47858) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Parallel replicas: Improved the overall performance by better utilizing the local replica, and forbid the reading with parallel replicas from non-replicated MergeTree by default. [#47858](https://github.com/ClickHouse/ClickHouse/pull/47858) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Support filter push down to left table for JOIN with `Join`, `Dictionary` and `EmbeddedRocksDB` tables if the experimental Analyzer is enabled. [#47280](https://github.com/ClickHouse/ClickHouse/pull/47280) ([Maksim Kita](https://github.com/kitaisreal)). * Now ReplicatedMergeTree with zero copy replication has less load to Keeper. [#47676](https://github.com/ClickHouse/ClickHouse/pull/47676) ([alesapin](https://github.com/alesapin)). * Fix create materialized view with MaterializedPostgreSQL [#40807](https://github.com/ClickHouse/ClickHouse/pull/40807) ([Maksim Buren](https://github.com/maks-buren630501)). #### Improvement * Enable `input_format_json_ignore_unknown_keys_in_named_tuple` by default. [#46742](https://github.com/ClickHouse/ClickHouse/pull/46742) ([Kruglov Pavel](https://github.com/Avogar)). -* Allow to ignore errors while pushing to MATERIALIZED VIEW (add new setting `materialized_views_ignore_errors`, by default to `false`, but it is set to `true` for flushing logs to `system.*_log` tables unconditionally). [#46658](https://github.com/ClickHouse/ClickHouse/pull/46658) ([Azat Khuzhin](https://github.com/azat)). +* Allow errors to be ignored while pushing to MATERIALIZED VIEW (add new setting `materialized_views_ignore_errors`, by default to `false`, but it is set to `true` for flushing logs to `system.*_log` tables unconditionally). [#46658](https://github.com/ClickHouse/ClickHouse/pull/46658) ([Azat Khuzhin](https://github.com/azat)). * Track the file queue of distributed sends in memory. [#45491](https://github.com/ClickHouse/ClickHouse/pull/45491) ([Azat Khuzhin](https://github.com/azat)). -* Now `X-ClickHouse-Query-Id` and `X-ClickHouse-Timezone` headers are added to response in all queries via http protocol. Previously it was done only for `SELECT` queries. [#46364](https://github.com/ClickHouse/ClickHouse/pull/46364) ([Anton Popov](https://github.com/CurtizJ)). +* Now `X-ClickHouse-Query-Id` and `X-ClickHouse-Timezone` headers are added to responses in all queries via HTTP protocol. Previously it was done only for `SELECT` queries. [#46364](https://github.com/ClickHouse/ClickHouse/pull/46364) ([Anton Popov](https://github.com/CurtizJ)). * External tables from `MongoDB`: support for connection to a replica set via a URI with a host:port enum and support for the readPreference option in MongoDB dictionaries. Example URI: mongodb://db0.example.com:27017,db1.example.com:27017,db2.example.com:27017/?replicaSet=myRepl&readPreference=primary. [#46524](https://github.com/ClickHouse/ClickHouse/pull/46524) ([artem-yadr](https://github.com/artem-yadr)). * This improvement should be invisible for users. Re-implement projection analysis on top of query plan. Added setting `query_plan_optimize_projection=1` to switch between old and new version. Fixes [#44963](https://github.com/ClickHouse/ClickHouse/issues/44963). [#46537](https://github.com/ClickHouse/ClickHouse/pull/46537) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Use parquet format v2 instead of v1 in output format by default. Add setting `output_format_parquet_version` to control parquet version, possible values `1.0`, `2.4`, `2.6`, `2.latest` (default). [#46617](https://github.com/ClickHouse/ClickHouse/pull/46617) ([Kruglov Pavel](https://github.com/Avogar)). -* It is now possible using new configuration syntax to configure Kafka topics with periods (`.`) in their name. [#46752](https://github.com/ClickHouse/ClickHouse/pull/46752) ([Robert Schulze](https://github.com/rschu1ze)). +* Use Parquet format v2 instead of v1 in output format by default. Add setting `output_format_parquet_version` to control parquet version, possible values `1.0`, `2.4`, `2.6`, `2.latest` (default). [#46617](https://github.com/ClickHouse/ClickHouse/pull/46617) ([Kruglov Pavel](https://github.com/Avogar)). +* It is now possible to use the new configuration syntax to configure Kafka topics with periods (`.`) in their name. [#46752](https://github.com/ClickHouse/ClickHouse/pull/46752) ([Robert Schulze](https://github.com/rschu1ze)). * Fix heuristics that check hyperscan patterns for problematic repeats. [#46819](https://github.com/ClickHouse/ClickHouse/pull/46819) ([Robert Schulze](https://github.com/rschu1ze)). * Don't report ZK node exists to system.errors when a block was created concurrently by a different replica. [#46820](https://github.com/ClickHouse/ClickHouse/pull/46820) ([Raúl Marín](https://github.com/Algunenano)). * Increase the limit for opened files in `clickhouse-local`. It will be able to read from `web` tables on servers with a huge number of CPU cores. Do not back off reading from the URL table engine in case of too many opened files. This closes [#46852](https://github.com/ClickHouse/ClickHouse/issues/46852). [#46853](https://github.com/ClickHouse/ClickHouse/pull/46853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -79,7 +79,7 @@ * Added update `system.backups` after every processed task to track the progress of backups. [#46989](https://github.com/ClickHouse/ClickHouse/pull/46989) ([Aleksandr Musorin](https://github.com/AVMusorin)). * Allow types conversion in Native input format. Add settings `input_format_native_allow_types_conversion` that controls it (enabled by default). [#46990](https://github.com/ClickHouse/ClickHouse/pull/46990) ([Kruglov Pavel](https://github.com/Avogar)). * Allow IPv4 in the `range` function to generate IP ranges. [#46995](https://github.com/ClickHouse/ClickHouse/pull/46995) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Improve exception message when it's impossible to make part move from one volume/disk to another. [#47032](https://github.com/ClickHouse/ClickHouse/pull/47032) ([alesapin](https://github.com/alesapin)). +* Improve exception message when it's impossible to move a part from one volume/disk to another. [#47032](https://github.com/ClickHouse/ClickHouse/pull/47032) ([alesapin](https://github.com/alesapin)). * Support `Bool` type in `JSONType` function. Previously `Null` type was mistakenly returned for bool values. [#47046](https://github.com/ClickHouse/ClickHouse/pull/47046) ([Anton Popov](https://github.com/CurtizJ)). * Use `_request_body` parameter to configure predefined HTTP queries. [#47086](https://github.com/ClickHouse/ClickHouse/pull/47086) ([Constantine Peresypkin](https://github.com/pkit)). * Automatic indentation in the built-in UI SQL editor when Enter is pressed. [#47113](https://github.com/ClickHouse/ClickHouse/pull/47113) ([Alexey Korepanov](https://github.com/alexkorep)). @@ -87,27 +87,27 @@ * Previously, the `repeat` function's second argument only accepted an unsigned integer type, which meant it could not accept values such as -1. This behavior differed from that of the Spark function. In this update, the repeat function has been modified to match the behavior of the Spark function. It now accepts the same types of inputs, including negative integers. Extensive testing has been performed to verify the correctness of the updated implementation. [#47134](https://github.com/ClickHouse/ClickHouse/pull/47134) ([KevinyhZou](https://github.com/KevinyhZou)). Note: the changelog entry was rewritten by ChatGPT. * Remove `::__1` part from stacktraces. Display `std::basic_string Date: Thu, 30 Mar 2023 20:42:06 +0200 Subject: [PATCH 129/233] add zk retries for distributed ddl --- src/Interpreters/DDLWorker.cpp | 1 + src/Interpreters/DDLWorker.h | 1 + src/Interpreters/executeDDLQueryOnCluster.cpp | 52 ++++++++++++++++--- src/Interpreters/executeDDLQueryOnCluster.h | 3 +- 4 files changed, 48 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 22bece0ef04..5389d11a2ce 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 6cf034edae8..62ca6cba5e8 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c40b9c779c9..108d9bccf33 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -37,6 +37,27 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +struct RetriesForDDL +{ + ZooKeeperRetriesInfo info; + ZooKeeperRetriesControl ctl; +}; + +static RetriesForDDL getRetriesForDistributedDDL() +{ + const auto & config_ref = Context::getGlobalContextInstance()->getConfigRef(); + auto info = ZooKeeperRetriesInfo( + "DistributedDDL", + &Poco::Logger::get("DDLQueryStatusSource"), + config_ref.getInt("distributed_ddl_keeper_max_retries", 5), + config_ref.getInt("distributed_ddl_keeper_initial_backoff_ms", 100), + config_ref.getInt("distributed_ddl_keeper_max_backoff_ms", 5000) + ); + + auto ctl = ZooKeeperRetriesControl("executeDDLQueryOnCluster", info); + return {info, ctl}; +} + bool isSupportedAlterType(int type) { assert(type != ASTAlterCommand::NO_TYPE); @@ -174,7 +195,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.tracing_context = OpenTelemetry::CurrentContext(); String node_path = ddl_worker.enqueueQuery(entry); - return getDistributedDDLStatus(node_path, entry, context); + return getDistributedDDLStatus(node_path, entry, context, /* hosts_to_wait */ std::nullopt); } @@ -182,7 +203,7 @@ class DDLQueryStatusSource final : public ISource { public: DDLQueryStatusSource( - const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const std::optional & hosts_to_wait = {}); + const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const std::optional & hosts_to_wait); String getName() const override { return "DDLQueryStatus"; } Chunk generate() override; @@ -230,7 +251,8 @@ private: }; -BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional & hosts_to_wait) +BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, + const std::optional & hosts_to_wait) { BlockIO io; if (context->getSettingsRef().distributed_ddl_task_timeout == 0) @@ -380,7 +402,6 @@ Chunk DDLQueryStatusSource::generate() if (is_replicated_database && context->getSettingsRef().database_replicated_enforce_synchronous_settings) node_to_wait = "synced"; - auto zookeeper = context->getZooKeeper(); size_t try_number = 0; while (true) @@ -420,7 +441,18 @@ Chunk DDLQueryStatusSource::generate() sleepForMilliseconds(std::min(1000, 50 * (try_number + 1))); } - if (!zookeeper->exists(node_path)) + bool node_exists = false; + Strings tmp_hosts; + Strings tmp_active_hosts; + + getRetriesForDistributedDDL().ctl.retryLoop([&](){ + auto zookeeper = context->getZooKeeper(); + node_exists = zookeeper->exists(node_path); + tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait); + tmp_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active"); + }); + + if (!node_exists) { /// Paradoxically, this exception will be throw even in case of "never_throw" mode. @@ -432,12 +464,12 @@ Chunk DDLQueryStatusSource::generate() return {}; } - Strings new_hosts = getNewAndUpdate(getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait)); + Strings new_hosts = getNewAndUpdate(tmp_hosts); ++try_number; if (new_hosts.empty()) continue; - current_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active"); + current_active_hosts = std::move(tmp_active_hosts); MutableColumns columns = output.getHeader().cloneEmptyColumns(); for (const String & host_id : new_hosts) @@ -447,7 +479,11 @@ Chunk DDLQueryStatusSource::generate() if (node_to_wait == "finished") { String status_data; - if (zookeeper->tryGet(fs::path(node_path) / "finished" / host_id, status_data)) + bool finished_exists = false; + getRetriesForDistributedDDL().ctl.retryLoop([&](){ + finished_exists = context->getZooKeeper()->tryGet(fs::path(node_path) / "finished" / host_id, status_data); + }); + if (finished_exists) status.tryDeserializeText(status_data); } else diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 8df199f0ede..19fb3fdb5a6 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace zkutil @@ -43,7 +44,7 @@ struct DDLQueryOnClusterParams BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const DDLQueryOnClusterParams & params = {}); BlockIO getDistributedDDLStatus( - const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional & hosts_to_wait = {}); + const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional & hosts_to_wait); bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context); From 429975b3455dcebf7563eebcea3cf4a7035eb685 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 30 Mar 2023 17:01:16 -0300 Subject: [PATCH 130/233] Update index.md --- docs/en/sql-reference/functions/index.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 22e79ec6623..01da5f01679 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -91,7 +91,7 @@ The command must read arguments from `STDIN` and must output the result to `STDO **Example** Creating `test_function` using XML configuration. -File test_function.xml. +File `test_function.xml` (`/etc/clickhouse-server/test_function.xml` with default path settings). ```xml @@ -108,7 +108,7 @@ File test_function.xml. ``` -Script file inside `user_scripts` folder `test_function.py`. +Script file inside `user_scripts` folder `test_function.py` (`/var/lib/clickhouse/user_scripts/test_function.py` with default path settings). ```python #!/usr/bin/python3 @@ -136,7 +136,7 @@ Result: ``` Creating `test_function_sum` manually specifying `execute_direct` to `0` using XML configuration. -File test_function.xml. +File `test_function.xml` (`/etc/clickhouse-server/test_function.xml` with default path settings). ```xml @@ -173,7 +173,7 @@ Result: ``` Creating `test_function_sum_json` with named arguments and format [JSONEachRow](../../interfaces/formats.md#jsoneachrow) using XML configuration. -File test_function.xml. +File `test_function.xml` (`/etc/clickhouse-server/test_function.xml` with default path settings). ```xml @@ -195,7 +195,7 @@ File test_function.xml. ``` -Script file inside `user_scripts` folder `test_function_sum_json.py`. +Script file inside `user_scripts` folder `test_function_sum_json.py` (`/var/lib/clickhouse/user_scripts/test_function_sum_json.py` with default path settings). ```python #!/usr/bin/python3 @@ -228,7 +228,7 @@ Result: ``` Executable user defined functions can take constant parameters configured in `command` setting (works only for user defined functions with `executable` type). -File test_function_parameter_python.xml. +File `test_function_parameter_python.xml` (`/etc/clickhouse-server/test_function_parameter_python.xml` with default path settings). ```xml @@ -244,7 +244,7 @@ File test_function_parameter_python.xml. ``` -Script file inside `user_scripts` folder `test_function_parameter_python.py`. +Script file inside `user_scripts` folder `test_function_parameter_python.py` (`/var/lib/clickhouse/user_scripts/test_function_parameter_python.py` with default path settings). ```python #!/usr/bin/python3 From 38389d878c1b166179207079d19a64d6fd866ce5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 Mar 2023 21:06:53 +0000 Subject: [PATCH 131/233] fix one more race in StorageS3 --- src/Storages/IStorageDataLake.h | 2 +- src/Storages/StorageS3.cpp | 56 +++++++++++++++++---------------- src/Storages/StorageS3.h | 6 ++-- 3 files changed, 33 insertions(+), 31 deletions(-) diff --git a/src/Storages/IStorageDataLake.h b/src/Storages/IStorageDataLake.h index 37776294491..a54544c8228 100644 --- a/src/Storages/IStorageDataLake.h +++ b/src/Storages/IStorageDataLake.h @@ -39,7 +39,7 @@ public: std::optional format_settings_) : Storage( getAdjustedConfiguration( - context_, Storage::updateConfiguration(context_, configuration_), &Poco::Logger::get("Storage" + String(name))), + context_, Storage::copyAndUpdateConfiguration(context_, configuration_), &Poco::Logger::get("Storage" + String(name))), table_id_, columns_, constraints_, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 344ac56df78..8ecdb01e93c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1016,13 +1016,15 @@ Pipe StorageS3::read( size_t max_block_size, size_t num_streams) { - bool has_wildcards = s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos + auto query_s3_configuration = copyAndUpdateConfiguration(local_context, s3_configuration); + + bool has_wildcards = + query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; + if (partition_by && has_wildcards) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - updateConfiguration(local_context, s3_configuration); - Pipes pipes; std::unordered_set column_names_set(column_names.begin(), column_names.end()); @@ -1035,7 +1037,7 @@ Pipe StorageS3::read( } std::shared_ptr iterator_wrapper = createFileIterator( - s3_configuration, + query_s3_configuration, keys, is_key_with_globs, distributed_processing, @@ -1078,11 +1080,11 @@ Pipe StorageS3::read( format_settings, columns_description, max_block_size, - s3_configuration.request_settings, + query_s3_configuration.request_settings, compression_method, - s3_configuration.client, - s3_configuration.url.bucket, - s3_configuration.url.version_id, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, iterator_wrapper, max_download_threads)); } @@ -1095,11 +1097,11 @@ Pipe StorageS3::read( SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - updateConfiguration(local_context, s3_configuration); + auto query_s3_configuration = copyAndUpdateConfiguration(local_context, s3_configuration); auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(keys.back(), compression_method); - bool has_wildcards = s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; + bool has_wildcards = query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; auto insert_query = std::dynamic_pointer_cast(query); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; @@ -1114,19 +1116,19 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr local_context, format_settings, chosen_compression_method, - s3_configuration, - s3_configuration.url.bucket, + query_s3_configuration, + query_s3_configuration.url.bucket, keys.back()); } else { if (is_key_with_globs) throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", s3_configuration.url.key); + "S3 key '{}' contains globs, so the table is in readonly mode", query_s3_configuration.url.key); bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; - if (!truncate_in_insert && S3::objectExists(*s3_configuration.client, s3_configuration.url.bucket, keys.back(), s3_configuration.url.version_id, s3_configuration.request_settings)) + if (!truncate_in_insert && S3::objectExists(*query_s3_configuration.client, query_s3_configuration.url.bucket, keys.back(), query_s3_configuration.url.version_id, query_s3_configuration.request_settings)) { if (local_context->getSettingsRef().s3_create_new_file_on_insert) { @@ -1138,7 +1140,7 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr new_key = keys[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : keys[0].substr(pos)); ++index; } - while (S3::objectExists(*s3_configuration.client, s3_configuration.url.bucket, new_key, s3_configuration.url.version_id, s3_configuration.request_settings)); + while (S3::objectExists(*query_s3_configuration.client, query_s3_configuration.url.bucket, new_key, query_s3_configuration.url.version_id, query_s3_configuration.request_settings)); keys.push_back(new_key); } else @@ -1147,7 +1149,7 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", - s3_configuration.url.bucket, + query_s3_configuration.url.bucket, keys.back()); } @@ -1157,19 +1159,19 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr local_context, format_settings, chosen_compression_method, - s3_configuration, - s3_configuration.url.bucket, + query_s3_configuration, + query_s3_configuration.url.bucket, keys.back()); } } void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { - updateConfiguration(local_context, s3_configuration); + auto query_s3_configuration = copyAndUpdateConfiguration(local_context, s3_configuration); if (is_key_with_globs) throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", s3_configuration.url.key); + "S3 key '{}' contains globs, so the table is in readonly mode", query_s3_configuration.url.key); Aws::S3::Model::Delete delkeys; @@ -1182,10 +1184,10 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ProfileEvents::increment(ProfileEvents::S3DeleteObjects); S3::DeleteObjectsRequest request; - request.SetBucket(s3_configuration.url.bucket); + request.SetBucket(query_s3_configuration.url.bucket); request.SetDelete(delkeys); - auto response = s3_configuration.client->DeleteObjects(request); + auto response = query_s3_configuration.client->DeleteObjects(request); if (!response.IsSuccess()) { const auto & err = response.GetError(); @@ -1197,7 +1199,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, } -StorageS3::Configuration StorageS3::updateConfiguration(ContextPtr local_context, const StorageS3::Configuration & configuration) +StorageS3::Configuration StorageS3::copyAndUpdateConfiguration(ContextPtr local_context, const StorageS3::Configuration & configuration) { StorageS3::Configuration new_configuration(configuration); updateConfiguration(local_context, new_configuration); @@ -1355,14 +1357,14 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context } ColumnsDescription StorageS3::getTableStructureFromData( - StorageS3::Configuration & configuration, + const StorageS3::Configuration & configuration, const std::optional & format_settings, ContextPtr ctx) { - updateConfiguration(ctx, configuration); + auto query_s3_configuration = copyAndUpdateConfiguration(ctx, configuration); return getTableStructureFromDataImpl( - configuration.format, configuration, configuration.compression_method, - configuration.url.key.find_first_of("*?{") != std::string::npos, format_settings, ctx); + query_s3_configuration.format, query_s3_configuration, query_s3_configuration.compression_method, + query_s3_configuration.url.key.find_first_of("*?{") != std::string::npos, format_settings, ctx); } ColumnsDescription StorageS3::getTableStructureFromDataImpl( diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index a34113b8bae..0bccc4a419f 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -293,13 +293,13 @@ public: static StorageS3::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); static ColumnsDescription getTableStructureFromData( - StorageS3::Configuration & configuration, + const StorageS3::Configuration & configuration, const std::optional & format_settings, ContextPtr ctx); protected: - static StorageS3::Configuration updateConfiguration(ContextPtr local_context, const Configuration & configuration); - static void updateConfiguration(ContextPtr, Configuration &); + static StorageS3::Configuration copyAndUpdateConfiguration(ContextPtr local_context, const Configuration & configuration); + static void updateConfiguration(ContextPtr local_context, Configuration & configuration); private: friend class StorageS3Cluster; From b7f411443d9c79be817202d186c35baa17f3f151 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 00:36:34 +0200 Subject: [PATCH 132/233] fix test numbers again --- ...rence => 02705_projection_and_ast_optimizations_bug.reference} | 0 ...ons_bug.sql => 02705_projection_and_ast_optimizations_bug.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25402_projection_and_ast_optimizations_bug.reference => 02705_projection_and_ast_optimizations_bug.reference} (100%) rename tests/queries/0_stateless/{25402_projection_and_ast_optimizations_bug.sql => 02705_projection_and_ast_optimizations_bug.sql} (100%) diff --git a/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.reference b/tests/queries/0_stateless/02705_projection_and_ast_optimizations_bug.reference similarity index 100% rename from tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.reference rename to tests/queries/0_stateless/02705_projection_and_ast_optimizations_bug.reference diff --git a/tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.sql b/tests/queries/0_stateless/02705_projection_and_ast_optimizations_bug.sql similarity index 100% rename from tests/queries/0_stateless/25402_projection_and_ast_optimizations_bug.sql rename to tests/queries/0_stateless/02705_projection_and_ast_optimizations_bug.sql From 98833bcc82218cf0ec60582bfa65f4392345339d Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 30 Mar 2023 16:41:29 -0600 Subject: [PATCH 133/233] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 139 +----------------- 1 file changed, 6 insertions(+), 133 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 8b8217ed5be..e3b162a8dbf 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -244,11 +244,10 @@ FROM ) WHERE view_range > 1 ORDER BY is_comments_enabled ASC, - num_views ASC + num_views ASC; ``` ```response - ┌─views─────────────┬─is_comments_enabled─┬────prob_like_dislike─┐ │ < 10.00 │ false │ 0.08224180712685371 │ │ < 100.00 │ false │ 0.06346337759167248 │ @@ -273,134 +272,10 @@ ORDER BY └───────────────────┴─────────────────────┴──────────────────────┘ 22 rows in set. Elapsed: 8.460 sec. Processed 4.56 billion rows, 77.48 GB (538.73 million rows/s., 9.16 GB/s.) - ``` Enabling comments seems to be correlated with a higher rate of engagement. - -### How does the number of videos change over time - notable events? - -```sql -SELECT - toStartOfMonth(toDateTime(upload_date)) AS month, - uniq(uploader_id) AS uploaders, - count() as num_videos, - sum(view_count) as view_count -FROM youtube -WHERE (month >= '2005-01-01') AND (month < '2021-12-01') -GROUP BY month -ORDER BY month ASC -``` - -```response -┌──────month─┬─uploaders─┬─num_videos─┬───view_count─┐ -│ 2005-04-01 │ 5 │ 6 │ 213597737 │ -│ 2005-05-01 │ 6 │ 9 │ 2944005 │ -│ 2005-06-01 │ 165 │ 351 │ 18624981 │ -│ 2005-07-01 │ 395 │ 1168 │ 94164872 │ -│ 2005-08-01 │ 1171 │ 3128 │ 124540774 │ -│ 2005-09-01 │ 2418 │ 5206 │ 475536249 │ -│ 2005-10-01 │ 6750 │ 13747 │ 737593613 │ -│ 2005-11-01 │ 13706 │ 28078 │ 1896116976 │ -│ 2005-12-01 │ 24756 │ 49885 │ 2478418930 │ -│ 2006-01-01 │ 49992 │ 100447 │ 4532656581 │ -│ 2006-02-01 │ 67882 │ 138485 │ 5677516317 │ -│ 2006-03-01 │ 103358 │ 212237 │ 8430301366 │ -│ 2006-04-01 │ 114615 │ 234174 │ 9980760440 │ -│ 2006-05-01 │ 152682 │ 332076 │ 14129117212 │ -│ 2006-06-01 │ 193962 │ 429538 │ 17014143263 │ -│ 2006-07-01 │ 234401 │ 530311 │ 18721143410 │ -│ 2006-08-01 │ 281280 │ 614128 │ 20473502342 │ -│ 2006-09-01 │ 312434 │ 679906 │ 23158422265 │ -│ 2006-10-01 │ 404873 │ 897590 │ 27357846117 │ -``` - -A spike of uploaders [around covid is noticeable](https://www.theverge.com/2020/3/27/21197642/youtube-with-me-style-videos-views-coronavirus-cook-workout-study-home-beauty). - - -### More subtitiles over time and when - -With advances in speech recognition, it’s easier than ever to create subtitles for video with youtube adding auto-captioning in late 2009 - was the jump then? - -```sql -SELECT - toStartOfMonth(upload_date) AS month, - countIf(has_subtitles) / count() AS percent_subtitles, - percent_subtitles - any(percent_subtitles) OVER (ORDER BY month ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS previous -FROM youtube -WHERE (month >= '2015-01-01') AND (month < '2021-12-02') -GROUP BY month -ORDER BY month ASC -``` - -```response -┌──────month─┬───percent_subtitles─┬────────────────previous─┐ -│ 2015-01-01 │ 0.2652653881082824 │ 0.2652653881082824 │ -│ 2015-02-01 │ 0.3147556050309162 │ 0.049490216922633834 │ -│ 2015-03-01 │ 0.32460464492371877 │ 0.009849039892802558 │ -│ 2015-04-01 │ 0.33471963051468445 │ 0.010114985590965686 │ -│ 2015-05-01 │ 0.3168087575501062 │ -0.017910872964578273 │ -│ 2015-06-01 │ 0.3162609788438222 │ -0.0005477787062839745 │ -│ 2015-07-01 │ 0.31828767677518033 │ 0.0020266979313581235 │ -│ 2015-08-01 │ 0.3045551564286859 │ -0.013732520346494415 │ -│ 2015-09-01 │ 0.311221133995152 │ 0.006665977566466086 │ -│ 2015-10-01 │ 0.30574870926812175 │ -0.005472424727030245 │ -│ 2015-11-01 │ 0.31125409712077234 │ 0.0055053878526505895 │ -│ 2015-12-01 │ 0.3190967954651779 │ 0.007842698344405541 │ -│ 2016-01-01 │ 0.32636021432496176 │ 0.007263418859783877 │ - -``` - -The data results show a spike in 2009. Apparently at that, time YouTube was removing their community captions feature, which allowed you to upload captions for other people's video. -This prompted a very successful campaign to have creators add captions to their videos for hard of hearing and deaf viewers. - - -### Top uploaders over time - -```sql -WITH uploaders AS - ( - SELECT uploader - FROM youtube - GROUP BY uploader - ORDER BY sum(view_count) DESC - LIMIT 10 - ) -SELECT - month, - uploader, - sum(view_count) AS total_views, - avg(dislike_count / like_count) AS like_to_dislike_ratio -FROM youtube -WHERE uploader IN (uploaders) -GROUP BY - toStartOfMonth(upload_date) AS month, - uploader -ORDER BY - month ASC, - total_views DESC - -1001 rows in set. Elapsed: 34.917 sec. Processed 4.58 billion rows, 69.08 GB (131.15 million rows/s., 1.98 GB/s.) -``` - -```response -┌──────month─┬─uploader───────────────────┬─total_views─┬─like_to_dislike_ratio─┐ -│ 1970-01-01 │ T-Series │ 10957099 │ 0.022784656361208206 │ -│ 1970-01-01 │ Ryan's World │ 0 │ 0.003035559410234172 │ -│ 1970-01-01 │ SET India │ 0 │ nan │ -│ 2006-09-01 │ Cocomelon - Nursery Rhymes │ 256406497 │ 0.7005566715978622 │ -│ 2007-06-01 │ Cocomelon - Nursery Rhymes │ 33641320 │ 0.7088650914344298 │ -│ 2008-02-01 │ WWE │ 43733469 │ 0.07198856488734842 │ -│ 2008-03-01 │ WWE │ 16514541 │ 0.1230603715431997 │ -│ 2008-04-01 │ WWE │ 5907295 │ 0.2089399470159618 │ -│ 2008-05-01 │ WWE │ 7779627 │ 0.09101676560436774 │ -│ 2008-06-01 │ WWE │ 7018780 │ 0.0974184753155297 │ -│ 2008-07-01 │ WWE │ 4686447 │ 0.1263845422065158 │ -│ 2008-08-01 │ WWE │ 4514312 │ 0.08384574274791441 │ -│ 2008-09-01 │ WWE │ 3717092 │ 0.07872802579349912 │ -``` - ### How do like ratio changes as views go up? ```sql @@ -421,13 +296,10 @@ GROUP BY ORDER BY view_range ASC, is_comments_enabled ASC -) - -20 rows in set. Elapsed: 9.043 sec. Processed 4.56 billion rows, 77.48 GB (503.99 million rows/s., 8.57 GB/s.) +); ``` ```response - ┌─view_range────────┬─is_comments_enabled─┬─like_ratio─┐ │ < 10.00 │ false │ 0.66 │ │ < 10.00 │ true │ 0.66 │ @@ -451,6 +323,7 @@ ORDER BY │ < 10.00 billion │ true │ 19.5 │ └───────────────────┴─────────────────────┴────────────┘ +20 rows in set. Elapsed: 63.664 sec. Processed 4.56 billion rows, 113.93 GB (71.59 million rows/s., 1.79 GB/s.) ``` ### How are views distributed? @@ -468,9 +341,7 @@ FROM ) ARRAY JOIN quantiles, - labels - -12 rows in set. Elapsed: 1.864 sec. Processed 4.56 billion rows, 36.46 GB (2.45 billion rows/s., 19.56 GB/s.) + labels; ``` ```response @@ -488,4 +359,6 @@ ARRAY JOIN │ 20th │ 16 │ │ 10th │ 6 │ └────────────┴─────────┘ + +12 rows in set. Elapsed: 1.864 sec. Processed 4.56 billion rows, 36.46 GB (2.45 billion rows/s., 19.56 GB/s.) ``` \ No newline at end of file From 81375431d99ea81de4f4e58931e75d97e590fada Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 00:42:16 +0200 Subject: [PATCH 134/233] fix some tests --- tests/integration/helpers/client.py | 13 ++++++- tests/integration/test_drop_replica/test.py | 35 ++++++++++++------- .../test_system_replicated_fetches/test.py | 11 +++--- ...tem_parts_race_condition_drop_zookeeper.sh | 2 +- 4 files changed, 42 insertions(+), 19 deletions(-) diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index ab1cc65e9a9..66e0f0bc907 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -202,6 +202,15 @@ class CommandRequest: self.timer = Timer(timeout, kill_process) self.timer.start() + def remove_trash_from_stderr(self, stderr): + if not stderr: + return stderr + lines = stderr.split("\n") + lines = [ + x for x in lines if ("completion_queue" not in x and "Kick failed" not in x) + ] + return "\n".join(lines) + def get_answer(self): self.process.wait(timeout=DEFAULT_QUERY_TIMEOUT) self.stdout_file.seek(0) @@ -218,7 +227,9 @@ class CommandRequest: logging.debug(f"Timed out. Last stdout:{stdout}, stderr:{stderr}") raise QueryTimeoutExceedException("Client timed out!") - if (self.process.returncode != 0 or stderr) and not self.ignore_error: + if ( + self.process.returncode != 0 or self.remove_trash_from_stderr(stderr) + ) and not self.ignore_error: raise QueryRuntimeException( "Client failed! Return code: {}, stderr: {}".format( self.process.returncode, stderr diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index f16a5b729f6..e87edb0a578 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -90,6 +90,11 @@ def start_cluster(): cluster.shutdown() +def check_exists(zk, path): + zk.sync(path) + return zk.exists(path) + + def test_drop_replica(start_cluster): node_1_1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" @@ -158,10 +163,11 @@ def test_drop_replica(start_cluster): ) node_1_3.query("SYSTEM DROP REPLICA 'node_1_1'") - exists_replica_1_1 = zk.exists( + exists_replica_1_1 = check_exists( + zk, "/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format( shard=1, replica="node_1_1" - ) + ), ) assert exists_replica_1_1 != None @@ -171,26 +177,29 @@ def test_drop_replica(start_cluster): shard=1 ) ) - exists_replica_1_1 = zk.exists( + exists_replica_1_1 = check_exists( + zk, "/clickhouse/tables/test2/{shard}/replicated/test_table/replicas/{replica}".format( shard=1, replica="node_1_1" - ) + ), ) assert exists_replica_1_1 == None node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table") - exists_replica_1_1 = zk.exists( + exists_replica_1_1 = check_exists( + zk, "/clickhouse/tables/test/{shard}/replicated/test_table/replicas/{replica}".format( shard=1, replica="node_1_1" - ) + ), ) assert exists_replica_1_1 == None node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1") - exists_replica_1_1 = zk.exists( + exists_replica_1_1 = check_exists( + zk, "/clickhouse/tables/test1/{shard}/replicated/test_table/replicas/{replica}".format( shard=1, replica="node_1_1" - ) + ), ) assert exists_replica_1_1 == None @@ -199,17 +208,19 @@ def test_drop_replica(start_cluster): shard=1 ) ) - exists_replica_1_1 = zk.exists( + exists_replica_1_1 = check_exists( + zk, "/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format( shard=1, replica="node_1_1" - ) + ), ) assert exists_replica_1_1 == None node_1_2.query("SYSTEM DROP REPLICA 'node_1_1'") - exists_replica_1_1 = zk.exists( + exists_replica_1_1 = check_exists( + zk, "/clickhouse/tables/test4/{shard}/replicated/test_table/replicas/{replica}".format( shard=1, replica="node_1_1" - ) + ), ) assert exists_replica_1_1 == None diff --git a/tests/integration/test_system_replicated_fetches/test.py b/tests/integration/test_system_replicated_fetches/test.py index 2b516ebf69b..bcf9b38d3d4 100644 --- a/tests/integration/test_system_replicated_fetches/test.py +++ b/tests/integration/test_system_replicated_fetches/test.py @@ -90,11 +90,12 @@ def test_system_replicated_fetches(started_cluster): ) for elem in fetches_result: - assert ( - elem["bytes_read_compressed"] <= elem["total_size_bytes_compressed"] - ), "Bytes read ({}) more than total bytes ({}). It's a bug".format( - elem["bytes_read_compressed"], elem["total_size_bytes_compressed"] - ) + # FIXME https://github.com/ClickHouse/ClickHouse/issues/45435 + # assert ( + # elem["bytes_read_compressed"] <= elem["total_size_bytes_compressed"] + # ), "Bytes read ({}) more than total bytes ({}). It's a bug".format( + # elem["bytes_read_compressed"], elem["total_size_bytes_compressed"] + # ) assert ( 0.0 <= elem["progress"] <= 1.0 ), "Progress shouldn't less than 0 and bigger than 1, got {}".format( diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index f143c97bdf4..ff6a4b8fc42 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel, no-upgrade-check +# Tags: race, zookeeper, no-parallel, no-upgrade-check, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 2850d2822d5931f62ea93d241a2684c4d51586a2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 01:38:23 +0200 Subject: [PATCH 135/233] more consistent workflows --- .github/workflows/master.yml | 763 +++++++++++++++++++++++++++++++++-- 1 file changed, 740 insertions(+), 23 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7c5e477ab60..fc2c4def2ea 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1131,7 +1131,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 + RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1167,6 +1167,114 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated2: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseS3_0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports @@ -1190,7 +1298,7 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseS3: + FunctionalStatelessTestReleaseS3_1: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: @@ -1202,6 +1310,8 @@ jobs: CHECK_NAME=Stateless tests (release, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1271,7 +1381,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 + RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1307,7 +1417,79 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (asan) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan3: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (asan) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1343,7 +1525,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1379,7 +1561,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1415,7 +1597,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1438,7 +1620,79 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestUBsan: + FunctionalStatelessTestTsan3: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=5 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan4: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=5 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestUBsan0: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: @@ -1450,6 +1704,44 @@ jobs: CHECK_NAME=Stateless tests (ubsan) REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestUBsan1: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1485,7 +1777,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1521,7 +1813,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1557,7 +1849,115 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan3: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (msan) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan4: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (msan) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan5: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (msan) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=5 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1593,7 +1993,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1629,7 +2029,7 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -1665,7 +2065,79 @@ jobs: REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=5 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug3: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=5 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - 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 ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug4: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2116,7 +2588,7 @@ jobs: CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2151,7 +2623,7 @@ jobs: CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2186,7 +2658,112 @@ jobs: CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan3: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan4: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan5: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=5 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2221,7 +2798,7 @@ jobs: CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2256,7 +2833,7 @@ jobs: CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2291,7 +2868,7 @@ jobs: CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2326,7 +2903,77 @@ jobs: CHECK_NAME=Integration tests (tsan) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan4: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (tsan) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan5: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (tsan) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=5 + RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2361,7 +3008,7 @@ jobs: CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 + RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports uses: actions/download-artifact@v3 @@ -2396,7 +3043,77 @@ jobs: CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease2: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports uses: actions/download-artifact@v3 From 64b1f74a6074a1a0b458a5cd011b6de98a5688d5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 01:39:49 +0200 Subject: [PATCH 136/233] fix --- src/Interpreters/executeDDLQueryOnCluster.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 108d9bccf33..a93cc2f06ba 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -445,7 +445,8 @@ Chunk DDLQueryStatusSource::generate() Strings tmp_hosts; Strings tmp_active_hosts; - getRetriesForDistributedDDL().ctl.retryLoop([&](){ + getRetriesForDistributedDDL().ctl.retryLoop([&]() + { auto zookeeper = context->getZooKeeper(); node_exists = zookeeper->exists(node_path); tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait); @@ -480,7 +481,8 @@ Chunk DDLQueryStatusSource::generate() { String status_data; bool finished_exists = false; - getRetriesForDistributedDDL().ctl.retryLoop([&](){ + getRetriesForDistributedDDL().ctl.retryLoop([&]() + { finished_exists = context->getZooKeeper()->tryGet(fs::path(node_path) / "finished" / host_id, status_data); }); if (finished_exists) From 356716f0a1d510d7331e9cb7cbba3eed763050ef Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 03:26:16 +0300 Subject: [PATCH 137/233] Update master.yml --- .github/workflows/master.yml | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index fc2c4def2ea..a0b2e9d60fa 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3833,23 +3833,36 @@ jobs: - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 + - FunctionalStatelessTestDebug3 + - FunctionalStatelessTestDebug4 - FunctionalStatelessTestRelease - FunctionalStatelessTestReleaseDatabaseOrdinary - FunctionalStatelessTestReleaseDatabaseReplicated0 - FunctionalStatelessTestReleaseDatabaseReplicated1 + - FunctionalStatelessTestReleaseDatabaseReplicated2 + - FunctionalStatelessTestReleaseDatabaseReplicated3 - FunctionalStatelessTestAarch64 - FunctionalStatelessTestAsan0 - FunctionalStatelessTestAsan1 + - FunctionalStatelessTestAsan2 + - FunctionalStatelessTestAsan3 - FunctionalStatelessTestTsan0 - FunctionalStatelessTestTsan1 - FunctionalStatelessTestTsan2 + - FunctionalStatelessTestTsan3 + - FunctionalStatelessTestTsan4 - FunctionalStatelessTestMsan0 - FunctionalStatelessTestMsan1 - FunctionalStatelessTestMsan2 - - FunctionalStatelessTestUBsan + - FunctionalStatelessTestMsan3 + - FunctionalStatelessTestMsan4 + - FunctionalStatelessTestMsan5 + - FunctionalStatelessTestUBsan0 + - FunctionalStatelessTestUBsan1 - FunctionalStatefulTestDebug - FunctionalStatefulTestRelease - - FunctionalStatelessTestReleaseS3 + - FunctionalStatelessTestReleaseS3_0 + - FunctionalStatelessTestReleaseS3_1 - FunctionalStatefulTestAarch64 - FunctionalStatefulTestAsan - FunctionalStatefulTestTsan @@ -3863,12 +3876,19 @@ jobs: - IntegrationTestsAsan0 - IntegrationTestsAsan1 - IntegrationTestsAsan2 + - IntegrationTestsAsan3 + - IntegrationTestsAsan4 + - IntegrationTestsAsan5 - IntegrationTestsRelease0 - IntegrationTestsRelease1 + - IntegrationTestsRelease2 + - IntegrationTestsRelease3 - IntegrationTestsTsan0 - IntegrationTestsTsan1 - IntegrationTestsTsan2 - IntegrationTestsTsan3 + - IntegrationTestsTsan4 + - IntegrationTestsTsan5 - PerformanceComparisonX86-0 - PerformanceComparisonX86-1 - PerformanceComparisonX86-2 From e16b43463114d076702c3cedd432c6066b8aeaa3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 Mar 2023 06:03:05 +0200 Subject: [PATCH 138/233] Fix compiling examples without Hive Skip compiling comma_separated_streams since it requires Hive. Signed-off-by: Azat Khuzhin --- src/Processors/examples/CMakeLists.txt | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 72c3a16d32f..5d43a0d7d08 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,2 +1,4 @@ -clickhouse_add_executable (comma_separated_streams comma_separated_streams.cpp) -target_link_libraries (comma_separated_streams PRIVATE dbms) +if (TARGET ch_contrib::hivemetastore) + clickhouse_add_executable (comma_separated_streams comma_separated_streams.cpp) + target_link_libraries (comma_separated_streams PRIVATE dbms) +endif() From 0f4c8144a65facf5e79502eeb9a2f2efe64fcb8e Mon Sep 17 00:00:00 2001 From: Vadym Chekan Date: Thu, 30 Mar 2023 22:46:18 -0700 Subject: [PATCH 139/233] In messages, put values into quotes Configuration values, such as disk names, backup engine names, etc, may give error message unintended sense, for example, if trying to backup to `disk` instead of `Disk`, the error message will be "Not found backup engine disk", which can be interpreted as "disk of backup engine not found". It might be not clear that the word "disk" comes from the query and is not part of the error message. --- src/Backups/BackupFactory.cpp | 4 ++-- src/Backups/registerBackupEnginesFileAndDisk.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupFactory.cpp b/src/Backups/BackupFactory.cpp index 7c870737b1d..898ac7bc490 100644 --- a/src/Backups/BackupFactory.cpp +++ b/src/Backups/BackupFactory.cpp @@ -20,14 +20,14 @@ BackupMutablePtr BackupFactory::createBackup(const CreateParams & params) const const String & engine_name = params.backup_info.backup_engine_name; auto it = creators.find(engine_name); if (it == creators.end()) - throw Exception(ErrorCodes::BACKUP_ENGINE_NOT_FOUND, "Not found backup engine {}", engine_name); + throw Exception(ErrorCodes::BACKUP_ENGINE_NOT_FOUND, "Not found backup engine '{}'", engine_name); return (it->second)(params); } void BackupFactory::registerBackupEngine(const String & engine_name, const CreatorFn & creator_fn) { if (creators.contains(engine_name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup engine {} was registered twice", engine_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup engine '{}' was registered twice", engine_name); creators[engine_name] = creator_fn; } diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 020da13d6e1..51b14fbc1d8 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -41,7 +41,7 @@ namespace key = "backups.allowed_disk[" + std::to_string(++counter) + "]"; if (!config.has(key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Disk {} is not allowed for backups, see the 'backups.allowed_disk' configuration parameter", quoteString(disk_name)); + "Disk '{}' is not allowed for backups, see the 'backups.allowed_disk' configuration parameter", quoteString(disk_name)); } } @@ -54,7 +54,7 @@ namespace bool path_ok = path.empty() || (path.is_relative() && (*path.begin() != "..")); if (!path_ok) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} to backup must be inside the specified disk {}", + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' to backup must be inside the specified disk '{}'", quoteString(path.c_str()), quoteString(disk_name)); } From f31f11dd67d8dbce08ba98c5ae85a61e9fb7ddaa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 31 Mar 2023 09:12:41 +0000 Subject: [PATCH 140/233] Disable env credentials for stateless tests --- tests/config/config.d/disable_s3_env_credentials.xml | 5 +++++ tests/config/install.sh | 1 + 2 files changed, 6 insertions(+) create mode 100644 tests/config/config.d/disable_s3_env_credentials.xml diff --git a/tests/config/config.d/disable_s3_env_credentials.xml b/tests/config/config.d/disable_s3_env_credentials.xml new file mode 100644 index 00000000000..24a7e0f2f35 --- /dev/null +++ b/tests/config/config.d/disable_s3_env_credentials.xml @@ -0,0 +1,5 @@ + + + 0 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index a6391f6f43f..44eab0e4db0 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -55,6 +55,7 @@ ln -sf $SRC_PATH/config.d/custom_disks_base_path.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/reverse_dns_query_function.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/compressed_marks_and_index.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/disable_s3_env_credentials.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] From 42abb843252bbb60c60f43006864a28cd47f18bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 Mar 2023 12:16:31 +0300 Subject: [PATCH 141/233] Fix 01710_projection_optimize_materialize flakiness Fails with different index granularity CI: https://s3.amazonaws.com/clickhouse-test-reports/48242/672dbf7cd894be6f5c0ac685d493371f2996229d/stateless_tests__asan__[3/4].html Signed-off-by: Azat Khuzhin --- .../0_stateless/01710_projection_optimize_materialize.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01710_projection_optimize_materialize.sql b/tests/queries/0_stateless/01710_projection_optimize_materialize.sql index 92d3ead828c..e704c3e5610 100644 --- a/tests/queries/0_stateless/01710_projection_optimize_materialize.sql +++ b/tests/queries/0_stateless/01710_projection_optimize_materialize.sql @@ -1,3 +1,4 @@ +-- Tags: no-random-merge-tree-settings drop table if exists z; create table z (pk Int64, d Date, id UInt64, c UInt64) Engine MergeTree partition by d order by pk settings ratio_of_defaults_for_sparse_serialization = 1.0; From 792979a5275d5f8a112f6b59e320ded050fb7e03 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 12:24:47 +0300 Subject: [PATCH 142/233] Revert "Randomize JIT settings in tests" --- tests/clickhouse-test | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fa88bc19efd..a355c2f8e73 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -558,9 +558,6 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), - "min_count_to_compile_expression": lambda: random.choice([0, 3]), - "min_count_to_compile_aggregate_expression": lambda: random.choice([0, 3]), - "min_count_to_compile_sort_description": lambda: random.choice([0, 3]), } @staticmethod From bac7def719603c9fae013113694cc0b7fb69244a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 Mar 2023 12:26:45 +0300 Subject: [PATCH 143/233] Fix UB (signed integer overflow) in StorageMergeTree::backupData() UBsan report: /build/src/Storages/StorageMergeTree.cpp:2088:80: runtime error: signed integer overflow: 9223372036854775807 + 1 cannot be represented in type 'long' 0 0x30c2ae7b in DB::StorageMergeTree::backupData(DB::BackupEntriesCollector&, std::__1::basic_string, std::__1::allocator> const&, std::__1::optional, 7ul, std::__1::allocator>>> const&) build_docker/./src/Storages/StorageMergeTree.cpp:2088:80 1 0x2dd641b5 in DB::BackupEntriesCollector::makeBackupEntriesForTableData(DB::QualifiedTableName const&) build_docker/./src/Backups/BackupEntriesCollector.cpp:703:18 2 0x2dd5c2a7 in DB::BackupEntriesCollector::makeBackupEntriesForTablesData() build_docker/./src/Backups/BackupEntriesCollector.cpp:676:9 3 0x2dd5848d in DB::BackupEntriesCollector::run() build_docker/./src/Backups/BackupEntriesCollector.cpp:119:5 4 0x2dd84da3 in DB::BackupsWorker::doBackup(std::__1::shared_ptr const&, std::__1::basic_string, std::__1::allocator> const&, std::__1::basic_string, std::__1::allocator> const&, DB::BackupInfo const&, DB::BackupSettings, std::__1::shared_ptr, std::__1::shared_ptr const&, std::__1::shared_ptr, bool) build_docker/./src/Backups/BackupsWorker.cpp:359:59 5 0x2dd82405 in DB::BackupsWorker::startMakingBackup(std::__1::shared_ptr const&, std::__1::shared_ptr const&) build_docker/./src/Backups/BackupsWorker.cpp:248:13 6 0x2dd81d0a in DB::BackupsWorker::start(std::__1::shared_ptr const&, std::__1::shared_ptr) build_docker/./src/Backups/BackupsWorker.cpp:179:16 7 0x2f4d0d5f in DB::InterpreterBackupQuery::execute() build_docker/./src/Interpreters/InterpreterBackupQuery.cpp:39:30 SUMMARY: UndefinedBehaviorSanitizer: undefined-behavior /build/src/Storages/StorageMergeTree.cpp:2088:80 in Signed-off-by: Azat Khuzhin --- src/Storages/StorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index abab2b2dc68..71a826fbc22 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2082,10 +2082,10 @@ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collec Int64 min_data_version = std::numeric_limits::max(); for (const auto & data_part : data_parts) - min_data_version = std::min(min_data_version, data_part->info.getDataVersion()); + min_data_version = std::min(min_data_version, data_part->info.getDataVersion() + 1); backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, local_context)); - backup_entries_collector.addBackupEntries(backupMutations(min_data_version + 1, data_path_in_backup)); + backup_entries_collector.addBackupEntries(backupMutations(min_data_version, data_path_in_backup)); } From 8d91a9f4c2c83cf0c82442f0ad309a1b0f1c1f2b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 31 Mar 2023 09:36:03 +0000 Subject: [PATCH 144/233] Sort descendingly by elapsed time --- src/Interpreters/InterpreterShowProcesslistQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/src/Interpreters/InterpreterShowProcesslistQuery.cpp index 1522af4bcbe..f9241368a8f 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -12,7 +12,7 @@ namespace DB BlockIO InterpreterShowProcesslistQuery::execute() { - return executeQuery("SELECT * FROM system.processes ORDER BY query_id", getContext(), true); + return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", getContext(), true); } } From 9806a831c096724d23faba4c1af1fbaa1d41121a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 31 Mar 2023 09:41:59 +0000 Subject: [PATCH 145/233] Small test refactoring --- .../0_stateless/25402_show_columns.reference | 22 ++++++------ .../0_stateless/25402_show_columns.sql | 34 ++++++------------- 2 files changed, 22 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/25402_show_columns.reference b/tests/queries/0_stateless/25402_show_columns.reference index 98438bd6595..da967d59cda 100644 --- a/tests/queries/0_stateless/25402_show_columns.reference +++ b/tests/queries/0_stateless/25402_show_columns.reference @@ -1,38 +1,38 @@ int32 Nullable(Int32) 1 \N str String 0 SOR \N uint64 UInt64 0 PRI SOR \N ---- +--- EXTENDED int32 Nullable(Int32) 1 \N str String 0 SOR \N uint64 UInt64 0 PRI SOR \N ---- +--- FULL int32 Nullable(Int32) 1 \N \N example comment str String 0 SOR \N \N uint64 UInt64 0 PRI SOR \N \N ---- +--- LIKE int32 Nullable(Int32) 1 \N uint64 UInt64 0 PRI SOR \N ---- +--- NOT LIKE str String 0 SOR \N ---- +--- ILIKE int32 Nullable(Int32) 1 \N uint64 UInt64 0 PRI SOR \N ---- +--- NOT ILIKE str String 0 SOR \N ---- +--- WHERE int32 Nullable(Int32) 1 \N uint64 UInt64 0 PRI SOR \N ---- +--- LIMIT int32 Nullable(Int32) 1 \N ---- +--- Original table int32 Nullable(Int32) 1 \N str String 0 SOR \N uint64 UInt64 0 PRI SOR \N ---- +--- Equally named table in other database int32 Int32 0 \N str String 0 \N uint64 UInt64 0 PRI SOR \N ---- +--- Short form int32 Int32 0 \N str String 0 \N uint64 UInt64 0 PRI SOR \N diff --git a/tests/queries/0_stateless/25402_show_columns.sql b/tests/queries/0_stateless/25402_show_columns.sql index 28ac54bd193..ce206fedee4 100644 --- a/tests/queries/0_stateless/25402_show_columns.sql +++ b/tests/queries/0_stateless/25402_show_columns.sql @@ -17,41 +17,30 @@ ORDER BY (uint64, str); SHOW COLUMNS FROM tab; -SELECT '---'; - +SELECT '--- EXTENDED'; SHOW EXTENDED COLUMNS FROM tab; -SELECT '---'; - +SELECT '--- FULL'; SHOW FULL COLUMNS FROM tab; -SELECT '---'; - +SELECT '--- LIKE'; SHOW COLUMNS FROM tab LIKE '%int%'; -SELECT '---'; - +SELECT '--- NOT LIKE'; SHOW COLUMNS FROM tab NOT LIKE '%int%'; -SELECT '---'; - +SELECT '--- ILIKE'; SHOW COLUMNS FROM tab ILIKE '%INT%'; -SELECT '---'; - +SELECT '--- NOT ILIKE'; SHOW COLUMNS FROM tab NOT ILIKE '%INT%'; -SELECT '---'; - +SELECT '--- WHERE'; SHOW COLUMNS FROM tab WHERE field LIKE '%int%'; -SELECT '---'; - +SELECT '--- LIMIT'; SHOW COLUMNS FROM tab LIMIT 1; -SELECT '---'; - - -- Create a table in a different database. Intentionally useing the same table/column names as above so -- we notice if something is buggy in the implementation of SHOW COLUMNS. DROP DATABASE IF EXISTS database_123456789abcde; @@ -67,14 +56,13 @@ CREATE TABLE database_123456789abcde.tab ENGINE = MergeTree ORDER BY uint64; +SELECT '--- Original table'; SHOW COLUMNS FROM tab; -SELECT '---'; - +SELECT '--- Equally named table in other database'; SHOW COLUMNS FROM tab FROM database_123456789abcde; -SELECT '---'; - +SELECT '--- Short form'; SHOW COLUMNS FROM database_123456789abcde.tab; DROP DATABASE database_123456789abcde; From dd79bf0a03753bebc9d22af11771cce6830d8015 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 31 Mar 2023 12:20:50 +0200 Subject: [PATCH 146/233] Update autogenerated version to 23.4.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++--- .../StorageSystemContributors.generated.cpp | 35 +++++++++++++++++++ 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index b52b2eda992..9bb148c12a9 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54472) +SET(VERSION_REVISION 54473) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 3) +SET(VERSION_MINOR 4) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 52bf836e03a6ba7cf2d654eaaf73231701abc3a2) -SET(VERSION_DESCRIBE v23.3.1.2537-testing) -SET(VERSION_STRING 23.3.1.2537) +SET(VERSION_GITHASH 46e85357ce2da2a99f56ee83a079e892d7ec3726) +SET(VERSION_DESCRIBE v23.4.1.1-testing) +SET(VERSION_STRING 23.4.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index ca19687918c..a6704144fde 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -35,6 +35,7 @@ const char * auto_contributors[] { "Aleksei Filatov", "Aleksei Levushkin", "Aleksei Semiglazov", + "Aleksei Tikhomirov", "Aleksey", "Aleksey Akulovich", "Alex", @@ -84,6 +85,7 @@ const char * auto_contributors[] { "Alexey Gusev", "Alexey Ilyukhov", "Alexey Ivanov", + "Alexey Korepanov", "Alexey Milovidov", "Alexey Perevyshin", "Alexey Tronov", @@ -135,6 +137,7 @@ const char * auto_contributors[] { "Andrii R", "Andy Liang", "Andy Yang", + "AndyB", "Anish Bhanwala", "Anmol Arora", "Anna", @@ -155,6 +158,7 @@ const char * auto_contributors[] { "Anton Yuzhaninov", "Anton Zhabolenko", "Antonio Andelic", + "Antonio Bonuccelli", "Ariel Robaldo", "Arsen Hakobyan", "Arslan G", @@ -227,6 +231,7 @@ const char * auto_contributors[] { "Christoph Wurm", "Chun-Sheng, Li", "Ciprian Hacman", + "Clayton McClure", "Clement Rodriguez", "ClickHouse Admin", "Clément Rodriguez", @@ -256,6 +261,7 @@ const char * auto_contributors[] { "Dario", "DarkWanderer", "Darío", + "Dave Lahn", "Denis Burlaka", "Denis Glazachev", "Denis Krivak", @@ -391,6 +397,7 @@ const char * auto_contributors[] { "HeenaBansal2009", "Hiroaki Nakamura", "Hongbin", + "Hosun Lee", "HuFuwang", "Hui Wang", "ILya Limarenko", @@ -457,12 +464,15 @@ const char * auto_contributors[] { "Jiebin Sun", "Joanna Hulboj", "Jochen Schalanda", + "Joey", + "Johannes Visintini", "John", "John Hummel", "John Skopis", "Jonatas Freitas", "Jonathan-Ackerman", "Jordi Villar", + "Joris Giovannangeli", "Jose", "Josh Taylor", "João Figueiredo", @@ -481,6 +491,7 @@ const char * auto_contributors[] { "Kevin Chiang", "Kevin Michel", "Kevin Zhang", + "KevinyhZou", "KinderRiven", "Kiran", "Kirill Danshin", @@ -525,6 +536,7 @@ const char * auto_contributors[] { "Li Yin", "Liu Cong", "LiuCong", + "LiuNeng", "LiuYangkuan", "Lloyd-Pottiger", "Lopatin Konstantin", @@ -544,6 +556,7 @@ const char * auto_contributors[] { "Maksim Buren", "Maksim Fedotov", "Maksim Kita", + "Maksym Sobolyev", "Mallik Hassan", "Malte", "Manuel de la Peña", @@ -625,6 +638,7 @@ const char * auto_contributors[] { "Mikhail Salosin", "Mikhail Surin", "Mikhail f. Shiryaev", + "MikhailBurdukov", "MikuSugar", "Milad Arabi", "Mingliang Pan", @@ -694,6 +708,7 @@ const char * auto_contributors[] { "OmarBazaraa", "OnePiece", "Onehr7", + "Ongkong", "Orivej Desh", "Orkhan Zeynalli", "Oskar Wojciski", @@ -701,6 +716,7 @@ const char * auto_contributors[] { "PHO", "Pablo Alegre", "Pablo Marcos", + "Palash Goel", "Paramtamtam", "Patrick Zippenfenig", "Paul Loyd", @@ -795,6 +811,7 @@ const char * auto_contributors[] { "Sergei Bocharov", "Sergei Semin", "Sergei Shtykov", + "Sergei Solomatov", "Sergei Trifonov", "Sergei Tsetlin (rekub)", "Sergey Demurin", @@ -844,6 +861,7 @@ const char * auto_contributors[] { "Stupnikov Andrey", "SuperBot", "SuperDJY", + "SupunKavinda", "Suzy Wang", "SuzyWangIBMer", "Sébastien", @@ -865,6 +883,7 @@ const char * auto_contributors[] { "The-Alchemist", "Thom O'Connor", "Thomas Berdy", + "Thomas Casteleyn", "Tian Xinhui", "Tiaonmmn", "Tigran Khudaverdyan", @@ -996,6 +1015,7 @@ const char * auto_contributors[] { "Zhipeng", "Zijie Lu", "Zoran Pandovski", + "[데이터플랫폼팀] 이호선", "a.palagashvili", "aaapetrenko", "abdrakhmanov", @@ -1011,6 +1031,7 @@ const char * auto_contributors[] { "akuzm", "alekseik1", "alesapin", + "alex filatov", "alex-zaitsev", "alex.lvxin", "alexX512", @@ -1035,12 +1056,14 @@ const char * auto_contributors[] { "anton", "ap11", "aprudaev", + "artem-yadr", "artpaul", "asiana21", "atereh", "attack204", "avasiliev", "avogar", + "avoiderboi", "avsharapov", "awakeljw", "awesomeleo", @@ -1079,6 +1102,7 @@ const char * auto_contributors[] { "chou.fan", "christophe.kalenzaga", "clarkcaoliu", + "clickhouse-adrianfraguela", "clickhouse-robot-curie", "cms", "cmsxbc", @@ -1111,6 +1135,7 @@ const char * auto_contributors[] { "dmitriy", "dmitry kuzmin", "dongyifeng", + "ducle.canh", "eaxdev", "eejoin", "egatov", @@ -1123,6 +1148,8 @@ const char * auto_contributors[] { "erikbaan", "ermaotech", "evtan", + "exX512", + "exmy", "exprmntr", "ezhaka", "f1yegor", @@ -1152,6 +1179,7 @@ const char * auto_contributors[] { "fuwhu", "fuzhe1989", "fuzzERot", + "fyu", "g-arslan", "ggerogery", "giordyb", @@ -1178,9 +1206,11 @@ const char * auto_contributors[] { "hhell", "homeward", "hotid", + "houbaron", "huangzhaowei", "hustnn", "huzhichengdd", + "iammagicc", "ianton-ru", "ice1x", "idfer", @@ -1212,6 +1242,7 @@ const char * auto_contributors[] { "jinjunzh", "jkuklis", "jthmath", + "jun won", "jus1096", "jyz0309", "karnevil13", @@ -1223,6 +1254,7 @@ const char * auto_contributors[] { "kigerzhang", "kirillikoff", "kmeaw", + "kolechenkov", "koloshmet", "kolsys", "konnectr", @@ -1252,6 +1284,7 @@ const char * auto_contributors[] { "liangqian", "libenwang", "lichengxiang", + "liding1992", "linceyou", "lincion", "lingo-xp", @@ -1330,6 +1363,7 @@ const char * auto_contributors[] { "nauta", "nautaa", "ndchikin", + "nellicus", "neng.liu", "never lee", "ni1l", @@ -1545,6 +1579,7 @@ const char * auto_contributors[] { "Дмитрий Канатников", "Иванов Евгений", "Илья Исаев", + "Коренберг Марк", "Коренберг ☢️ Марк", "Павел Литвиненко", "Смитюх Вячеслав", From 0c9d7f73ce1a6cd1fad8ce87e2c82cd6c118b064 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 31 Mar 2023 10:35:58 +0000 Subject: [PATCH 147/233] Update version_date.tsv and changelogs after v23.3.1.2823-lts --- SECURITY.md | 5 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.3.1.2823-lts.md | 545 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 552 insertions(+), 5 deletions(-) create mode 100644 docs/changelogs/v23.3.1.2823-lts.md diff --git a/SECURITY.md b/SECURITY.md index 7c6648c70eb..566a1820834 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.3 | ✔️ | | 23.2 | ✔️ | | 23.1 | ✔️ | -| 22.12 | ✔️ | +| 22.12 | ❌ | | 22.11 | ❌ | | 22.10 | ❌ | | 22.9 | ❌ | @@ -24,7 +25,7 @@ The following versions of ClickHouse server are currently being supported with s | 22.6 | ❌ | | 22.5 | ❌ | | 22.4 | ❌ | -| 22.3 | ✔️ | +| 22.3 | ❌ | | 22.2 | ❌ | | 22.1 | ❌ | | 21.* | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 6496a2b2a12..37b58758b9e 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.2.4.12" +ARG VERSION="23.3.1.2823" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index f4ca498a7e7..822aa752655 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.2.4.12" +ARG VERSION="23.3.1.2823" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 13b3ebdb01c..ec7e164e51f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.2.4.12" +ARG VERSION="23.3.1.2823" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.3.1.2823-lts.md b/docs/changelogs/v23.3.1.2823-lts.md new file mode 100644 index 00000000000..0c9be3601da --- /dev/null +++ b/docs/changelogs/v23.3.1.2823-lts.md @@ -0,0 +1,545 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.1.2823-lts (46e85357ce2) FIXME as compared to v23.2.1.2537-stable (52bf836e03a) + +#### Backward Incompatible Change +* Relax symbols that are allowed in URL authority in *domain*RFC()/netloc(). [#46841](https://github.com/ClickHouse/ClickHouse/pull/46841) ([Azat Khuzhin](https://github.com/azat)). +* Prohibit create tables based on KafkaEngine with DEFAULT/EPHEMERAL/ALIAS/MATERIALIZED statements for columns. [#47138](https://github.com/ClickHouse/ClickHouse/pull/47138) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* An "asynchronous connection drain" feature is removed. Related settings and metrics are removed as well. It was an internal feature, so the removal should not affect users who had never heard about that feature. [#47486](https://github.com/ClickHouse/ClickHouse/pull/47486) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support 256-bit Decimal data type (more than 38 digits) in `arraySum`/`Min`/`Max`/`Avg`/`Product`, `arrayCumSum`/`CumSumNonNegative`, `arrayDifference`, array construction, IN operator, query parameters, `groupArrayMovingSum`, statistical functions, `min`/`max`/`any`/`argMin`/`argMax`, PostgreSQL wire protocol, MySQL table engine and function, `sumMap`, `mapAdd`, `mapSubtract`, `arrayIntersect`. Add support for big integers in `arrayIntersect`. Statistical aggregate functions involving moments (such as `corr` or various `TTest`s) will use `Float64` as their internal representation (they were using `Decimal128` before this change, but it was pointless), and these functions can return `nan` instead of `inf` in case of infinite variance. Some functions were allowed on `Decimal256` data types but returned `Decimal128` in previous versions - now it is fixed. This closes [#47569](https://github.com/ClickHouse/ClickHouse/issues/47569). This closes [#44864](https://github.com/ClickHouse/ClickHouse/issues/44864). This closes [#28335](https://github.com/ClickHouse/ClickHouse/issues/28335). [#47594](https://github.com/ClickHouse/ClickHouse/pull/47594) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make backup_threads/restore_threads server settings. [#47881](https://github.com/ClickHouse/ClickHouse/pull/47881) ([Azat Khuzhin](https://github.com/azat)). +* Fix the isIPv6String function which could have outputted a false positive result in the case of an incorrect IPv6 address. For example `1234::1234:` was considered a valid IPv6 address. [#47895](https://github.com/ClickHouse/ClickHouse/pull/47895) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### New Feature +* Add new mode for splitting the work on replicas using settings `parallel_replicas_custom_key` and `parallel_replicas_custom_key_filter_type`. If the cluster consists of a single shard with multiple replicas, up to `max_parallel_replicas` will be randomly picked and turned into shards. For each shard, a corresponding filter is added to the query on the initiator before being sent to the shard. If the cluster consists of multiple shards, it will behave the same as `sample_key` but with the possibility to define an arbitrary key. [#45108](https://github.com/ClickHouse/ClickHouse/pull/45108) ([Antonio Andelic](https://github.com/antonio2368)). +* Added query setting `partial_result_on_first_cancel` allowing the canceled query (e.g. due to Ctrl-C) to return a partial result. [#45689](https://github.com/ClickHouse/ClickHouse/pull/45689) ([Alexey Perevyshin](https://github.com/alexX512)). +* Added support of arbitrary tables engines for temporary tables except for Replicated and KeeperMap engines. Partially close [#31497](https://github.com/ClickHouse/ClickHouse/issues/31497). [#46071](https://github.com/ClickHouse/ClickHouse/pull/46071) ([Roman Vasin](https://github.com/rvasin)). +* Add replication of user-defined SQL functions using ZooKeeper. [#46085](https://github.com/ClickHouse/ClickHouse/pull/46085) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Implement `system.server_settings` (similar to `system.settings`), which will contain server configurations. [#46550](https://github.com/ClickHouse/ClickHouse/pull/46550) ([pufit](https://github.com/pufit)). +* Intruduce a function `WIDTH_BUCKET`. [#42974](https://github.com/ClickHouse/ClickHouse/issues/42974). [#46790](https://github.com/ClickHouse/ClickHouse/pull/46790) ([avoiderboi](https://github.com/avoiderboi)). +* Add new function parseDateTime/parseDateTimeInJodaSyntax according to specified format string. parseDateTime parses string to datetime in MySQL syntax, parseDateTimeInJodaSyntax parses in Joda syntax. [#46815](https://github.com/ClickHouse/ClickHouse/pull/46815) ([李扬](https://github.com/taiyang-li)). +* Use `dummy UInt8` for default structure of table function `null`. Closes [#46930](https://github.com/ClickHouse/ClickHouse/issues/46930). [#47006](https://github.com/ClickHouse/ClickHouse/pull/47006) ([flynn](https://github.com/ucasfl)). +* Dec 15, 2021 support for parseDateTimeBestEffort function. closes [#46816](https://github.com/ClickHouse/ClickHouse/issues/46816). [#47071](https://github.com/ClickHouse/ClickHouse/pull/47071) ([chen](https://github.com/xiedeyantu)). +* Add function ULIDStringToDateTime(). Closes [#46945](https://github.com/ClickHouse/ClickHouse/issues/46945). [#47087](https://github.com/ClickHouse/ClickHouse/pull/47087) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add settings `http_wait_end_of_query` and `http_response_buffer_size` that corresponds to URL params `wait_end_of_query` and `buffer_size` for HTTP interface. [#47108](https://github.com/ClickHouse/ClickHouse/pull/47108) ([Vladimir C](https://github.com/vdimir)). +* Support for `UNDROP TABLE` query. Closes [#46811](https://github.com/ClickHouse/ClickHouse/issues/46811). [#47241](https://github.com/ClickHouse/ClickHouse/pull/47241) ([chen](https://github.com/xiedeyantu)). +* Add `system.marked_dropped_tables` table that shows tables that were dropped from `Atomic` databases but were not completely removed yet. [#47364](https://github.com/ClickHouse/ClickHouse/pull/47364) ([chen](https://github.com/xiedeyantu)). +* Add `INSTR` as alias of `positionCaseInsensitive` for MySQL compatibility. Closes [#47529](https://github.com/ClickHouse/ClickHouse/issues/47529). [#47535](https://github.com/ClickHouse/ClickHouse/pull/47535) ([flynn](https://github.com/ucasfl)). +* Added `toDecimalString` function allowing to convert numbers to string with fixed precision. [#47838](https://github.com/ClickHouse/ClickHouse/pull/47838) ([Andrey Zvonov](https://github.com/zvonand)). +* Added operator "REGEXP" (similar to operators "LIKE", "IN", "MOD" etc.) for better compatibility with MySQL. [#47869](https://github.com/ClickHouse/ClickHouse/pull/47869) ([Robert Schulze](https://github.com/rschu1ze)). +* Allow executing reading pipeline for DIRECT dictionary with CLICKHOUSE source in multiple threads. To enable set `dictionary_use_async_executor=1` in `SETTINGS` section for source in `CREATE DICTIONARY` statement. [#47986](https://github.com/ClickHouse/ClickHouse/pull/47986) ([Vladimir C](https://github.com/vdimir)). +* Add merge tree setting `max_number_of_mutatuins_for_replica`. It limit the number of part mutations per replica to the specified amount. Zero means no limit on the number of mutations per replica (the execution can still be constrained by other settings). [#48047](https://github.com/ClickHouse/ClickHouse/pull/48047) ([Vladimir C](https://github.com/vdimir)). + +#### Performance Improvement +* Optimize one nullable key aggregate performance. [#45772](https://github.com/ClickHouse/ClickHouse/pull/45772) ([LiuNeng](https://github.com/liuneng1994)). +* Implemented lowercase tokenbf_v1 index utilization for hasTokenOrNull, hasTokenCaseInsensitive and hasTokenCaseInsensitiveOrNull. [#46252](https://github.com/ClickHouse/ClickHouse/pull/46252) ([ltrk2](https://github.com/ltrk2)). +* Optimize the generic SIMD StringSearcher by searching first two chars. [#46289](https://github.com/ClickHouse/ClickHouse/pull/46289) ([Jiebin Sun](https://github.com/jiebinn)). +* System.detached_parts could be significant large. - added several sources with respects block size limitation - in each block iothread pool is used to calculate part size, ie to make syscalls in parallel. [#46624](https://github.com/ClickHouse/ClickHouse/pull/46624) ([Sema Checherinda](https://github.com/CheSema)). +* Increase the default value of `max_replicated_merges_in_queue` for ReplicatedMergeTree tables from 16 to 1000. It allows faster background merge operation on clusters with a very large number of replicas, such as clusters with shared storage in ClickHouse Cloud. [#47050](https://github.com/ClickHouse/ClickHouse/pull/47050) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backups for large numbers of files were unbelievably slow in previous versions. [#47251](https://github.com/ClickHouse/ClickHouse/pull/47251) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support filter push down to left table for JOIN with StorageJoin, StorageDictionary, StorageEmbeddedRocksDB. [#47280](https://github.com/ClickHouse/ClickHouse/pull/47280) ([Maksim Kita](https://github.com/kitaisreal)). +* Marks in memory are now compressed, using 3-6x less memory. [#47290](https://github.com/ClickHouse/ClickHouse/pull/47290) ([Michael Kolupaev](https://github.com/al13n321)). +* Updated copier to use group by instead of distinct to get list of partitions. For large tables this reduced the select time from over 500s to under 1s. [#47386](https://github.com/ClickHouse/ClickHouse/pull/47386) ([Clayton McClure](https://github.com/cmcclure-twilio)). +* Address https://github.com/clickhouse/clickhouse/issues/46453. bisect marked https://github.com/clickhouse/clickhouse/pull/35525 as the bad changed. this pr looks to reverse the changes in that pr. [#47544](https://github.com/ClickHouse/ClickHouse/pull/47544) ([Ongkong](https://github.com/ongkong)). +* Fixed excessive reading in queries with `FINAL`. [#47801](https://github.com/ClickHouse/ClickHouse/pull/47801) ([Nikita Taranov](https://github.com/nickitat)). +* Setting `max_final_threads` would be set to number of cores at server startup (by the same algorithm as we use for `max_threads`). This improves concurrency of `final` execution on servers with high number of CPUs. [#47915](https://github.com/ClickHouse/ClickHouse/pull/47915) ([Nikita Taranov](https://github.com/nickitat)). +* Avoid breaking batches on read requests to improve performance. [#47978](https://github.com/ClickHouse/ClickHouse/pull/47978) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Improvement +* Add map related functions: mapFromArrays, which allows us to create map from a pair of arrays. [#31125](https://github.com/ClickHouse/ClickHouse/pull/31125) ([李扬](https://github.com/taiyang-li)). +* Rewrite distributed sends to avoid using filesystem as a queue, use in-memory queue instead. [#45491](https://github.com/ClickHouse/ClickHouse/pull/45491) ([Azat Khuzhin](https://github.com/azat)). +* Allow separate grants for named collections (e.g. to be able to give `SHOW/CREATE/ALTER/DROP named collection` access only to certain collections, instead of all at once). Closes [#40894](https://github.com/ClickHouse/ClickHouse/issues/40894). Add new access type `NAMED_COLLECTION_CONTROL` which is not given to default user unless explicitly added to user config (is required to be able to do `GRANT ALL`), also `show_named_collections` is no longer obligatory to be manually specified for default user to be able to have full access rights as was in 23.2. [#46241](https://github.com/ClickHouse/ClickHouse/pull/46241) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Now `X-ClickHouse-Query-Id` and `X-ClickHouse-Timezone` headers are added to response in all queries via http protocol. Previously it was done only for `SELECT` queries. [#46364](https://github.com/ClickHouse/ClickHouse/pull/46364) ([Anton Popov](https://github.com/CurtizJ)). +* Support for connection to a replica set via a URI with a host:port enum and support for the readPreference option in MongoDB dictionaries. Example URI: mongodb://db0.example.com:27017,db1.example.com:27017,db2.example.com:27017/?replicaSet=myRepl&readPreference=primary. [#46524](https://github.com/ClickHouse/ClickHouse/pull/46524) ([artem-yadr](https://github.com/artem-yadr)). +* Re-implement projection analysis on top of query plan. Added setting `query_plan_optimize_projection=1` to switch between old and new version. Fixes [#44963](https://github.com/ClickHouse/ClickHouse/issues/44963). [#46537](https://github.com/ClickHouse/ClickHouse/pull/46537) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Use parquet format v2 instead of v1 in output format by default. Add setting `output_format_parquet_version` to control parquet version, possible values `v1_0`, `v2_4`, `v2_6`, `v2_latest` (default). [#46617](https://github.com/ClickHouse/ClickHouse/pull/46617) ([Kruglov Pavel](https://github.com/Avogar)). +* Not for changelog - part of [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#46632](https://github.com/ClickHouse/ClickHouse/pull/46632) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Allow to ignore errors while pushing to MATERILIZED VIEW (add new setting `materialized_views_ignore_errors`, by default to `false`, but it is set to `true` for flushing logs to `system.*_log` tables unconditionally). [#46658](https://github.com/ClickHouse/ClickHouse/pull/46658) ([Azat Khuzhin](https://github.com/azat)). +* Enable input_format_json_ignore_unknown_keys_in_named_tuple by default. [#46742](https://github.com/ClickHouse/ClickHouse/pull/46742) ([Kruglov Pavel](https://github.com/Avogar)). +* It is now possible using new configuration syntax to configure Kafka topics with periods in their name. [#46752](https://github.com/ClickHouse/ClickHouse/pull/46752) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix heuristics that check hyperscan patterns for problematic repeats. [#46819](https://github.com/ClickHouse/ClickHouse/pull/46819) ([Robert Schulze](https://github.com/rschu1ze)). +* Don't report ZK node exists to system.errors when a block was created concurrently by a different replica. [#46820](https://github.com/ClickHouse/ClickHouse/pull/46820) ([Raúl Marín](https://github.com/Algunenano)). +* Allow PREWHERE for Merge with different DEFAULT expression for column. [#46831](https://github.com/ClickHouse/ClickHouse/pull/46831) ([Azat Khuzhin](https://github.com/azat)). +* Increase the limit for opened files in `clickhouse-local`. It will be able to read from `web` tables on servers with a huge number of CPU cores. Do not back off reading from the URL table engine in case of too many opened files. This closes [#46852](https://github.com/ClickHouse/ClickHouse/issues/46852). [#46853](https://github.com/ClickHouse/ClickHouse/pull/46853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Exceptions thrown when numbers cannot be parsed now have an easier-to-read exception message. [#46917](https://github.com/ClickHouse/ClickHouse/pull/46917) ([Robert Schulze](https://github.com/rschu1ze)). +* Added update `system.backups` after every processed task. [#46989](https://github.com/ClickHouse/ClickHouse/pull/46989) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Allow types conversion in Native input format. Add settings `input_format_native_allow_types_conversion` that controls it (enabled by default). [#46990](https://github.com/ClickHouse/ClickHouse/pull/46990) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow IPv4 in the `range` function to generate IP ranges. [#46995](https://github.com/ClickHouse/ClickHouse/pull/46995) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Role change was not promoted sometimes before https://github.com/ClickHouse/ClickHouse/pull/46772 This PR just adds tests. [#47002](https://github.com/ClickHouse/ClickHouse/pull/47002) ([Ilya Golshtein](https://github.com/ilejn)). +* Improve exception message when it's impossible to make part move from one volume/disk to another. [#47032](https://github.com/ClickHouse/ClickHouse/pull/47032) ([alesapin](https://github.com/alesapin)). +* Support `Bool` type in `JSONType` function. Previously `Null` type was mistakenly returned for bool values. [#47046](https://github.com/ClickHouse/ClickHouse/pull/47046) ([Anton Popov](https://github.com/CurtizJ)). +* Use _request_body parameter to configure predefined http queries. [#47086](https://github.com/ClickHouse/ClickHouse/pull/47086) ([Constantine Peresypkin](https://github.com/pkit)). +* Removing logging of custom disk structure. [#47103](https://github.com/ClickHouse/ClickHouse/pull/47103) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow nested custom disks. Previously custom disks supported only flat disk structure. [#47106](https://github.com/ClickHouse/ClickHouse/pull/47106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Automatic indentation in the built-in UI SQL editor when Enter is pressed. [#47113](https://github.com/ClickHouse/ClickHouse/pull/47113) ([Alexey Korepanov](https://github.com/alexkorep)). +* Allow control compression in Parquet/ORC/Arrow output formats, support more compression for input formats. This closes [#13541](https://github.com/ClickHouse/ClickHouse/issues/13541). [#47114](https://github.com/ClickHouse/ClickHouse/pull/47114) ([Kruglov Pavel](https://github.com/Avogar)). +* Self-extraction with 'sudo' will attempt to set uid and gid of extracted files to running user. [#47116](https://github.com/ClickHouse/ClickHouse/pull/47116) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Currently the funtion repeat's second argument must be unsigned integer type, which can not accept a integer value like -1. And this is different from the spark function, so I fix this here to make it same as spark. And it tested as below. [#47134](https://github.com/ClickHouse/ClickHouse/pull/47134) ([KevinyhZou](https://github.com/KevinyhZou)). +* Remove `::__1` part from stacktraces. Display `std::basic_string 1 trailing % [#46869](https://github.com/ClickHouse/ClickHouse/pull/46869) ([Robert Schulze](https://github.com/rschu1ze)). +* Add new metrics to system.asynchronous_metrics [#46886](https://github.com/ClickHouse/ClickHouse/pull/46886) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky `test_concurrent_queries_restriction_by_query_kind` [#46887](https://github.com/ClickHouse/ClickHouse/pull/46887) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test test_async_backups_to_same_destination. [#46888](https://github.com/ClickHouse/ClickHouse/pull/46888) ([Vitaly Baranov](https://github.com/vitlibar)). +* Make ASTSelectQuery::formatImpl() more robust [#46889](https://github.com/ClickHouse/ClickHouse/pull/46889) ([Robert Schulze](https://github.com/rschu1ze)). +* tests: fix 02116_interactive_hello for "official build" [#46911](https://github.com/ClickHouse/ClickHouse/pull/46911) ([Azat Khuzhin](https://github.com/azat)). +* Fix some expect tests leftovers and enable them in fasttest [#46915](https://github.com/ClickHouse/ClickHouse/pull/46915) ([Azat Khuzhin](https://github.com/azat)). +* Increase ddl timeout for DROP statement in backup restore tests [#46920](https://github.com/ClickHouse/ClickHouse/pull/46920) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* A better alternative to [#46344](https://github.com/ClickHouse/ClickHouse/issues/46344) [#46921](https://github.com/ClickHouse/ClickHouse/pull/46921) ([Robert Schulze](https://github.com/rschu1ze)). +* Code review from @tavplubix [#46922](https://github.com/ClickHouse/ClickHouse/pull/46922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Planner: trivial count optimization [#46923](https://github.com/ClickHouse/ClickHouse/pull/46923) ([Igor Nikonov](https://github.com/devcrafter)). +* Typo: SIZES_OF_ARRAYS_DOESNT_MATCH --> SIZES_OF_ARRAYS_DONT_MATCH [#46940](https://github.com/ClickHouse/ClickHouse/pull/46940) ([Robert Schulze](https://github.com/rschu1ze)). +* Another fix for clone() for ASTColumnMatchers [#46947](https://github.com/ClickHouse/ClickHouse/pull/46947) ([Nikolay Degterinsky](https://github.com/evillique)). +* Un-inline likePatternToRegexp() [#46950](https://github.com/ClickHouse/ClickHouse/pull/46950) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix missing format_description [#46959](https://github.com/ClickHouse/ClickHouse/pull/46959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ARM: Activate LDAPR with -march flag instead via -XClang [#46960](https://github.com/ClickHouse/ClickHouse/pull/46960) ([Robert Schulze](https://github.com/rschu1ze)). +* Preset description on the tweak reset [#46963](https://github.com/ClickHouse/ClickHouse/pull/46963) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update version_date.tsv and changelogs after v22.3.19.6-lts [#46964](https://github.com/ClickHouse/ClickHouse/pull/46964) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.8.14.53-lts [#46969](https://github.com/ClickHouse/ClickHouse/pull/46969) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Better exception messages when schema_inference_hints is ill-formatted [#46971](https://github.com/ClickHouse/ClickHouse/pull/46971) ([Kruglov Pavel](https://github.com/Avogar)). +* Decrease log level in "disks" [#46976](https://github.com/ClickHouse/ClickHouse/pull/46976) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Change the cherry-pick PR body [#46977](https://github.com/ClickHouse/ClickHouse/pull/46977) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Rename recent stateless tests to fix order [#46991](https://github.com/ClickHouse/ClickHouse/pull/46991) ([Kruglov Pavel](https://github.com/Avogar)). +* Pass headers from StorageURL to WriteBufferFromHTTP [#46996](https://github.com/ClickHouse/ClickHouse/pull/46996) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Change level log in executeQuery [#46997](https://github.com/ClickHouse/ClickHouse/pull/46997) ([Andrey Bystrov](https://github.com/AndyBys)). +* Add thevar1able to trusted contributors [#46998](https://github.com/ClickHouse/ClickHouse/pull/46998) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Use /etc/default/clickhouse in systemd too [#47003](https://github.com/ClickHouse/ClickHouse/pull/47003) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tmp_path_template in HTTPHandler::processQuery [#47007](https://github.com/ClickHouse/ClickHouse/pull/47007) ([Vladimir C](https://github.com/vdimir)). +* Fix flaky azure test [#47011](https://github.com/ClickHouse/ClickHouse/pull/47011) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Temporary enable force_sync for keeper in CI [#47024](https://github.com/ClickHouse/ClickHouse/pull/47024) ([alesapin](https://github.com/alesapin)). +* ActionsDAG: do not change result of and() during optimization - part 2 [#47028](https://github.com/ClickHouse/ClickHouse/pull/47028) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Add upgrade check to stateful dependent field [#47031](https://github.com/ClickHouse/ClickHouse/pull/47031) ([Kruglov Pavel](https://github.com/Avogar)). +* Disable path check in SQLite storage for clickhouse-local [#47052](https://github.com/ClickHouse/ClickHouse/pull/47052) ([Nikolay Degterinsky](https://github.com/evillique)). +* Terminate long-running offline non-busy runners in EC2 [#47064](https://github.com/ClickHouse/ClickHouse/pull/47064) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix Keeper with `force_sync = false` [#47065](https://github.com/ClickHouse/ClickHouse/pull/47065) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v23.2.2.20-stable [#47069](https://github.com/ClickHouse/ClickHouse/pull/47069) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.1.4.58-stable [#47070](https://github.com/ClickHouse/ClickHouse/pull/47070) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.12.4.76-stable [#47074](https://github.com/ClickHouse/ClickHouse/pull/47074) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix empty result when selection from only one side of join in analyzer [#47093](https://github.com/ClickHouse/ClickHouse/pull/47093) ([Vladimir C](https://github.com/vdimir)). +* Suppress "Cannot flush" for Distributed tables in upgrade check [#47095](https://github.com/ClickHouse/ClickHouse/pull/47095) ([Azat Khuzhin](https://github.com/azat)). +* Make stacktraces in hung check more readable [#47096](https://github.com/ClickHouse/ClickHouse/pull/47096) ([Alexander Tokmakov](https://github.com/tavplubix)). +* release lambda resources before detaching thread group [#47098](https://github.com/ClickHouse/ClickHouse/pull/47098) ([Sema Checherinda](https://github.com/CheSema)). +* Analyzer Planner fixes before enable by default [#47101](https://github.com/ClickHouse/ClickHouse/pull/47101) ([Maksim Kita](https://github.com/kitaisreal)). +* do flushUntrackedMemory when context switches [#47102](https://github.com/ClickHouse/ClickHouse/pull/47102) ([Sema Checherinda](https://github.com/CheSema)). +* fix: keeper systemd service file include invalid inline comment [#47105](https://github.com/ClickHouse/ClickHouse/pull/47105) ([SuperDJY](https://github.com/cmsxbc)). +* Add code for autoscaling lambda [#47107](https://github.com/ClickHouse/ClickHouse/pull/47107) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Enable lightweight delete support by default [#47109](https://github.com/ClickHouse/ClickHouse/pull/47109) ([Alexander Gololobov](https://github.com/davenger)). +* Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Shorten some code with CTAD [#47139](https://github.com/ClickHouse/ClickHouse/pull/47139) ([Robert Schulze](https://github.com/rschu1ze)). +* Make 01710_projections more stable. [#47145](https://github.com/ClickHouse/ClickHouse/pull/47145) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* fix_JSON_searchField [#47147](https://github.com/ClickHouse/ClickHouse/pull/47147) ([Aleksei Tikhomirov](https://github.com/aletik256)). +* Mark 01771_bloom_filter_not_has as no-parallel and long [#47148](https://github.com/ClickHouse/ClickHouse/pull/47148) ([Azat Khuzhin](https://github.com/azat)). +* Use unique names and paths in `test_replicated_database` [#47152](https://github.com/ClickHouse/ClickHouse/pull/47152) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add stupid retries in clickhouse-test health check. [#47158](https://github.com/ClickHouse/ClickHouse/pull/47158) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* 02346_full_text_search.sql: Add result separators to simplify analysis [#47166](https://github.com/ClickHouse/ClickHouse/pull/47166) ([Robert Schulze](https://github.com/rschu1ze)). +* More correct handling of fatal errors [#47175](https://github.com/ClickHouse/ClickHouse/pull/47175) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update read in StorageMemory [#47180](https://github.com/ClickHouse/ClickHouse/pull/47180) ([Konstantin Morozov](https://github.com/k-morozov)). +* Doc update for mapFromArrays() [#47183](https://github.com/ClickHouse/ClickHouse/pull/47183) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix failure context for Upgrade check [#47191](https://github.com/ClickHouse/ClickHouse/pull/47191) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add support for different expected errors [#47196](https://github.com/ClickHouse/ClickHouse/pull/47196) ([Raúl Marín](https://github.com/Algunenano)). +* Fix ip coding on s390x [#47208](https://github.com/ClickHouse/ClickHouse/pull/47208) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Add real client (initiator server) address into the logs for interserver mode [#47214](https://github.com/ClickHouse/ClickHouse/pull/47214) ([Azat Khuzhin](https://github.com/azat)). +* Fix 01019_alter_materialized_view_consistent [#47215](https://github.com/ClickHouse/ClickHouse/pull/47215) ([Vladimir C](https://github.com/vdimir)). +* Fix RewriteArrayExistsToHasPass [#47225](https://github.com/ClickHouse/ClickHouse/pull/47225) ([Maksim Kita](https://github.com/kitaisreal)). +* Release shared ptrs after finishing a transaction [#47245](https://github.com/ClickHouse/ClickHouse/pull/47245) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add default constructor for `MultiReadResponse` [#47254](https://github.com/ClickHouse/ClickHouse/pull/47254) ([Antonio Andelic](https://github.com/antonio2368)). +* Join threads if exception happened in `ZooKeeperImpl` constructor [#47261](https://github.com/ClickHouse/ClickHouse/pull/47261) ([Antonio Andelic](https://github.com/antonio2368)). +* use std::lerp, constexpr hex.h [#47268](https://github.com/ClickHouse/ClickHouse/pull/47268) ([Mike Kot](https://github.com/myrrc)). +* Update version_date.tsv and changelogs after v23.2.3.17-stable [#47269](https://github.com/ClickHouse/ClickHouse/pull/47269) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix bug in zero copy replica which can lead to dataloss [#47274](https://github.com/ClickHouse/ClickHouse/pull/47274) ([alesapin](https://github.com/alesapin)). +* Fix typo [#47282](https://github.com/ClickHouse/ClickHouse/pull/47282) ([Nikolay Degterinsky](https://github.com/evillique)). +* Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix test 02566_ipv4_ipv6_binary_formats [#47295](https://github.com/ClickHouse/ClickHouse/pull/47295) ([Kruglov Pavel](https://github.com/Avogar)). +* Set fixed index_granularity for test 00636 [#47298](https://github.com/ClickHouse/ClickHouse/pull/47298) ([Sema Checherinda](https://github.com/CheSema)). +* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix 02570_fallback_from_async_insert [#47308](https://github.com/ClickHouse/ClickHouse/pull/47308) ([Vladimir C](https://github.com/vdimir)). +* Catch exceptions in LiveViewPeriodicRefreshTask [#47309](https://github.com/ClickHouse/ClickHouse/pull/47309) ([Vladimir C](https://github.com/vdimir)). +* Fix MergeTreeTransaction::isReadOnly [#47310](https://github.com/ClickHouse/ClickHouse/pull/47310) ([Vladimir C](https://github.com/vdimir)). +* Fix an assertion with implicit transactions in interserver mode [#47312](https://github.com/ClickHouse/ClickHouse/pull/47312) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `File exists` error in Upgrade check [#47314](https://github.com/ClickHouse/ClickHouse/pull/47314) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support transformQueryForExternalDatabase for analyzer [#47316](https://github.com/ClickHouse/ClickHouse/pull/47316) ([Vladimir C](https://github.com/vdimir)). +* Disable parallel format in health check [#47318](https://github.com/ClickHouse/ClickHouse/pull/47318) ([Ilya Yatsishin](https://github.com/qoega)). +* Analyzer - fix combine logic for limit expression and limit setting [#47324](https://github.com/ClickHouse/ClickHouse/pull/47324) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Suppress expected errors from test 01111 in Upgrade check [#47365](https://github.com/ClickHouse/ClickHouse/pull/47365) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix GROUPING function initialization for grouping sets [#47370](https://github.com/ClickHouse/ClickHouse/pull/47370) ([Dmitry Novik](https://github.com/novikd)). +* Add join_algorithm='grace_hash' to stress tests [#47372](https://github.com/ClickHouse/ClickHouse/pull/47372) ([Pradeep Chhetri](https://github.com/chhetripradeep)). +* Fix 02343_group_by_use_nulls test in new analyzer [#47373](https://github.com/ClickHouse/ClickHouse/pull/47373) ([Dmitry Novik](https://github.com/novikd)). +* Disable 02368_cancel_write_into_hdfs in stress tests [#47382](https://github.com/ClickHouse/ClickHouse/pull/47382) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Analyzer planner fixes before enable by default [#47383](https://github.com/ClickHouse/ClickHouse/pull/47383) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix `ALTER CLEAR COLUMN` with sparse columns [#47384](https://github.com/ClickHouse/ClickHouse/pull/47384) ([Anton Popov](https://github.com/CurtizJ)). +* Fix: apply reading in order for distinct [#47385](https://github.com/ClickHouse/ClickHouse/pull/47385) ([Igor Nikonov](https://github.com/devcrafter)). +* add checks for ptr [#47398](https://github.com/ClickHouse/ClickHouse/pull/47398) ([Sema Checherinda](https://github.com/CheSema)). +* Remove distinct on top of MergingAggregatedStep [#47399](https://github.com/ClickHouse/ClickHouse/pull/47399) ([Igor Nikonov](https://github.com/devcrafter)). +* Update LRUFileCachePriority.cpp [#47411](https://github.com/ClickHouse/ClickHouse/pull/47411) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make test 02473_optimize_old_parts less flaky [#47416](https://github.com/ClickHouse/ClickHouse/pull/47416) ([Michael Kolupaev](https://github.com/al13n321)). +* Add test to prevent regressions when using bitmapHasAny [#47419](https://github.com/ClickHouse/ClickHouse/pull/47419) ([Jordi Villar](https://github.com/jrdi)). +* Update README.md [#47421](https://github.com/ClickHouse/ClickHouse/pull/47421) ([Tyler Hannan](https://github.com/tylerhannan)). +* Refactor query cache (make use of CacheBase) [#47428](https://github.com/ClickHouse/ClickHouse/pull/47428) ([Robert Schulze](https://github.com/rschu1ze)). +* Suppress Hung Check with UBsan [#47429](https://github.com/ClickHouse/ClickHouse/pull/47429) ([Alexander Tokmakov](https://github.com/tavplubix)). +* [docs] Document add async_insert_max_query_number [#47431](https://github.com/ClickHouse/ClickHouse/pull/47431) ([Antonio Bonuccelli](https://github.com/nellicus)). +* Apply settings for EXPLAIN earlier (in the same way we do for SELECT). [#47433](https://github.com/ClickHouse/ClickHouse/pull/47433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update version_date.tsv and changelogs after v23.2.4.12-stable [#47448](https://github.com/ClickHouse/ClickHouse/pull/47448) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix aggregation-in-order with aliases. [#47449](https://github.com/ClickHouse/ClickHouse/pull/47449) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix 01429_join_on_error_messages [#47450](https://github.com/ClickHouse/ClickHouse/pull/47450) ([Vladimir C](https://github.com/vdimir)). +* Update version_date.tsv and changelogs after v23.1.5.24-stable [#47452](https://github.com/ClickHouse/ClickHouse/pull/47452) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.12.5.34-stable [#47453](https://github.com/ClickHouse/ClickHouse/pull/47453) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Better error messages in ReplicatedMergeTreeAttachThread [#47454](https://github.com/ClickHouse/ClickHouse/pull/47454) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update version_date.tsv and changelogs after v22.8.15.23-lts [#47455](https://github.com/ClickHouse/ClickHouse/pull/47455) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Disable grace hash join in upgrade check [#47474](https://github.com/ClickHouse/ClickHouse/pull/47474) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Revert [#46622](https://github.com/ClickHouse/ClickHouse/issues/46622) (test_async_insert_memory) [#47476](https://github.com/ClickHouse/ClickHouse/pull/47476) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `00933_test_fix_extra_seek_on_compressed_cache` in releases. [#47490](https://github.com/ClickHouse/ClickHouse/pull/47490) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix long test `02371_select_projection_normal_agg.sql` [#47491](https://github.com/ClickHouse/ClickHouse/pull/47491) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert [#45878](https://github.com/ClickHouse/ClickHouse/issues/45878) and add a test [#47492](https://github.com/ClickHouse/ClickHouse/pull/47492) ([Kruglov Pavel](https://github.com/Avogar)). +* Planner JOIN TREE build fix [#47498](https://github.com/ClickHouse/ClickHouse/pull/47498) ([Maksim Kita](https://github.com/kitaisreal)). +* Better support of identifiers from compound expressions in analyzer [#47506](https://github.com/ClickHouse/ClickHouse/pull/47506) ([Anton Popov](https://github.com/CurtizJ)). +* Adapt some tests to pass with and without the analyzer [#47525](https://github.com/ClickHouse/ClickHouse/pull/47525) ([Raúl Marín](https://github.com/Algunenano)). +* Small enhancements [#47534](https://github.com/ClickHouse/ClickHouse/pull/47534) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support constants in INTERPOLATE clause (new analyzer) [#47539](https://github.com/ClickHouse/ClickHouse/pull/47539) ([Dmitry Novik](https://github.com/novikd)). +* Remove TOTALS handling in FillingTransform [#47542](https://github.com/ClickHouse/ClickHouse/pull/47542) ([Igor Nikonov](https://github.com/devcrafter)). +* Hide too noisy log messages, fix some tests [#47547](https://github.com/ClickHouse/ClickHouse/pull/47547) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix some flaky tests [#47553](https://github.com/ClickHouse/ClickHouse/pull/47553) ([Azat Khuzhin](https://github.com/azat)). +* remove counters for threads, fix negative counters [#47564](https://github.com/ClickHouse/ClickHouse/pull/47564) ([Sema Checherinda](https://github.com/CheSema)). +* Fix typo [#47565](https://github.com/ClickHouse/ClickHouse/pull/47565) ([hq1](https://github.com/aerosol)). +* Fixes for upgrade check [#47570](https://github.com/ClickHouse/ClickHouse/pull/47570) ([Azat Khuzhin](https://github.com/azat)). +* Change error code in case of columns definitions was empty in ODBC [#47573](https://github.com/ClickHouse/ClickHouse/pull/47573) ([Azat Khuzhin](https://github.com/azat)). +* Add missing SYSTEM FLUSH LOGS for log messages statistics [#47575](https://github.com/ClickHouse/ClickHouse/pull/47575) ([Azat Khuzhin](https://github.com/azat)). +* Fix performance regression in aggregation [#47582](https://github.com/ClickHouse/ClickHouse/pull/47582) ([Anton Popov](https://github.com/CurtizJ)). +* ReadFromMergeTree explain prewhere and row policy actions [#47583](https://github.com/ClickHouse/ClickHouse/pull/47583) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix possible failures of 01300_client_save_history_when_terminated_long [#47606](https://github.com/ClickHouse/ClickHouse/pull/47606) ([Azat Khuzhin](https://github.com/azat)). +* checksum: do not check inverted index files [#47607](https://github.com/ClickHouse/ClickHouse/pull/47607) ([save-my-heart](https://github.com/save-my-heart)). +* Add sanity checks for writing number in variable length format [#47608](https://github.com/ClickHouse/ClickHouse/pull/47608) ([Azat Khuzhin](https://github.com/azat)). +* Analyzer planner fixes before enable by default [#47622](https://github.com/ClickHouse/ClickHouse/pull/47622) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix exception message in clickhouse-test [#47625](https://github.com/ClickHouse/ClickHouse/pull/47625) ([Nikolay Degterinsky](https://github.com/evillique)). +* FillingTransform: remove unnecessary indirection when accessing columns [#47632](https://github.com/ClickHouse/ClickHouse/pull/47632) ([Igor Nikonov](https://github.com/devcrafter)). +* fix typo in HashJoin insertion that enables debug code in release build [#46726](https://github.com/ClickHouse/ClickHouse/issues/46726) [#47647](https://github.com/ClickHouse/ClickHouse/pull/47647) ([jorisgio](https://github.com/jorisgio)). +* clang-tidy >= 15: write CheckOptions in dictionary format [#47648](https://github.com/ClickHouse/ClickHouse/pull/47648) ([Robert Schulze](https://github.com/rschu1ze)). +* CMake: Build ClickHouse w/o GNU extensions [#47651](https://github.com/ClickHouse/ClickHouse/pull/47651) ([Robert Schulze](https://github.com/rschu1ze)). +* Faster fasttest [#47654](https://github.com/ClickHouse/ClickHouse/pull/47654) ([Robert Schulze](https://github.com/rschu1ze)). +* Add background pools size metrics [#47656](https://github.com/ClickHouse/ClickHouse/pull/47656) ([Sergei Trifonov](https://github.com/serxa)). +* Improve ThreadPool [#47657](https://github.com/ClickHouse/ClickHouse/pull/47657) ([Vitaly Baranov](https://github.com/vitlibar)). +* cmake: remove support for gold linker [#47660](https://github.com/ClickHouse/ClickHouse/pull/47660) ([Robert Schulze](https://github.com/rschu1ze)). +* Updated events and recordings [#47668](https://github.com/ClickHouse/ClickHouse/pull/47668) ([clickhouse-adrianfraguela](https://github.com/clickhouse-adrianfraguela)). +* Follow-up to [#47660](https://github.com/ClickHouse/ClickHouse/issues/47660): Further removal of gold linker support [#47669](https://github.com/ClickHouse/ClickHouse/pull/47669) ([Robert Schulze](https://github.com/rschu1ze)). +* Enable parallel execution for two tests [#47670](https://github.com/ClickHouse/ClickHouse/pull/47670) ([Robert Schulze](https://github.com/rschu1ze)). +* Restore native macos build [#47673](https://github.com/ClickHouse/ClickHouse/pull/47673) ([Robert Schulze](https://github.com/rschu1ze)). +* CMake: Remove further cruft from build [#47680](https://github.com/ClickHouse/ClickHouse/pull/47680) ([Robert Schulze](https://github.com/rschu1ze)). +* fix test / remove hardcoded database [#47682](https://github.com/ClickHouse/ClickHouse/pull/47682) ([Denny Crane](https://github.com/den-crane)). +* Apply log_queries_cut_to_length in MergeTreeWhereOptimizer [#47684](https://github.com/ClickHouse/ClickHouse/pull/47684) ([Vladimir C](https://github.com/vdimir)). +* Fix logical error in evaluate constant expression [#47685](https://github.com/ClickHouse/ClickHouse/pull/47685) ([Vladimir C](https://github.com/vdimir)). +* Try making `test_keeper_mntr_data_size` less flaky [#47687](https://github.com/ClickHouse/ClickHouse/pull/47687) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix limit offset [#47688](https://github.com/ClickHouse/ClickHouse/pull/47688) ([flynn](https://github.com/ucasfl)). +* Fix startup on older systemd versions [#47689](https://github.com/ClickHouse/ClickHouse/pull/47689) ([Thomas Casteleyn](https://github.com/Hipska)). +* More random query id in tests [#47700](https://github.com/ClickHouse/ClickHouse/pull/47700) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a style check for unsafe code [#47703](https://github.com/ClickHouse/ClickHouse/pull/47703) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make the code in Join less disgusting [#47712](https://github.com/ClickHouse/ClickHouse/pull/47712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixup git reference to LLVM [#47719](https://github.com/ClickHouse/ClickHouse/pull/47719) ([Robert Schulze](https://github.com/rschu1ze)). +* Preparation for libcxx(abi), llvm, clang-tidy 16 [#47722](https://github.com/ClickHouse/ClickHouse/pull/47722) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename cfg parameter query_cache.size to query_cache.max_size [#47724](https://github.com/ClickHouse/ClickHouse/pull/47724) ([Robert Schulze](https://github.com/rschu1ze)). +* Add optimization for MemoryStorageStep [#47726](https://github.com/ClickHouse/ClickHouse/pull/47726) ([Konstantin Morozov](https://github.com/k-morozov)). +* Fix aggregation with constant key in planner [#47727](https://github.com/ClickHouse/ClickHouse/pull/47727) ([Dmitry Novik](https://github.com/novikd)). +* Disable setting in 02343_group_by_use_nulls_distributed (for new analyzer) [#47728](https://github.com/ClickHouse/ClickHouse/pull/47728) ([Dmitry Novik](https://github.com/novikd)). +* Add a test for [#21469](https://github.com/ClickHouse/ClickHouse/issues/21469) [#47736](https://github.com/ClickHouse/ClickHouse/pull/47736) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#23804](https://github.com/ClickHouse/ClickHouse/issues/23804) [#47737](https://github.com/ClickHouse/ClickHouse/pull/47737) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#18937](https://github.com/ClickHouse/ClickHouse/issues/18937) [#47738](https://github.com/ClickHouse/ClickHouse/pull/47738) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#17756](https://github.com/ClickHouse/ClickHouse/issues/17756) [#47739](https://github.com/ClickHouse/ClickHouse/pull/47739) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#23162](https://github.com/ClickHouse/ClickHouse/issues/23162) [#47740](https://github.com/ClickHouse/ClickHouse/pull/47740) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* remove unused code [#47743](https://github.com/ClickHouse/ClickHouse/pull/47743) ([flynn](https://github.com/ucasfl)). +* Fix broken cross-compiled macos builds [#47744](https://github.com/ClickHouse/ClickHouse/pull/47744) ([Robert Schulze](https://github.com/rschu1ze)). +* Randomize query cache settings [#47749](https://github.com/ClickHouse/ClickHouse/pull/47749) ([Robert Schulze](https://github.com/rschu1ze)). +* Clarify steps for reopened cherry-pick PRs [#47755](https://github.com/ClickHouse/ClickHouse/pull/47755) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix ZK exception error message [#47757](https://github.com/ClickHouse/ClickHouse/pull/47757) ([Raúl Marín](https://github.com/Algunenano)). +* Add ComparisonTupleEliminationVisitor [#47758](https://github.com/ClickHouse/ClickHouse/pull/47758) ([Vladimir C](https://github.com/vdimir)). +* Add a fuse for backport branches w/o a created PR [#47760](https://github.com/ClickHouse/ClickHouse/pull/47760) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix partition ID byte order for s390x [#47769](https://github.com/ClickHouse/ClickHouse/pull/47769) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Stop `wait for quorum` retries on shutdown [#47770](https://github.com/ClickHouse/ClickHouse/pull/47770) ([Igor Nikonov](https://github.com/devcrafter)). +* More preparation for upgrade to libcxx(abi), llvm, clang-tidy 16 [#47771](https://github.com/ClickHouse/ClickHouse/pull/47771) ([Robert Schulze](https://github.com/rschu1ze)). +* Only valid Reviews.STATES overwrite existing reviews [#47789](https://github.com/ClickHouse/ClickHouse/pull/47789) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Apply black formatter to all python scripts [#47790](https://github.com/ClickHouse/ClickHouse/pull/47790) ([Anton Popov](https://github.com/CurtizJ)). +* Try fix window view test [#47791](https://github.com/ClickHouse/ClickHouse/pull/47791) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update test for nested lambdas [#47795](https://github.com/ClickHouse/ClickHouse/pull/47795) ([Dmitry Novik](https://github.com/novikd)). +* Decrease scale_down ratio for faster deflation [#47798](https://github.com/ClickHouse/ClickHouse/pull/47798) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix 993 and two other tests [#47802](https://github.com/ClickHouse/ClickHouse/pull/47802) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix flaky test 02417_opentelemetry_insert_on_distributed_table [#47811](https://github.com/ClickHouse/ClickHouse/pull/47811) ([Azat Khuzhin](https://github.com/azat)). +* Make 01086_odbc_roundtrip less flaky [#47820](https://github.com/ClickHouse/ClickHouse/pull/47820) ([Antonio Andelic](https://github.com/antonio2368)). +* Place short return before big block, improve logging [#47822](https://github.com/ClickHouse/ClickHouse/pull/47822) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* [FixTests] Remove wrong chassert() in UserDefinedSQLObjectsLoaderFromZooKeeper.cpp [#47839](https://github.com/ClickHouse/ClickHouse/pull/47839) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix test test_replicated_merge_tree_encryption_codec [#47851](https://github.com/ClickHouse/ClickHouse/pull/47851) ([Vitaly Baranov](https://github.com/vitlibar)). +* Allow injecting timeout errors on Keeper [#47856](https://github.com/ClickHouse/ClickHouse/pull/47856) ([Raúl Marín](https://github.com/Algunenano)). +* Comment stale cherry-pick PRs once a day to remind for resolving conflicts [#47857](https://github.com/ClickHouse/ClickHouse/pull/47857) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Followup to [#47802](https://github.com/ClickHouse/ClickHouse/issues/47802) [#47864](https://github.com/ClickHouse/ClickHouse/pull/47864) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Slightly better error message [#47868](https://github.com/ClickHouse/ClickHouse/pull/47868) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Make test_server_reload non-parallel [#47871](https://github.com/ClickHouse/ClickHouse/pull/47871) ([Alexander Tokmakov](https://github.com/tavplubix)). +* aspell-dict.txt: keep sorted things sorted [#47878](https://github.com/ClickHouse/ClickHouse/pull/47878) ([Robert Schulze](https://github.com/rschu1ze)). +* throw exception when all retries exhausted [#47902](https://github.com/ClickHouse/ClickHouse/pull/47902) ([Sema Checherinda](https://github.com/CheSema)). +* Fix GRANT query formatting [#47908](https://github.com/ClickHouse/ClickHouse/pull/47908) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix exception type in arrayElement function [#47909](https://github.com/ClickHouse/ClickHouse/pull/47909) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix logical error in DistributedSink [#47916](https://github.com/ClickHouse/ClickHouse/pull/47916) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix terminate in parts check thread [#47917](https://github.com/ClickHouse/ClickHouse/pull/47917) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Limit keeper request batching by size in bytes [#47918](https://github.com/ClickHouse/ClickHouse/pull/47918) ([Alexander Gololobov](https://github.com/davenger)). +* Improve replicated user defined functions [#47919](https://github.com/ClickHouse/ClickHouse/pull/47919) ([Vitaly Baranov](https://github.com/vitlibar)). +* Update 01072_window_view_multiple_columns_groupby.sh [#47928](https://github.com/ClickHouse/ClickHouse/pull/47928) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Added test. Closes [#12264](https://github.com/ClickHouse/ClickHouse/issues/12264) [#47931](https://github.com/ClickHouse/ClickHouse/pull/47931) ([Ilya Yatsishin](https://github.com/qoega)). +* Disallow concurrent backup restore test - removed SYSTEM SYNC [#47944](https://github.com/ClickHouse/ClickHouse/pull/47944) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Artifacts s3 prefix [#47945](https://github.com/ClickHouse/ClickHouse/pull/47945) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Set content-length for empty POST requests [#47950](https://github.com/ClickHouse/ClickHouse/pull/47950) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix test `02050_client_profile_events` [#47951](https://github.com/ClickHouse/ClickHouse/pull/47951) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix tsan error lock-order-inversion [#47953](https://github.com/ClickHouse/ClickHouse/pull/47953) ([Kruglov Pavel](https://github.com/Avogar)). +* Update docs for parseDateTime() (follow-up to [#46815](https://github.com/ClickHouse/ClickHouse/issues/46815)) [#47959](https://github.com/ClickHouse/ClickHouse/pull/47959) ([Robert Schulze](https://github.com/rschu1ze)). +* Docs: Update secondary index example [#47961](https://github.com/ClickHouse/ClickHouse/pull/47961) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix compilation on MacOS [#47967](https://github.com/ClickHouse/ClickHouse/pull/47967) ([Jordi Villar](https://github.com/jrdi)). +* [Refactoring] Move information about current hosts and list of all hosts to BackupCoordination [#47971](https://github.com/ClickHouse/ClickHouse/pull/47971) ([Vitaly Baranov](https://github.com/vitlibar)). +* Stabilize tests for new function parseDateTimeInJodaSyntax [#47974](https://github.com/ClickHouse/ClickHouse/pull/47974) ([Robert Schulze](https://github.com/rschu1ze)). +* Docs: Fix links [#47976](https://github.com/ClickHouse/ClickHouse/pull/47976) ([Robert Schulze](https://github.com/rschu1ze)). +* Try fix rabbitmq test [#47987](https://github.com/ClickHouse/ClickHouse/pull/47987) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Better type check in arrayElement function [#47989](https://github.com/ClickHouse/ClickHouse/pull/47989) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix incorrect code indentation [#48011](https://github.com/ClickHouse/ClickHouse/pull/48011) ([exmy](https://github.com/exmy)). +* CMake: Remove configuration of CMAKE_SHARED_LINKER_FLAGS [#48018](https://github.com/ClickHouse/ClickHouse/pull/48018) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove the old changelog script [#48042](https://github.com/ClickHouse/ClickHouse/pull/48042) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix automatic indentation in the built-in UI SQL editor [#48045](https://github.com/ClickHouse/ClickHouse/pull/48045) ([Nikolay Degterinsky](https://github.com/evillique)). +* Rename `system.marked_dropped_tables` to `dropped_tables` [#48048](https://github.com/ClickHouse/ClickHouse/pull/48048) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Automatically correct some mistakes in the changelog [#48052](https://github.com/ClickHouse/ClickHouse/pull/48052) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Docs: Document [FULL] keyword in SHOW TABLES [#48061](https://github.com/ClickHouse/ClickHouse/pull/48061) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix stateless tests numbers [#48063](https://github.com/ClickHouse/ClickHouse/pull/48063) ([Raúl Marín](https://github.com/Algunenano)). +* Docs: Update syntax of some SHOW queries [#48064](https://github.com/ClickHouse/ClickHouse/pull/48064) ([Robert Schulze](https://github.com/rschu1ze)). +* Simplify backup coordination for file infos [#48095](https://github.com/ClickHouse/ClickHouse/pull/48095) ([Vitaly Baranov](https://github.com/vitlibar)). +* materialized pg small fix [#48098](https://github.com/ClickHouse/ClickHouse/pull/48098) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update SQLite to 3.41.2 [#48101](https://github.com/ClickHouse/ClickHouse/pull/48101) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix test numbers again and enforce it with style [#48106](https://github.com/ClickHouse/ClickHouse/pull/48106) ([Raúl Marín](https://github.com/Algunenano)). +* s390x reinterpret as float64 [#48112](https://github.com/ClickHouse/ClickHouse/pull/48112) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Remove slow outdated test [#48114](https://github.com/ClickHouse/ClickHouse/pull/48114) ([alesapin](https://github.com/alesapin)). +* Cosmetic follow-up to [#46252](https://github.com/ClickHouse/ClickHouse/issues/46252) [#48128](https://github.com/ClickHouse/ClickHouse/pull/48128) ([Robert Schulze](https://github.com/rschu1ze)). +* Merging "Support undrop table" [#48130](https://github.com/ClickHouse/ClickHouse/pull/48130) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix double whitespace in exception message [#48132](https://github.com/ClickHouse/ClickHouse/pull/48132) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve script for updating clickhouse-docs [#48135](https://github.com/ClickHouse/ClickHouse/pull/48135) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix stdlib compatibility issues [#48150](https://github.com/ClickHouse/ClickHouse/pull/48150) ([DimasKovas](https://github.com/DimasKovas)). +* Make test test_disallow_concurrency less flaky [#48152](https://github.com/ClickHouse/ClickHouse/pull/48152) ([Vitaly Baranov](https://github.com/vitlibar)). +* Remove unused mockSystemDatabase from gtest_transform_query_for_exter… [#48162](https://github.com/ClickHouse/ClickHouse/pull/48162) ([Vladimir C](https://github.com/vdimir)). +* Update environmental-sensors.md [#48166](https://github.com/ClickHouse/ClickHouse/pull/48166) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Correctly handle NULL constants in logical optimizer for new analyzer [#48168](https://github.com/ClickHouse/ClickHouse/pull/48168) ([Antonio Andelic](https://github.com/antonio2368)). +* Try making KeeperMap test more stable [#48170](https://github.com/ClickHouse/ClickHouse/pull/48170) ([Antonio Andelic](https://github.com/antonio2368)). +* Deprecate EXPLAIN QUERY TREE with disabled analyzer. [#48177](https://github.com/ClickHouse/ClickHouse/pull/48177) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Use uniq file names in 02149_* tests to avoid SIGBUS in stress tests [#48187](https://github.com/ClickHouse/ClickHouse/pull/48187) ([Kruglov Pavel](https://github.com/Avogar)). +* Update style in ParserKQLSort.cpp [#48199](https://github.com/ClickHouse/ClickHouse/pull/48199) ([Ilya Yatsishin](https://github.com/qoega)). +* Remove support for std::unary/binary_function (removed in C++17) [#48204](https://github.com/ClickHouse/ClickHouse/pull/48204) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove unused setting [#48208](https://github.com/ClickHouse/ClickHouse/pull/48208) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove wrong assert from LogicalExpressionOptimizerPass [#48214](https://github.com/ClickHouse/ClickHouse/pull/48214) ([Antonio Andelic](https://github.com/antonio2368)). +* MySQL compatibility: Make str_to_date alias case-insensitive [#48220](https://github.com/ClickHouse/ClickHouse/pull/48220) ([Robert Schulze](https://github.com/rschu1ze)). +* Disable AST optimizations for projection analysis. [#48221](https://github.com/ClickHouse/ClickHouse/pull/48221) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Too big of a difference between test numbers [#48224](https://github.com/ClickHouse/ClickHouse/pull/48224) ([Vladimir C](https://github.com/vdimir)). +* Stabilize 02477_age [#48225](https://github.com/ClickHouse/ClickHouse/pull/48225) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename setting stop_reading_on_first_cancel [#48226](https://github.com/ClickHouse/ClickHouse/pull/48226) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Address flaky 02346_full_text_search [#48227](https://github.com/ClickHouse/ClickHouse/pull/48227) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix incorrect ThreadPool usage after ThreadPool introspection [#48244](https://github.com/ClickHouse/ClickHouse/pull/48244) ([Azat Khuzhin](https://github.com/azat)). +* fix test numbers again [#48264](https://github.com/ClickHouse/ClickHouse/pull/48264) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### Testing Improvement + +* Fixed functional test 02534_keyed_siphash and 02552_siphash128_reference for s390x. [#47615](https://github.com/ClickHouse/ClickHouse/pull/47615) ([Harry Lee](https://github.com/HarryLeeIBM)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index c2d9781177d..f46a422446e 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.3.1.2823-lts 2023-03-31 v23.2.4.12-stable 2023-03-10 v23.2.3.17-stable 2023-03-06 v23.2.2.20-stable 2023-03-01 From cc61d35bc5eb61f267ca67a30e2ef159228bb5fb Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 28 Mar 2023 17:57:58 +0000 Subject: [PATCH 148/233] Fix overflow in sparkbar function --- .../AggregateFunctionSparkbar.h | 30 ++++++++++++++++--- .../02016_aggregation_spark_bar.sql | 8 ++++- 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index 919b59448a1..0927ff8994d 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -43,7 +43,16 @@ struct AggregateFunctionSparkbarData auto [it, inserted] = points.insert({x, y}); if (!inserted) - it->getMapped() += y; + { + if (std::numeric_limits::max() - it->getMapped() > y) + { + it->getMapped() += y; + } + else + { + it->getMapped() = std::numeric_limits::max(); + } + } return it->getMapped(); } @@ -117,6 +126,7 @@ class AggregateFunctionSparkbar final { private: + static constexpr size_t BAR_LEVELS = 8; const size_t width = 0; /// Range for x specified in parameters. @@ -126,8 +136,8 @@ private: size_t updateFrame(ColumnString::Chars & frame, Y value) const { - static constexpr std::array bars{" ", "▁", "▂", "▃", "▄", "▅", "▆", "▇", "█"}; - const auto & bar = (isNaN(value) || value < 1 || 8 < value) ? bars[0] : bars[static_cast(value)]; + static constexpr std::array bars{" ", "▁", "▂", "▃", "▄", "▅", "▆", "▇", "█"}; + const auto & bar = (isNaN(value) || value < 1 || static_cast(BAR_LEVELS) < value) ? bars[0] : bars[static_cast(value)]; frame.insert(bar.begin(), bar.end()); return bar.size(); } @@ -211,10 +221,22 @@ private: for (auto & y : histogram) { + constexpr auto bucket_num = static_cast(BAR_LEVELS - 1); + if (isNaN(y) || y <= 0) + { y = 0; + continue; + } + + /// handle potential overflow + if (y_max > bucket_num && y >= std::numeric_limits::max() / bucket_num) + y = y / (y_max / bucket_num); else - y = y * 7 / y_max + 1; + y = y * bucket_num / y_max; + + if (y < std::numeric_limits::max()) + y += 1; } size_t sz = 0; diff --git a/tests/queries/0_stateless/02016_aggregation_spark_bar.sql b/tests/queries/0_stateless/02016_aggregation_spark_bar.sql index 2100a3dd4a6..4d2de566eda 100644 --- a/tests/queries/0_stateless/02016_aggregation_spark_bar.sql +++ b/tests/queries/0_stateless/02016_aggregation_spark_bar.sql @@ -33,7 +33,7 @@ SELECT sparkbar(4,toDate('2020-01-01'),toDate('2020-01-08'))(event_date,cnt) FRO SELECT sparkbar(5,toDate('2020-01-01'),toDate('2020-01-10'))(event_date,cnt) FROM spark_bar_test; SELECT sparkbar(9,toDate('2020-01-01'),toDate('2020-01-10'))(event_date,cnt) FROM spark_bar_test; -WITH number DIV 50 AS k, number % 50 AS value SELECT k, sparkbar(50, 0, 99)(number, value) FROM numbers(100) GROUP BY k ORDER BY k; +WITH number DIV 50 AS k, toUInt32(number % 50) AS value SELECT k, sparkbar(50, 0, 99)(number, value) FROM numbers(100) GROUP BY k ORDER BY k; SELECT sparkbar(128, 0, 9223372036854775806)(toUInt64(9223372036854775806), number % 65535) FROM numbers(100); SELECT sparkbar(128)(toUInt64(9223372036854775806), number % 65535) FROM numbers(100); @@ -59,4 +59,10 @@ SELECT sparkbar(2)(toInt32(number), number) FROM numbers(10); -- { serverError SELECT sparkbar(2, 0)(number, number) FROM numbers(10); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT sparkbar(2, 0, 5, 8)(number, number) FROM numbers(10); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- it causes overflow, just check that it doesn't crash under UBSan, do not check the result it's not really reasonable +SELECT sparkbar(10)(number, toInt64(number)) FROM numbers(toUInt64(9223372036854775807), 20) FORMAT Null; +SELECT sparkbar(10)(number, -number) FROM numbers(toUInt64(9223372036854775807), 7) FORMAT Null; +SELECT sparkbar(10)(number, number) FROM numbers(18446744073709551615, 7) FORMAT Null; +SELECT sparkbar(16)(number, number) FROM numbers(18446744073709551600, 16) FORMAT Null; + DROP TABLE IF EXISTS spark_bar_test; From def7b19033398e535d5c0cb46efcfb9b102c6ab5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 28 Mar 2023 18:00:25 +0000 Subject: [PATCH 149/233] Update 02016_aggregation_spark_bar.reference --- .../queries/0_stateless/02016_aggregation_spark_bar.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02016_aggregation_spark_bar.reference b/tests/queries/0_stateless/02016_aggregation_spark_bar.reference index 534942fc1d5..35a629d2bc0 100644 --- a/tests/queries/0_stateless/02016_aggregation_spark_bar.reference +++ b/tests/queries/0_stateless/02016_aggregation_spark_bar.reference @@ -46,7 +46,7 @@ SELECT sparkbar(5,toDate('2020-01-01'),toDate('2020-01-10'))(event_date,cnt) FRO ▃▄▆█ SELECT sparkbar(9,toDate('2020-01-01'),toDate('2020-01-10'))(event_date,cnt) FROM spark_bar_test; ▂▅▂▃▇▆█ -WITH number DIV 50 AS k, number % 50 AS value SELECT k, sparkbar(50, 0, 99)(number, value) FROM numbers(100) GROUP BY k ORDER BY k; +WITH number DIV 50 AS k, toUInt32(number % 50) AS value SELECT k, sparkbar(50, 0, 99)(number, value) FROM numbers(100) GROUP BY k ORDER BY k; 0 ▁▁▁▁▂▂▂▃▃▃▃▄▄▄▅▅▅▅▆▆▆▇▇▇█ 1 ▁▁▁▁▂▂▂▃▃▃▃▄▄▄▅▅▅▅▆▆▆▇▇▇█ SELECT sparkbar(128, 0, 9223372036854775806)(toUInt64(9223372036854775806), number % 65535) FROM numbers(100); @@ -54,7 +54,7 @@ SELECT sparkbar(128, 0, 9223372036854775806)(toUInt64(9223372036854775806), numb SELECT sparkbar(128)(toUInt64(9223372036854775806), number % 65535) FROM numbers(100); █ SELECT sparkbar(9)(x, y) FROM (SELECT * FROM Values('x UInt64, y UInt8', (18446744073709551615,255), (0,0), (0,0), (4036797895307271799,254))); - ▇ █ + █ █ SELECT sparkbar(8, 0, 7)((number + 1) % 8, 1), sparkbar(8, 0, 7)((number + 2) % 8, 1), sparkbar(8, 0, 7)((number + 3) % 8, 1) FROM numbers(7); ███████ █ ██████ ██ █████ SELECT sparkbar(2)(number, -number) FROM numbers(10); From 24b46774b0e72c67746fe2daf455458c357844e5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 29 Mar 2023 09:34:42 +0000 Subject: [PATCH 150/233] update AggregateFunctionSparkbar.h --- .../AggregateFunctionSparkbar.h | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index 0927ff8994d..29fb65e8fff 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -171,7 +171,7 @@ private: } PaddedPODArray histogram(width, 0); - PaddedPODArray fhistogram(width, 0); + PaddedPODArray count_histogram(width, 0); /// The number of points in each bucket for (const auto & point : data.points) { @@ -189,7 +189,7 @@ private: if (std::numeric_limits::max() - histogram[index] > point.getMapped()) { histogram[index] += point.getMapped(); - fhistogram[index] += 1; + count_histogram[index] += 1; } else { @@ -200,8 +200,8 @@ private: for (size_t i = 0; i < histogram.size(); ++i) { - if (fhistogram[i] > 0) - histogram[i] /= fhistogram[i]; + if (count_histogram[i] > 0) + histogram[i] /= count_histogram[i]; } Y y_max = 0; @@ -219,24 +219,21 @@ private: return; } + /// Scale the histogram to the range [0, BAR_LEVELS] for (auto & y : histogram) { - constexpr auto bucket_num = static_cast(BAR_LEVELS - 1); - if (isNaN(y) || y <= 0) { y = 0; continue; } + constexpr auto levels_num = static_cast(BAR_LEVELS - 1); /// handle potential overflow - if (y_max > bucket_num && y >= std::numeric_limits::max() / bucket_num) - y = y / (y_max / bucket_num); + if (y_max > levels_num && y >= std::numeric_limits::max() / levels_num) + y = y / (y_max / levels_num) + 1; else - y = y * bucket_num / y_max; - - if (y < std::numeric_limits::max()) - y += 1; + y = y * levels_num / y_max + 1; } size_t sz = 0; From f54fd15112de580ce0330173597e22128dc05f5b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 12:57:29 +0200 Subject: [PATCH 151/233] fix --- src/Databases/DatabaseReplicated.cpp | 2 +- src/Interpreters/DDLWorker.cpp | 1 - src/Interpreters/DDLWorker.h | 1 - src/Interpreters/executeDDLQueryOnCluster.cpp | 47 +++++++++---------- src/Interpreters/executeDDLQueryOnCluster.h | 3 +- tests/integration/helpers/client.py | 1 + 6 files changed, 25 insertions(+), 30 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1d39796e4fc..76eea059174 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -661,7 +661,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context); Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); - return getDistributedDDLStatus(node_path, entry, query_context, hosts_to_wait); + return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); } static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 5389d11a2ce..22bece0ef04 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 62ca6cba5e8..6cf034edae8 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index a93cc2f06ba..5b8609fd8af 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -37,25 +37,16 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -struct RetriesForDDL -{ - ZooKeeperRetriesInfo info; - ZooKeeperRetriesControl ctl; -}; - -static RetriesForDDL getRetriesForDistributedDDL() +static ZooKeeperRetriesInfo getRetriesInfo() { const auto & config_ref = Context::getGlobalContextInstance()->getConfigRef(); - auto info = ZooKeeperRetriesInfo( + return ZooKeeperRetriesInfo( "DistributedDDL", &Poco::Logger::get("DDLQueryStatusSource"), config_ref.getInt("distributed_ddl_keeper_max_retries", 5), config_ref.getInt("distributed_ddl_keeper_initial_backoff_ms", 100), config_ref.getInt("distributed_ddl_keeper_max_backoff_ms", 5000) - ); - - auto ctl = ZooKeeperRetriesControl("executeDDLQueryOnCluster", info); - return {info, ctl}; + ); } bool isSupportedAlterType(int type) @@ -195,7 +186,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.tracing_context = OpenTelemetry::CurrentContext(); String node_path = ddl_worker.enqueueQuery(entry); - return getDistributedDDLStatus(node_path, entry, context, /* hosts_to_wait */ std::nullopt); + return getDistributedDDLStatus(node_path, entry, context, /* hosts_to_wait */ nullptr); } @@ -203,7 +194,7 @@ class DDLQueryStatusSource final : public ISource { public: DDLQueryStatusSource( - const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const std::optional & hosts_to_wait); + const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const Strings * hosts_to_wait); String getName() const override { return "DDLQueryStatus"; } Chunk generate() override; @@ -251,8 +242,7 @@ private: }; -BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, - const std::optional & hosts_to_wait) +BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, const Strings * hosts_to_wait) { BlockIO io; if (context->getSettingsRef().distributed_ddl_task_timeout == 0) @@ -313,8 +303,8 @@ Block DDLQueryStatusSource::getSampleBlock(ContextPtr context_, bool hosts_to_wa } DDLQueryStatusSource::DDLQueryStatusSource( - const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const std::optional & hosts_to_wait) - : ISource(getSampleBlock(context_, hosts_to_wait.has_value())) + const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const Strings * hosts_to_wait) + : ISource(getSampleBlock(context_, static_cast(hosts_to_wait))) , node_path(zk_node_path) , context(context_) , watch(CLOCK_MONOTONIC_COARSE) @@ -445,13 +435,17 @@ Chunk DDLQueryStatusSource::generate() Strings tmp_hosts; Strings tmp_active_hosts; - getRetriesForDistributedDDL().ctl.retryLoop([&]() { - auto zookeeper = context->getZooKeeper(); - node_exists = zookeeper->exists(node_path); - tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait); - tmp_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active"); - }); + auto retries_info = getRetriesInfo(); + auto retries_ctl = ZooKeeperRetriesControl("executeDDLQueryOnCluster", retries_info); + retries_ctl.retryLoop([&]() + { + auto zookeeper = context->getZooKeeper(); + node_exists = zookeeper->exists(node_path); + tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait); + tmp_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active"); + }); + } if (!node_exists) { @@ -481,7 +475,10 @@ Chunk DDLQueryStatusSource::generate() { String status_data; bool finished_exists = false; - getRetriesForDistributedDDL().ctl.retryLoop([&]() + + auto retries_info = getRetriesInfo(); + auto retries_ctl = ZooKeeperRetriesControl("executeDDLQueryOnCluster", retries_info); + retries_ctl.retryLoop([&]() { finished_exists = context->getZooKeeper()->tryGet(fs::path(node_path) / "finished" / host_id, status_data); }); diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 19fb3fdb5a6..40db13d7ef5 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -43,8 +43,7 @@ struct DDLQueryOnClusterParams /// Returns DDLQueryStatusSource, which reads results of query execution on each host in the cluster. BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const DDLQueryOnClusterParams & params = {}); -BlockIO getDistributedDDLStatus( - const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional & hosts_to_wait); +BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, const Strings * hosts_to_wait); bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context); diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index 66e0f0bc907..c2676ac08a6 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -203,6 +203,7 @@ class CommandRequest: self.timer.start() def remove_trash_from_stderr(self, stderr): + # FIXME https://github.com/ClickHouse/ClickHouse/issues/48181 if not stderr: return stderr lines = stderr.split("\n") From 2d18689af67203c6512c651c474199824a4db092 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 31 Mar 2023 11:52:33 +0000 Subject: [PATCH 152/233] use common::addOverflow in AggregateFunctionSparkbar.h --- .../AggregateFunctionSparkbar.h | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index 29fb65e8fff..adbcea224d5 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -44,14 +46,9 @@ struct AggregateFunctionSparkbarData auto [it, inserted] = points.insert({x, y}); if (!inserted) { - if (std::numeric_limits::max() - it->getMapped() > y) - { - it->getMapped() += y; - } - else - { - it->getMapped() = std::numeric_limits::max(); - } + Y res; + bool has_overfllow = common::addOverflow(it->getMapped(), y, res) + it->getMapped() = has_overfllow ? std::numeric_limits::max() : res; } return it->getMapped(); } @@ -186,15 +183,18 @@ private: Float64 w = histogram.size(); size_t index = std::min(static_cast(w / delta * value), histogram.size() - 1); - if (std::numeric_limits::max() - histogram[index] > point.getMapped()) + Y res; + bool has_overfllow = common::addOverflow(histogram[index], point.getMapped(), res); + if (unlikely(has_overfllow)) { - histogram[index] += point.getMapped(); - count_histogram[index] += 1; + /// In case of overflow, just saturate + /// Do not count new values, because we do not know how many of them were added + histogram[index] = std::numeric_limits::max(); } else { - /// In case of overflow, just saturate - histogram[index] = std::numeric_limits::max(); + histogram[index] = res; + count_histogram[index] += 1; } } @@ -230,10 +230,12 @@ private: constexpr auto levels_num = static_cast(BAR_LEVELS - 1); /// handle potential overflow - if (y_max > levels_num && y >= std::numeric_limits::max() / levels_num) + Y scaled; + bool has_overfllow = common::mulOverflow(y, levels_num, scaled); + if (has_overfllow) y = y / (y_max / levels_num) + 1; else - y = y * levels_num / y_max + 1; + y = scaled / y_max + 1; } size_t sz = 0; From aa8e5a107772b403cb3469e446d00194c448440e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 31 Mar 2023 14:09:21 +0200 Subject: [PATCH 153/233] Don't use CURRENT_WRITE_BUFFER_IS_EXHAUSTED for expected behaviour --- src/IO/CascadeWriteBuffer.cpp | 7 ++++--- src/IO/CascadeWriteBuffer.h | 2 +- src/IO/MemoryReadWriteBuffer.cpp | 8 +------- src/IO/MemoryReadWriteBuffer.h | 6 ++++++ 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/IO/CascadeWriteBuffer.cpp b/src/IO/CascadeWriteBuffer.cpp index 629cbff90af..f0d98027609 100644 --- a/src/IO/CascadeWriteBuffer.cpp +++ b/src/IO/CascadeWriteBuffer.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB @@ -35,9 +36,9 @@ void CascadeWriteBuffer::nextImpl() curr_buffer->position() = position(); curr_buffer->next(); } - catch (const Exception & e) + catch (const MemoryWriteBuffer::CurrentBufferExhausted &) { - if (curr_buffer_num < num_sources && e.code() == ErrorCodes::CURRENT_WRITE_BUFFER_IS_EXHAUSTED) + if (curr_buffer_num < num_sources) { /// TODO: protocol should require set(position(), 0) before Exception @@ -46,7 +47,7 @@ void CascadeWriteBuffer::nextImpl() curr_buffer = setNextBuffer(); } else - throw; + throw Exception(ErrorCodes::CURRENT_WRITE_BUFFER_IS_EXHAUSTED, "MemoryWriteBuffer limit is exhausted"); } set(curr_buffer->position(), curr_buffer->buffer().end() - curr_buffer->position()); diff --git a/src/IO/CascadeWriteBuffer.h b/src/IO/CascadeWriteBuffer.h index ebd4f262aa2..1059c5b8ddb 100644 --- a/src/IO/CascadeWriteBuffer.h +++ b/src/IO/CascadeWriteBuffer.h @@ -16,7 +16,7 @@ namespace ErrorCodes * (lazy_sources contains not pointers themself, but their delayed constructors) * * Firtly, CascadeWriteBuffer redirects data to first buffer of the sequence - * If current WriteBuffer cannot receive data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body, + * If current WriteBuffer cannot receive data anymore, it throws special exception MemoryWriteBuffer::CurrentBufferExhausted in nextImpl() body, * CascadeWriteBuffer prepare next buffer and continuously redirects data to it. * If there are no buffers anymore CascadeWriteBuffer throws an exception. * diff --git a/src/IO/MemoryReadWriteBuffer.cpp b/src/IO/MemoryReadWriteBuffer.cpp index 93ce5ce7ce9..d6f89108561 100644 --- a/src/IO/MemoryReadWriteBuffer.cpp +++ b/src/IO/MemoryReadWriteBuffer.cpp @@ -5,12 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CURRENT_WRITE_BUFFER_IS_EXHAUSTED; -} - - class ReadBufferFromMemoryWriteBuffer : public ReadBuffer, boost::noncopyable, private Allocator { public: @@ -118,7 +112,7 @@ void MemoryWriteBuffer::addChunk() if (0 == next_chunk_size) { set(position(), 0); - throw Exception(ErrorCodes::CURRENT_WRITE_BUFFER_IS_EXHAUSTED, "MemoryWriteBuffer limit is exhausted"); + throw MemoryWriteBuffer::CurrentBufferExhausted(); } } diff --git a/src/IO/MemoryReadWriteBuffer.h b/src/IO/MemoryReadWriteBuffer.h index bcaf9a9a965..ee128c355c6 100644 --- a/src/IO/MemoryReadWriteBuffer.h +++ b/src/IO/MemoryReadWriteBuffer.h @@ -16,6 +16,12 @@ namespace DB class MemoryWriteBuffer : public WriteBuffer, public IReadableWriteBuffer, boost::noncopyable, private Allocator { public: + /// Special exception to throw when the current WriteBuffer cannot receive data + class CurrentBufferExhausted : public std::exception + { + public: + const char * what() const noexcept override { return "MemoryWriteBuffer limit is exhausted"; } + }; /// Use max_total_size_ = 0 for unlimited storage explicit MemoryWriteBuffer( From 5206d4b9265c280415a7546d6f105bd45ccdca17 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 31 Mar 2023 14:12:55 +0200 Subject: [PATCH 154/233] Randomize compression of marks and indices in tests --- tests/clickhouse-test | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a355c2f8e73..6b7745fd32d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -589,6 +589,10 @@ class MergeTreeSettingsRandomizer: "merge_max_block_size": lambda: random.randint(1, 8192 * 3), "index_granularity": lambda: random.randint(1, 65536), "min_bytes_for_wide_part": threshold_generator(0.3, 0.3, 0, 1024 * 1024 * 1024), + "compress_marks": lambda: random.randint(0, 1), + "compress_primary_key": lambda: random.randint(0, 1), + "marks_compress_block_size": lambda: random.randint(8000, 100000), + "primary_key_compress_block_size": lambda: random.randint(8000, 100000), } @staticmethod From d158b187035a90b2f0296098a5be90a78649d360 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 31 Mar 2023 14:23:30 +0200 Subject: [PATCH 155/233] Randomize allow_vertical_merges_from_compact_to_wide_parts in tests --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a355c2f8e73..b95d0e60aee 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -582,6 +582,7 @@ class MergeTreeSettingsRandomizer: "vertical_merge_algorithm_min_columns_to_activate": threshold_generator( 0.4, 0.4, 1, 100 ), + "allow_vertical_merges_from_compact_to_wide_parts": lambda: random.randint(0, 1), "min_merge_bytes_to_use_direct_io": threshold_generator( 0.25, 0.25, 1, 10 * 1024 * 1024 * 1024 ), From 7c3c5ad6bba822722c511903362afe74725257d6 Mon Sep 17 00:00:00 2001 From: natasha Date: Fri, 31 Mar 2023 13:34:28 +0100 Subject: [PATCH 156/233] fix the test --- .../0_stateless/01318_alter_add_constraint_format.reference | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01318_alter_add_constraint_format.reference b/tests/queries/0_stateless/01318_alter_add_constraint_format.reference index 7a3b41536e0..9f58d161539 100644 --- a/tests/queries/0_stateless/01318_alter_add_constraint_format.reference +++ b/tests/queries/0_stateless/01318_alter_add_constraint_format.reference @@ -1,2 +1 @@ -ALTER TABLE replicated_constraints1 - ADD CONSTRAINT IF NOT EXISTS b_constraint CHECK b > 10 +ALTER TABLE replicated_constraints1 ADD CONSTRAINT IF NOT EXISTS b_constraint CHECK b > 10 From bae92ddf4e4898d001b19cc611314872c42d3f21 Mon Sep 17 00:00:00 2001 From: natasha Date: Fri, 31 Mar 2023 13:34:55 +0100 Subject: [PATCH 157/233] fix the code, single line formatter was forgotten --- src/Parsers/ASTAlterQuery.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 426b63a9d28..57bdcffdf75 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -612,7 +612,10 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; frame_nested.expression_list_always_start_on_new_line = true; - static_cast(command_list)->formatImplMultiline(settings, state, frame_nested); + frame_nested.expression_list_prepend_whitespace = true; + settings.one_line + ? command_list->formatImpl(settings, state, frame_nested) + : command_list->as().formatImplMultiline(settings, state, frame_nested); } } From 3b5f19394974db3e83ed3d356193867d5ff24293 Mon Sep 17 00:00:00 2001 From: natasha Date: Fri, 31 Mar 2023 13:34:28 +0100 Subject: [PATCH 158/233] fix the test --- .../0_stateless/01318_alter_add_constraint_format.reference | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01318_alter_add_constraint_format.reference b/tests/queries/0_stateless/01318_alter_add_constraint_format.reference index 7a3b41536e0..9f58d161539 100644 --- a/tests/queries/0_stateless/01318_alter_add_constraint_format.reference +++ b/tests/queries/0_stateless/01318_alter_add_constraint_format.reference @@ -1,2 +1 @@ -ALTER TABLE replicated_constraints1 - ADD CONSTRAINT IF NOT EXISTS b_constraint CHECK b > 10 +ALTER TABLE replicated_constraints1 ADD CONSTRAINT IF NOT EXISTS b_constraint CHECK b > 10 From 6caacd41d4416792823b2c996c66babfafd42f6b Mon Sep 17 00:00:00 2001 From: natasha Date: Fri, 31 Mar 2023 13:34:55 +0100 Subject: [PATCH 159/233] fix the code, single line formatter was forgotten --- src/Parsers/ASTAlterQuery.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 426b63a9d28..57bdcffdf75 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -612,7 +612,10 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; frame_nested.expression_list_always_start_on_new_line = true; - static_cast(command_list)->formatImplMultiline(settings, state, frame_nested); + frame_nested.expression_list_prepend_whitespace = true; + settings.one_line + ? command_list->formatImpl(settings, state, frame_nested) + : command_list->as().formatImplMultiline(settings, state, frame_nested); } } From 84c30f2f632977a55b0a10b3adbb2ace37565cdb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 31 Mar 2023 14:49:53 +0200 Subject: [PATCH 160/233] I've got no style --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b95d0e60aee..42676d900a5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -582,7 +582,9 @@ class MergeTreeSettingsRandomizer: "vertical_merge_algorithm_min_columns_to_activate": threshold_generator( 0.4, 0.4, 1, 100 ), - "allow_vertical_merges_from_compact_to_wide_parts": lambda: random.randint(0, 1), + "allow_vertical_merges_from_compact_to_wide_parts": lambda: random.randint( + 0, 1 + ), "min_merge_bytes_to_use_direct_io": threshold_generator( 0.25, 0.25, 1, 10 * 1024 * 1024 * 1024 ), From 644d83653d4a1b520b6e43bf504c47904cf89766 Mon Sep 17 00:00:00 2001 From: natasha Date: Fri, 31 Mar 2023 14:10:16 +0100 Subject: [PATCH 161/233] only set the necessary FormatStateStacked settings --- src/Parsers/ASTAlterQuery.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 57bdcffdf75..61e5903fad5 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -611,11 +611,16 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; - frame_nested.expression_list_always_start_on_new_line = true; - frame_nested.expression_list_prepend_whitespace = true; - settings.one_line - ? command_list->formatImpl(settings, state, frame_nested) - : command_list->as().formatImplMultiline(settings, state, frame_nested); + if (settings.one_line) + { + frame_nested.expression_list_prepend_whitespace = true; + command_list->formatImpl(settings, state, frame_nested); + } + else + { + frame_nested.expression_list_always_start_on_new_line = true; + command_list->as().formatImplMultiline(settings, state, frame_nested); + } } } From 2a179aed4952383067544a65b8df8b1e97b7c931 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 31 Mar 2023 13:35:07 +0000 Subject: [PATCH 162/233] Fix usage common::addOverflow for floats in AggregateFunctionSparkbar --- .../AggregateFunctionSparkbar.h | 38 ++++++++++++++----- 1 file changed, 29 insertions(+), 9 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index adbcea224d5..78f7e9fcefa 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -46,9 +46,17 @@ struct AggregateFunctionSparkbarData auto [it, inserted] = points.insert({x, y}); if (!inserted) { - Y res; - bool has_overfllow = common::addOverflow(it->getMapped(), y, res) - it->getMapped() = has_overfllow ? std::numeric_limits::max() : res; + if constexpr (std::is_floating_point_v) + { + it->getMapped() += y; + return it->getMapped(); + } + else + { + Y res; + bool has_overfllow = common::addOverflow(it->getMapped(), y, res); + it->getMapped() = has_overfllow ? std::numeric_limits::max() : res; + } } return it->getMapped(); } @@ -184,7 +192,12 @@ private: size_t index = std::min(static_cast(w / delta * value), histogram.size() - 1); Y res; - bool has_overfllow = common::addOverflow(histogram[index], point.getMapped(), res); + bool has_overfllow = false; + if constexpr (std::is_floating_point_v) + res = histogram[index] + point.getMapped(); + else + has_overfllow = common::addOverflow(histogram[index], point.getMapped(), res); + if (unlikely(has_overfllow)) { /// In case of overflow, just saturate @@ -229,13 +242,20 @@ private: } constexpr auto levels_num = static_cast(BAR_LEVELS - 1); - /// handle potential overflow - Y scaled; - bool has_overfllow = common::mulOverflow(y, levels_num, scaled); - if (has_overfllow) + if constexpr (std::is_floating_point_v) + { y = y / (y_max / levels_num) + 1; + } else - y = scaled / y_max + 1; + { + Y scaled; + bool has_overfllow = common::mulOverflow(y, levels_num, scaled); + + if (has_overfllow) + y = y / (y_max / levels_num) + 1; + else + y = scaled / y_max + 1; + } } size_t sz = 0; From f715bd95f1c301eeac5b10742c8e0c1d30f7b886 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 31 Mar 2023 14:08:28 +0000 Subject: [PATCH 163/233] fix writing to StorageS3 --- src/Storages/StorageS3.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 079289716a2..e24badbfd07 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -880,12 +880,12 @@ public: private: const String format; const Block sample_block; - ContextPtr context; + const ContextPtr context; const CompressionMethod compression_method; - const StorageS3::Configuration & s3_configuration; + const StorageS3::Configuration s3_configuration; const String bucket; const String key; - std::optional format_settings; + const std::optional format_settings; ExpressionActionsPtr partition_by_expr; From 75ffc8451b5576a68bc68977d8ab98000e14eac1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 16:09:00 +0200 Subject: [PATCH 164/233] fix --- src/Parsers/SyncReplicaMode.h | 3 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 39 +++++++++++-------- .../MergeTree/ReplicatedMergeTreeQueue.h | 5 +-- src/Storages/StorageReplicatedMergeTree.cpp | 6 +-- .../02438_sync_replica_lightweight.sql | 7 +++- 5 files changed, 33 insertions(+), 27 deletions(-) diff --git a/src/Parsers/SyncReplicaMode.h b/src/Parsers/SyncReplicaMode.h index a98e1cace50..6b19eb57fdc 100644 --- a/src/Parsers/SyncReplicaMode.h +++ b/src/Parsers/SyncReplicaMode.h @@ -1,8 +1,9 @@ #pragma once +#include namespace DB { -enum class SyncReplicaMode +enum class SyncReplicaMode : uint8_t { DEFAULT, STRICT, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 7cc26c00098..34cdefc99e0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2463,11 +2463,33 @@ String ReplicatedMergeTreeMergePredicate::getCoveringVirtualPart(const String & ReplicatedMergeTreeQueue::SubscriberHandler -ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback) +ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback, + std::unordered_set & out_entry_names, SyncReplicaMode sync_mode) { std::lock_guard lock(state_mutex); std::lock_guard lock_subscribers(subscribers_mutex); + if (sync_mode != SyncReplicaMode::PULL) + { + /// We must get the list of entries to wait atomically with adding the callback + bool lightweight_entries_only = sync_mode == SyncReplicaMode::LIGHTWEIGHT; + static constexpr std::array lightweight_entries = + { + LogEntry::GET_PART, + LogEntry::ATTACH_PART, + LogEntry::DROP_RANGE, + LogEntry::REPLACE_RANGE, + LogEntry::DROP_PART + }; + out_entry_names.reserve(queue.size()); + for (const auto & entry : queue) + { + if (!lightweight_entries_only + || std::find(lightweight_entries.begin(), lightweight_entries.end(), entry->type) != lightweight_entries.end()) + out_entry_names.insert(entry->znode_name); + } + } + auto it = subscribers.emplace(subscribers.end(), std::move(callback)); /// Atomically notify about current size @@ -2476,21 +2498,6 @@ ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCall return SubscriberHandler(it, *this); } -std::unordered_set ReplicatedMergeTreeQueue::getEntryNamesSet(bool lightweight_entries_only) -{ - std::lock_guard lock(state_mutex); - std::unordered_set result; - result.reserve(queue.size()); - for (const auto & entry : queue) - { - bool is_lightweight = entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART - || entry->type == LogEntry::DROP_RANGE || entry->type == LogEntry::REPLACE_RANGE || entry->type == LogEntry::DROP_PART; - if (!lightweight_entries_only || is_lightweight) - result.insert(entry->znode_name); - } - return result; -} - void ReplicatedMergeTreeQueue::notifySubscribersOnPartialShutdown() { size_t queue_size; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 9b29c6cef5b..841c26bb361 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -425,9 +426,7 @@ public: ActionBlocker pull_log_blocker; /// Adds a subscriber - SubscriberHandler addSubscriber(SubscriberCallBack && callback); - - std::unordered_set getEntryNamesSet(bool lightweight_entries_only); + SubscriberHandler addSubscriber(SubscriberCallBack && callback, std::unordered_set & out_entry_names, SyncReplicaMode sync_mode); void notifySubscribersOnPartialShutdown(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1450b08a4fb..cbfe3f8cab2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7593,10 +7593,6 @@ bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_millisec std::unordered_set wait_for_ids; bool was_interrupted = false; - if (sync_mode == SyncReplicaMode::DEFAULT) - wait_for_ids = queue.getEntryNamesSet(/* lightweight_entries_only */ false); - else if (sync_mode == SyncReplicaMode::LIGHTWEIGHT) - wait_for_ids = queue.getEntryNamesSet(/* lightweight_entries_only */ true); Poco::Event target_entry_event; auto callback = [this, &target_entry_event, &wait_for_ids, &was_interrupted, sync_mode] @@ -7624,7 +7620,7 @@ bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_millisec if (wait_for_ids.empty()) target_entry_event.set(); }; - const auto handler = queue.addSubscriber(std::move(callback)); + const auto handler = queue.addSubscriber(std::move(callback), wait_for_ids, sync_mode); if (!target_entry_event.tryWait(max_wait_milliseconds)) return false; diff --git a/tests/queries/0_stateless/02438_sync_replica_lightweight.sql b/tests/queries/0_stateless/02438_sync_replica_lightweight.sql index 207571da147..1da48d95d9b 100644 --- a/tests/queries/0_stateless/02438_sync_replica_lightweight.sql +++ b/tests/queries/0_stateless/02438_sync_replica_lightweight.sql @@ -7,10 +7,12 @@ create table rmt2 (n int) engine=ReplicatedMergeTree('/test/{database}/02438/', system stop replicated sends rmt1; system stop merges rmt2; +set insert_keeper_fault_injection_probability=0; + insert into rmt1 values (1); insert into rmt1 values (2); system sync replica rmt2 pull; -- does not wait -select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2'; +select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2' order by new_part_name; select 1, n, _part from rmt1 order by n; select 2, n, _part from rmt2 order by n; @@ -20,7 +22,7 @@ optimize table rmt1 final; system start replicated sends rmt1; system sync replica rmt2 lightweight; -- waits for fetches, not merges -select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2'; +select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2' order by new_part_name; select 3, n, _part from rmt1 order by n; select 4, n, _part from rmt2 order by n; @@ -28,6 +30,7 @@ system start merges rmt2; system sync replica rmt2; insert into rmt2 values (3); +system sync replica rmt2 pull; optimize table rmt2 final; system sync replica rmt1 strict; From e1d299640b77c91ec71d41a7a6eea8c7975c761d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 31 Mar 2023 13:53:52 +0000 Subject: [PATCH 165/233] Small follow-up to #48017 --- docs/en/sql-reference/statements/show.md | 6 +++--- src/Parsers/ASTShowColumnsQuery.cpp | 9 ++++----- src/Parsers/ParserShowColumnsQuery.cpp | 6 +++--- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index c0ab51ae444..f6d6d51b123 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -219,9 +219,9 @@ The optional keyword `FULL` causes the output to include the collation, comment - key - `PRI` if the column is part of the primary key, `SOR` if the column is part of the sorting key, empty otherwise (String) - default - Default expression of the column if it is of type `ALIAS`, `DEFAULT`, or `MATERIALIZED`, otherwise `NULL`. (Nullable(String)) - extra - Additional information, currently unused (String) -- collation - Collation of the column, always `NULL` because ClickHouse has no per-column collations, only if `FULL` keyword was specified (Nullable(String)) -- comment - Comment on the column, only if `FULL` keyword was specified (String) -- privilege - The privilege you have on this column, currently not available, only if `FULL` keyword was specified (String) +- collation - (only if `FULL` keyword was specified) Collation of the column, always `NULL` because ClickHouse has no per-column collations (Nullable(String)) +- comment - (only if `FULL` keyword was specified) Comment on the column (String) +- privilege - (only if `FULL` keyword was specified) The privilege you have on this column, currently not available (String) **Examples** diff --git a/src/Parsers/ASTShowColumnsQuery.cpp b/src/Parsers/ASTShowColumnsQuery.cpp index d14cbdc9b84..1cb4b651f0a 100644 --- a/src/Parsers/ASTShowColumnsQuery.cpp +++ b/src/Parsers/ASTShowColumnsQuery.cpp @@ -24,15 +24,14 @@ void ASTShowColumnsQuery::formatQueryImpl(const FormatSettings & settings, Forma << "COLUMNS" << (settings.hilite ? hilite_none : ""); - if (from_database.empty()) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from_table); - else - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from_database) << "." << backQuoteIfNeed(from_table); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from_table); + if (!from_database.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from_database); if (!like.empty()) settings.ostr << (settings.hilite ? hilite_keyword : "") - << (not_like ? " NOT " : "") + << (not_like ? " NOT" : "") << (case_insensitive_like ? " ILIKE " : " LIKE") << (settings.hilite ? hilite_none : "") << DB::quote << like; diff --git a/src/Parsers/ParserShowColumnsQuery.cpp b/src/Parsers/ParserShowColumnsQuery.cpp index 1db31601437..03e66e4ae0f 100644 --- a/src/Parsers/ParserShowColumnsQuery.cpp +++ b/src/Parsers/ParserShowColumnsQuery.cpp @@ -13,7 +13,7 @@ namespace DB bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr like; - ASTPtr from_db; + ASTPtr from_database; ASTPtr from_table; auto query = std::make_shared(); @@ -43,10 +43,10 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!abbreviated_form) if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected)) - if (!ParserIdentifier().parse(pos, from_db, expected)) + if (!ParserIdentifier().parse(pos, from_database, expected)) return false; - tryGetIdentifierNameInto(from_db, query->from_database); + tryGetIdentifierNameInto(from_database, query->from_database); if (ParserKeyword("NOT").ignore(pos, expected)) query->not_like = true; From 9094772712b71fb80638e28e0e96c7bd5a11d6d0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 31 Mar 2023 14:22:10 +0000 Subject: [PATCH 166/233] Replace manual checking by validateFunctionArgumentTypes() --- src/Functions/parseDateTime.cpp | 32 +++++++------------------------- 1 file changed, 7 insertions(+), 25 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 785c423f3ea..8f1edc2049b 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -480,33 +480,15 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() != 2 && arguments.size() != 3) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", - getName(), - arguments.size()); + FunctionArgumentDescriptors args{ + {"time", &isString, nullptr, "String"}, + {"format", &isString, nullptr, "String"}, + }; - if (!isString(arguments[0].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {}. Should be String", - arguments[0].type->getName(), - getName()); + if (arguments.size() == 3) + args.emplace_back(FunctionArgumentDescriptor{"timezone", &isString, nullptr, "String"}); - if (!isString(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}. Should be String", - arguments[0].type->getName(), - getName()); - - if (arguments.size() == 3 && !isString(arguments[2].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {}. Should be String", - arguments[0].type->getName(), - getName()); + validateFunctionArgumentTypes(*this, arguments, args); String time_zone_name = getTimeZone(arguments).getTimeZone(); DataTypePtr date_type = std::make_shared(time_zone_name); From 1e3abc9e84cfd661f30232e29d200b20999a0d6e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 31 Mar 2023 13:29:20 +0000 Subject: [PATCH 167/233] Add strict mode for KeeperMap --- .../table-engines/special/keepermap.md | 5 +- src/Core/Settings.h | 1 + src/Interpreters/MutationsInterpreter.cpp | 9 ++ src/Storages/StorageKeeperMap.cpp | 137 +++++++++++++++--- src/Storages/StorageKeeperMap.h | 4 +- .../02706_keeper_map_insert_strict.reference | 3 + .../02706_keeper_map_insert_strict.sql | 20 +++ 7 files changed, 158 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02706_keeper_map_insert_strict.reference create mode 100644 tests/queries/0_stateless/02706_keeper_map_insert_strict.sql diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index 680413039e7..e5c4dea2339 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -78,7 +78,8 @@ Of course, it's possible to manually run `CREATE TABLE` with same path on nonrel ### Inserts -When new rows are inserted into `KeeperMap`, if the key already exists, the value will be updated, otherwise new key is created. +When new rows are inserted into `KeeperMap`, if the key does not exist, a new entry for the key is created. +If the key exists, and setting `keeper_map_strict_mode` is set to `true`, an exception is thrown, otherwise, the value for the key is overwritten. Example: @@ -89,6 +90,7 @@ INSERT INTO keeper_map_table VALUES ('some key', 1, 'value', 3.2); ### Deletes Rows can be deleted using `DELETE` query or `TRUNCATE`. +If the key exists, and setting `keeper_map_strict_mode` is set to `true`, fetching and deleting data will succeed only if it can be executed atomically. ```sql DELETE FROM keeper_map_table WHERE key LIKE 'some%' AND v1 > 1; @@ -105,6 +107,7 @@ TRUNCATE TABLE keeper_map_table; ### Updates Values can be updated using `ALTER TABLE` query. Primary key cannot be updated. +If setting `keeper_map_strict_mode` is set to `true`, fetching and updating data will succeed only if it's executed atomically. ```sql ALTER TABLE keeper_map_table UPDATE v1 = v1 * 10 + 2 WHERE key LIKE 'some%' AND v3 > 3.1; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e9db155fb12..d985f51eec0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -722,6 +722,7 @@ class IColumn; M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ M(UInt64, http_max_request_param_data_size, 10_MiB, "Limit on size of request data used as a query parameter in predefined HTTP requests.", 0) \ M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ + M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0b52a1a51bc..d1fcc006ffb 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -550,6 +550,12 @@ void MutationsInterpreter::prepare(bool dry_run) if (source.hasLightweightDeleteMask()) all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); + if (return_all_columns) + { + for (const auto & column : source.getStorage()->getVirtuals()) + all_columns.push_back(column); + } + NameSet updated_columns; bool materialize_ttl_recalculate_only = source.materializeTTLRecalculateOnly(); @@ -906,6 +912,8 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s { auto storage_snapshot = source.getStorageSnapshot(metadata_snapshot, context); auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); + if (return_all_columns) + options.withVirtuals(); auto all_columns = storage_snapshot->getColumns(options); /// Add _row_exists column if it is present in the part @@ -1256,6 +1264,7 @@ void MutationsInterpreter::validate() } QueryPlan plan; + initQueryPlan(stages.front(), plan); auto pipeline = addStreamsForLaterStages(stages, plan); } diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f570f132463..58d02372f2a 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -59,6 +59,8 @@ namespace ErrorCodes namespace { +constexpr std::string_view version_column_name = "_version"; + std::string formattedAST(const ASTPtr & ast) { if (!ast) @@ -77,7 +79,6 @@ void verifyTableId(const StorageID & table_id) table_id.getDatabaseName(), database->getEngineName()); } - } } @@ -86,11 +87,13 @@ class StorageKeeperMapSink : public SinkToStorage { StorageKeeperMap & storage; std::unordered_map new_values; + std::unordered_map versions; size_t primary_key_pos; + ContextPtr context; public: - StorageKeeperMapSink(StorageKeeperMap & storage_, const StorageMetadataPtr & metadata_snapshot) - : SinkToStorage(metadata_snapshot->getSampleBlock()), storage(storage_) + StorageKeeperMapSink(StorageKeeperMap & storage_, Block header, ContextPtr context_) + : SinkToStorage(std::move(header)), storage(storage_), context(std::move(context_)) { auto primary_key = storage.getPrimaryKey(); assert(primary_key.size() == 1); @@ -113,18 +116,36 @@ public: wb_value.restart(); size_t idx = 0; + + int32_t version = -1; for (const auto & elem : block) { + if (elem.name == version_column_name) + { + version = assert_cast &>(*elem.column).getData()[i]; + continue; + } + elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); ++idx; } auto key = base64Encode(wb_key.str(), /* url_encoding */ true); + + if (version != -1) + versions[key] = version; + new_values[std::move(key)] = std::move(wb_value.str()); } } void onFinish() override + { + finalize(/*strict*/ context->getSettingsRef().keeper_map_strict_mode); + } + + template + void finalize(bool strict) { auto zookeeper = storage.getClient(); @@ -147,21 +168,39 @@ public: for (const auto & [key, _] : new_values) key_paths.push_back(storage.fullPathForKey(key)); - auto results = zookeeper->exists(key_paths); + zkutil::ZooKeeper::MultiExistsResponse results; + + if constexpr (!for_update) + results = zookeeper->exists(key_paths); Coordination::Requests requests; requests.reserve(key_paths.size()); for (size_t i = 0; i < key_paths.size(); ++i) { auto key = fs::path(key_paths[i]).filename(); - if (results[i].error == Coordination::Error::ZOK) + + if constexpr (for_update) { - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + int32_t version = -1; + if (strict) + version = versions.at(key); + + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], version)); } else { - requests.push_back(zkutil::makeCreateRequest(key_paths[i], new_values[key], zkutil::CreateMode::Persistent)); - ++new_keys_num; + if (results[i].error == Coordination::Error::ZOK) + { + if (strict) + throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Value for key '{}' already exists", key); + + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + } + else + { + requests.push_back(zkutil::makeCreateRequest(key_paths[i], new_values[key], zkutil::CreateMode::Persistent)); + ++new_keys_num; + } } } @@ -193,6 +232,18 @@ class StorageKeeperMapSource : public ISource KeyContainerIter it; KeyContainerIter end; + bool with_version_column = false; + + static Block getHeader(Block header, bool with_version_column) + { + if (with_version_column) + header.insert( + {DataTypeInt32{}.createColumn(), + std::make_shared(), std::string{version_column_name}}); + + return header; + } + public: StorageKeeperMapSource( const StorageKeeperMap & storage_, @@ -200,8 +251,10 @@ public: size_t max_block_size_, KeyContainerPtr container_, KeyContainerIter begin_, - KeyContainerIter end_) - : ISource(header), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_) + KeyContainerIter end_, + bool with_version_column_) + : ISource(getHeader(header, with_version_column_)), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_) + , with_version_column(with_version_column_) { } @@ -225,12 +278,12 @@ public: for (auto & raw_key : raw_keys) raw_key = base64Encode(raw_key, /* url_encoding */ true); - return storage.getBySerializedKeys(raw_keys, nullptr); + return storage.getBySerializedKeys(raw_keys, nullptr, with_version_column); } else { size_t elem_num = std::min(max_block_size, static_cast(end - it)); - auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr); + auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr, with_version_column); it += elem_num; return chunk; } @@ -426,6 +479,16 @@ Pipe StorageKeeperMap::read( auto primary_key_type = sample_block.getByName(primary_key).type; std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context_); + bool with_version_column = false; + for (const auto & column : column_names) + { + if (column == version_column_name) + { + with_version_column = true; + break; + } + } + const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { if (keys->empty()) @@ -449,7 +512,7 @@ Pipe StorageKeeperMap::read( using KeyContainer = typename KeyContainerPtr::element_type; pipes.emplace_back(std::make_shared>( - *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end)); + *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end, with_version_column)); } return Pipe::unitePipes(std::move(pipes)); }; @@ -461,10 +524,10 @@ Pipe StorageKeeperMap::read( return process_keys(std::move(filtered_keys)); } -SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { checkTable(); - return std::make_shared(*this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); } void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) @@ -554,6 +617,12 @@ void StorageKeeperMap::drop() dropTable(client, metadata_drop_lock); } +NamesAndTypesList StorageKeeperMap::getVirtuals() const +{ + return NamesAndTypesList{ + {std::string{version_column_name}, std::make_shared()}}; +} + zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const { std::lock_guard lock{zookeeper_mutex}; @@ -670,13 +739,18 @@ Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPOD if (raw_keys.size() != keys[0].column->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); - return getBySerializedKeys(raw_keys, &null_map); + return getBySerializedKeys(raw_keys, &null_map, /* version_column */ false); } -Chunk StorageKeeperMap::getBySerializedKeys(const std::span keys, PaddedPODArray * null_map) const +Chunk StorageKeeperMap::getBySerializedKeys(const std::span keys, PaddedPODArray * null_map, bool with_version) const { Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); MutableColumns columns = sample_block.cloneEmptyColumns(); + MutableColumnPtr version_column = nullptr; + + if (with_version) + version_column = ColumnVector::create(); + size_t primary_key_pos = getPrimaryKeyPos(sample_block, getPrimaryKey()); if (null_map) @@ -706,6 +780,9 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k if (code == Coordination::Error::ZOK) { fillColumns(base64Decode(keys[i], true), response.data, primary_key_pos, sample_block, columns); + + if (version_column) + version_column->insert(response.stat.version); } else if (code == Coordination::Error::ZNONODE) { @@ -714,6 +791,9 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k (*null_map)[i] = 0; for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx) columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault()); + + if (version_column) + version_column->insert(-1); } } else @@ -723,6 +803,10 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k } size_t num_rows = columns.at(0)->size(); + + if (version_column) + columns.push_back(std::move(version_column)); + return Chunk(std::move(columns), num_rows); } @@ -763,6 +847,8 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca if (commands.empty()) return; + bool strict = local_context->getSettingsRef().keeper_map_strict_mode; + assert(commands.size() == 1); auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -784,8 +870,10 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca auto header = interpreter->getUpdatedHeader(); auto primary_key_pos = header.getPositionByName(primary_key); + auto version_position = header.getPositionByName(std::string{version_column_name}); auto client = getClient(); + Block block; while (executor.pull(block)) { @@ -793,14 +881,23 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca auto column = column_type_name.column; auto size = column->size(); + WriteBufferFromOwnString wb_key; Coordination::Requests delete_requests; + for (size_t i = 0; i < size; ++i) { + int32_t version = -1; + if (strict) + { + const auto & version_column = block.getByPosition(version_position).column; + version = assert_cast &>(*version_column).getData()[i]; + } + wb_key.restart(); column_type_name.type->getDefaultSerialization()->serializeBinary(*column, i, wb_key, {}); - delete_requests.emplace_back(zkutil::makeRemoveRequest(fullPathForKey(base64Encode(wb_key.str(), true)), -1)); + delete_requests.emplace_back(zkutil::makeRemoveRequest(fullPathForKey(base64Encode(wb_key.str(), true)), version)); } Coordination::Responses responses; @@ -834,11 +931,13 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); - auto sink = std::make_shared(*this, metadata_snapshot); + auto sink = std::make_shared(*this, executor.getHeader(), local_context); Block block; while (executor.pull(block)) sink->consume(Chunk{block.getColumns(), block.rows()}); + + sink->finalize(local_context->getSettingsRef().keeper_map_strict_mode); sink->onFinish(); } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index a16c662e547..f71ff3cc65a 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -46,11 +46,13 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; void drop() override; + NamesAndTypesList getVirtuals() const override; + std::string getName() const override { return "KeeperMap"; } Names getPrimaryKey() const override { return {primary_key}; } Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const override; - Chunk getBySerializedKeys(std::span keys, PaddedPODArray * null_map) const; + Chunk getBySerializedKeys(std::span keys, PaddedPODArray * null_map, bool with_version) const; Block getSampleBlock(const Names &) const override; diff --git a/tests/queries/0_stateless/02706_keeper_map_insert_strict.reference b/tests/queries/0_stateless/02706_keeper_map_insert_strict.reference new file mode 100644 index 00000000000..a6bdbb192e4 --- /dev/null +++ b/tests/queries/0_stateless/02706_keeper_map_insert_strict.reference @@ -0,0 +1,3 @@ +1 1.1 +1 2.1 +1 2.1 diff --git a/tests/queries/0_stateless/02706_keeper_map_insert_strict.sql b/tests/queries/0_stateless/02706_keeper_map_insert_strict.sql new file mode 100644 index 00000000000..97c801ec46e --- /dev/null +++ b/tests/queries/0_stateless/02706_keeper_map_insert_strict.sql @@ -0,0 +1,20 @@ +-- Tags: no-ordinary-database, no-fasttest + +DROP TABLE IF EXISTS 02706_keeper_map_insert_strict SYNC; + +CREATE TABLE 02706_keeper_map_insert_strict (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test_02706_keeper_map_insert_strict') PRIMARY KEY(key); + +INSERT INTO 02706_keeper_map_insert_strict VALUES (1, 1.1), (2, 2.2); +SELECT * FROM 02706_keeper_map_insert_strict WHERE key = 1; + +SET keeper_map_strict_mode = false; + +INSERT INTO 02706_keeper_map_insert_strict VALUES (1, 2.1); +SELECT * FROM 02706_keeper_map_insert_strict WHERE key = 1; + +SET keeper_map_strict_mode = true; + +INSERT INTO 02706_keeper_map_insert_strict VALUES (1, 2.1); -- { serverError KEEPER_EXCEPTION } +SELECT * FROM 02706_keeper_map_insert_strict WHERE key = 1; + +DROP TABLE 02706_keeper_map_insert_strict; From 24eec7039e314b8d430aa191047134d7fff071c4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 16:56:01 +0200 Subject: [PATCH 168/233] fix --- ...{25402_show_columns.reference => 02706_show_columns.reference} | 0 .../{25402_show_columns.sql => 02706_show_columns.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25402_show_columns.reference => 02706_show_columns.reference} (100%) rename tests/queries/0_stateless/{25402_show_columns.sql => 02706_show_columns.sql} (100%) diff --git a/tests/queries/0_stateless/25402_show_columns.reference b/tests/queries/0_stateless/02706_show_columns.reference similarity index 100% rename from tests/queries/0_stateless/25402_show_columns.reference rename to tests/queries/0_stateless/02706_show_columns.reference diff --git a/tests/queries/0_stateless/25402_show_columns.sql b/tests/queries/0_stateless/02706_show_columns.sql similarity index 100% rename from tests/queries/0_stateless/25402_show_columns.sql rename to tests/queries/0_stateless/02706_show_columns.sql From ae0707ba01a9020fbc41e1526a5b2dc8d79d6593 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 31 Mar 2023 17:23:42 +0200 Subject: [PATCH 169/233] Remove unused error codes --- src/Functions/parseDateTime.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 8f1edc2049b..9d8db4d43a3 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -20,9 +20,7 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; From 048ea93c09bc4fff737c79f635de4b1e1353d36b Mon Sep 17 00:00:00 2001 From: kst-morozov Date: Fri, 31 Mar 2023 17:59:19 +0200 Subject: [PATCH 170/233] upadte doc --- .../settings.md | 2 +- .../settings/merge-tree-settings.md | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 08be318f334..3d579b14366 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1045,7 +1045,7 @@ Default value: `0`. Sets the number of threads performing background merges and mutations for tables with MergeTree engines. This setting is also could be applied at server startup from the `default` profile configuration for backward compatibility at the ClickHouse server start. You can only increase the number of threads at runtime. To lower the number of threads you have to restart the server. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. -Before changing it, please also take a look at related MergeTree settings, such as `number_of_free_entries_in_pool_to_lower_max_size_of_merge` and `number_of_free_entries_in_pool_to_execute_mutation`. +Before changing it, please also take a look at related MergeTree settings, such as [number_of_free_entries_in_pool_to_lower_max_size_of_merge](../../operations/settings/merge-tree-settings.md#number-of-free-entries-in-pool-to-lower-max-size-of-merge) and [number_of_free_entries_in_pool_to_execute_mutation](../../operations/settings/merge-tree-settings.md#number-of-free-entries-in-pool-to-execute-mutation). Possible values: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 5bc174727ad..f36694cb81b 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -553,6 +553,32 @@ Default value: 8192 Merge reads rows from parts in blocks of `merge_max_block_size` rows, then merges and writes the result into a new part. The read block is placed in RAM, so `merge_max_block_size` affects the size of the RAM required for the merge. Thus, merges can consume a large amount of RAM for tables with very wide rows (if the average row size is 100kb, then when merging 10 parts, (100kb * 10 * 8192) = ~ 8GB of RAM). By decreasing `merge_max_block_size`, you can reduce the amount of RAM required for a merge but slow down a merge. +## number_of_free_entries_in_pool_to_lower_max_size_of_merge {#number-of-free-entries-in-pool-to-lower-max-size-of-merge} + +When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). +This is to allow small merges to process - not filling the pool with long running merges. + +Possible values: + +- Any positive integer. + +Default value: 8 + +## number_of_free_entries_in_pool_to_execute_mutation {#number-of-free-entries-in-pool-to-execute-mutation} + +When there is less than specified number of free entries in pool, do not execute part mutations. +This is to leave free threads for regular merges and avoid "Too many parts". + +Possible values: + +- Any positive integer. + +Default value: 20 + +**Usage** + +The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](../../operations/settings/server-configuration-parameters/settings.md#background_pool_size) * [background_pool_size](../../operations/settings/server-configuration-parameters/settings.md#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. + ## max_part_loading_threads {#max-part-loading-threads} The maximum number of threads that read parts when ClickHouse starts. From 8c65cc734ab3ba2abce5ce8fc234ca9532b75fa4 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 31 Mar 2023 10:32:38 -0600 Subject: [PATCH 171/233] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 161 +++++++++++++++++- 1 file changed, 153 insertions(+), 8 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index e3b162a8dbf..2609b8db852 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -67,7 +67,8 @@ CREATE TABLE youtube ( `id` String, `fetch_date` DateTime, - `upload_date` String, + `upload_date_str` String, + `upload_date` Date, `title` String, `uploader_id` String, `uploader` String, @@ -87,7 +88,7 @@ CREATE TABLE youtube `video_badges` String ) ENGINE = MergeTree -ORDER BY (upload_date, uploader); +ORDER BY (uploader, upload_date); ``` 3. The following command streams the records from the S3 files into the `youtube` table. @@ -101,8 +102,9 @@ INSERT INTO youtube SETTINGS input_format_null_as_default = 1 SELECT id, - parseDateTimeBestEffortUS(toString(fetch_date)) AS fetch_date, - upload_date, + parseDateTimeBestEffortUSOrZero(toString(fetch_date)) AS fetch_date, + upload_date AS upload_date_str, + toDate(parseDateTimeBestEffortUS(upload_date::String)) AS upload_date, ifNull(title, '') AS title, uploader_id, ifNull(uploader, '') AS uploader, @@ -121,12 +123,23 @@ SELECT ifNull(uploader_badges, '') AS uploader_badges, ifNull(video_badges, '') AS video_badges FROM s3Cluster( - 'default', - 'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', - 'JSONLines' - ); + 'default', + 'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', + 'JSONLines' +) +SETTINGS + max_download_threads = 24, + max_insert_threads = 64, + max_insert_block_size = 100000000, + min_insert_block_size_rows = 100000000, + min_insert_block_size_bytes = 500000000; ``` +Some comments about our `INSERT` command: + +- The `parseDateTimeBestEffortUSOrZero` function is handy when the incoming date fields may not be in the proper format. If `fetch_date` does not get parsed properly, it will be set to `0` +- + 4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase. It will take a while to insert 4.56B rows, depending on your server resources. (Withtout any tweaking of settings, it takes about 4.5 hours.) ```sql @@ -276,6 +289,132 @@ ORDER BY Enabling comments seems to be correlated with a higher rate of engagement. +<<<<<<< Updated upstream +======= + +### How does the number of videos change over time - notable events? + +```sql +SELECT + toStartOfMonth(toDateTime(upload_date)) AS month, + uniq(uploader_id) AS uploaders, + count() as num_videos, + sum(view_count) as view_count +FROM youtube +WHERE (month >= '2005-01-01') AND (month < '2021-12-01') +GROUP BY month +ORDER BY month ASC +``` + +```response +┌──────month─┬─uploaders─┬─num_videos─┬───view_count─┐ +│ 2005-04-01 │ 5 │ 6 │ 213597737 │ +│ 2005-05-01 │ 6 │ 9 │ 2944005 │ +│ 2005-06-01 │ 165 │ 351 │ 18624981 │ +│ 2005-07-01 │ 395 │ 1168 │ 94164872 │ +│ 2005-08-01 │ 1171 │ 3128 │ 124540774 │ +│ 2005-09-01 │ 2418 │ 5206 │ 475536249 │ +│ 2005-10-01 │ 6750 │ 13747 │ 737593613 │ +│ 2005-11-01 │ 13706 │ 28078 │ 1896116976 │ +│ 2005-12-01 │ 24756 │ 49885 │ 2478418930 │ +│ 2006-01-01 │ 49992 │ 100447 │ 4532656581 │ +│ 2006-02-01 │ 67882 │ 138485 │ 5677516317 │ +│ 2006-03-01 │ 103358 │ 212237 │ 8430301366 │ +│ 2006-04-01 │ 114615 │ 234174 │ 9980760440 │ +│ 2006-05-01 │ 152682 │ 332076 │ 14129117212 │ +│ 2006-06-01 │ 193962 │ 429538 │ 17014143263 │ +│ 2006-07-01 │ 234401 │ 530311 │ 18721143410 │ +│ 2006-08-01 │ 281280 │ 614128 │ 20473502342 │ +│ 2006-09-01 │ 312434 │ 679906 │ 23158422265 │ +│ 2006-10-01 │ 404873 │ 897590 │ 27357846117 │ +``` + +A spike of uploaders [around covid is noticeable](https://www.theverge.com/2020/3/27/21197642/youtube-with-me-style-videos-views-coronavirus-cook-workout-study-home-beauty). + + +### More subtitiles over time and when + +With advances in speech recognition, it’s easier than ever to create subtitles for video with youtube adding auto-captioning in late 2009 - was the jump then? + +```sql +SELECT + toStartOfMonth(upload_date) AS month, + countIf(has_subtitles) / count() AS percent_subtitles, + percent_subtitles - any(percent_subtitles) OVER (ORDER BY month ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS previous +FROM youtube +WHERE (month >= '2015-01-01') AND (month < '2021-12-02') +GROUP BY month +ORDER BY month ASC +``` + +```response +┌──────month─┬───percent_subtitles─┬────────────────previous─┐ +│ 2015-01-01 │ 0.2652653881082824 │ 0.2652653881082824 │ +│ 2015-02-01 │ 0.3147556050309162 │ 0.049490216922633834 │ +│ 2015-03-01 │ 0.32460464492371877 │ 0.009849039892802558 │ +│ 2015-04-01 │ 0.33471963051468445 │ 0.010114985590965686 │ +│ 2015-05-01 │ 0.3168087575501062 │ -0.017910872964578273 │ +│ 2015-06-01 │ 0.3162609788438222 │ -0.0005477787062839745 │ +│ 2015-07-01 │ 0.31828767677518033 │ 0.0020266979313581235 │ +│ 2015-08-01 │ 0.3045551564286859 │ -0.013732520346494415 │ +│ 2015-09-01 │ 0.311221133995152 │ 0.006665977566466086 │ +│ 2015-10-01 │ 0.30574870926812175 │ -0.005472424727030245 │ +│ 2015-11-01 │ 0.31125409712077234 │ 0.0055053878526505895 │ +│ 2015-12-01 │ 0.3190967954651779 │ 0.007842698344405541 │ +│ 2016-01-01 │ 0.32636021432496176 │ 0.007263418859783877 │ + +``` + +The data results show a spike in 2009. Apparently at that, time YouTube was removing their community captions feature, which allowed you to upload captions for other people's video. +This prompted a very successful campaign to have creators add captions to their videos for hard of hearing and deaf viewers. + + +### Top uploaders over time + +```sql +WITH uploaders AS + ( + SELECT uploader + FROM youtube + GROUP BY uploader + ORDER BY sum(view_count) DESC + LIMIT 10 + ) +SELECT + month, + uploader, + sum(view_count) AS total_views, + avg(dislike_count / like_count) AS like_to_dislike_ratio +FROM youtube +WHERE uploader IN (uploaders) +GROUP BY + toStartOfMonth(upload_date) AS month, + uploader +ORDER BY + month ASC, + total_views DESC + +1001 rows in set. Elapsed: 34.917 sec. Processed 4.58 billion rows, 69.08 GB (131.15 million rows/s., 1.98 GB/s.) +``` + +```response +┌──────month─┬─uploader───────────────────┬─total_views─┬─like_to_dislike_ratio─┐ +│ 1970-01-01 │ T-Series │ 10957099 │ 0.022784656361208206 │ +│ 1970-01-01 │ Ryan's World │ 0 │ 0.003035559410234172 │ +│ 1970-01-01 │ SET India │ 0 │ nan │ +│ 2006-09-01 │ Cocomelon - Nursery Rhymes │ 256406497 │ 0.7005566715978622 │ +│ 2007-06-01 │ Cocomelon - Nursery Rhymes │ 33641320 │ 0.7088650914344298 │ +│ 2008-02-01 │ WWE │ 43733469 │ 0.07198856488734842 │ +│ 2008-03-01 │ WWE │ 16514541 │ 0.1230603715431997 │ +│ 2008-04-01 │ WWE │ 5907295 │ 0.2089399470159618 │ +│ 2008-05-01 │ WWE │ 7779627 │ 0.09101676560436774 │ +│ 2008-06-01 │ WWE │ 7018780 │ 0.0974184753155297 │ +│ 2008-07-01 │ WWE │ 4686447 │ 0.1263845422065158 │ +│ 2008-08-01 │ WWE │ 4514312 │ 0.08384574274791441 │ +│ 2008-09-01 │ WWE │ 3717092 │ 0.07872802579349912 │ +``` + +>>>>>>> Stashed changes ### How do like ratio changes as views go up? ```sql @@ -296,7 +435,13 @@ GROUP BY ORDER BY view_range ASC, is_comments_enabled ASC +<<<<<<< Updated upstream ); +======= +) + +20 rows in set. Elapsed: 9.043 sec. Processed 4.56 billion rows, 77.48 GB (503.99 million rows/s., 8.57 GB/s.) +>>>>>>> Stashed changes ``` ```response From 0b70a8600be22204c7489174b6407d3a0a97d859 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 31 Mar 2023 13:19:49 -0400 Subject: [PATCH 172/233] Update LVM info in tips.md --- docs/en/operations/tips.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 13353cd8e6a..c5b489e92c5 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -51,10 +51,14 @@ But for storing archives with rare queries, shelves will work. ## RAID {#raid} When using HDD, you can combine their RAID-10, RAID-5, RAID-6 or RAID-50. -For Linux, software RAID is better (with `mdadm`). We do not recommend using LVM. +For Linux, software RAID is better (with `mdadm`). When creating RAID-10, select the `far` layout. If your budget allows, choose RAID-10. +LVM by itself (without RAID or mdadm) is ok, but making RAID with it or combining it with mdadm is a less explored option, and there will be more chances for mistakes +(selecting wrong chunk size; misalignment of chunks; choosing a wrong raid type; forgetting to cleanup disks). If you are confident +in using LVM, there is nothing against using it. + If you have more than 4 disks, use RAID-6 (preferred) or RAID-50, instead of RAID-5. When using RAID-5, RAID-6 or RAID-50, always increase stripe_cache_size, since the default value is usually not the best choice. From 4cb3c9260771a31199f2d793480f7ac3fa3a42fa Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Fri, 31 Mar 2023 12:38:45 -0600 Subject: [PATCH 173/233] Update merge-tree-settings.md For some reason the links are broken - I'm attempting to fix them here --- docs/en/operations/settings/merge-tree-settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 41e66679874..4008b71ef8b 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -577,7 +577,7 @@ Default value: 20 **Usage** -The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](../../operations/settings/server-configuration-parameters/settings.md#background_pool_size) * [background_pool_size](../../operations/settings/server-configuration-parameters/settings.md#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. +The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](/docs/en/operations/server-configuration-parameters/settings#background_pool_size) * [background_pool_size](/docs/en/operations/server-configuration-parameters/settings#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. ## max_part_loading_threads {#max-part-loading-threads} From abe78469f78bc089f11cfbeaffae34591d5e1f2a Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 31 Mar 2023 12:41:22 -0600 Subject: [PATCH 174/233] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 44 ++++++++----------- 1 file changed, 19 insertions(+), 25 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 2609b8db852..c0a54906212 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -104,7 +104,7 @@ SELECT id, parseDateTimeBestEffortUSOrZero(toString(fetch_date)) AS fetch_date, upload_date AS upload_date_str, - toDate(parseDateTimeBestEffortUS(upload_date::String)) AS upload_date, + toDate(parseDateTimeBestEffortUSOrZero(upload_date::String)) AS upload_date, ifNull(title, '') AS title, uploader_id, ifNull(uploader, '') AS uploader, @@ -138,9 +138,17 @@ SETTINGS Some comments about our `INSERT` command: - The `parseDateTimeBestEffortUSOrZero` function is handy when the incoming date fields may not be in the proper format. If `fetch_date` does not get parsed properly, it will be set to `0` -- +- The `upload_date` column contains valid dates, but it also contains strings like "4 hours ago" - which is certainly not a valid date. We decided to store the original value in `upload_date_str` and attempt to parse it with `toDate(parseDateTimeBestEffortUSOrZero(upload_date::String))`. If the parsing fails we just get `0` +- We used `ifNull` to avoid getting `NULL` values in our table. If an incoming value is `NULL`, the `ifNull` function is setting the value to an empty string +- It takes a long time to download the data, so we added a `SETTINGS` clause to spread out the work over more threads while making sure the block sizes stayed fairly large -4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase. It will take a while to insert 4.56B rows, depending on your server resources. (Withtout any tweaking of settings, it takes about 4.5 hours.) +Here is the response when the data is fully loaded: + +```response + +``` + +4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase. It will take a while to insert 4.56B rows, depending on your server resources. (Without any tweaking of settings, it takes about 4.5 hours.) ```sql SELECT formatReadableQuantity(count()) @@ -237,7 +245,6 @@ The results look like: When commenting is disabled, are people more likely to like or dislike to express their feelings about a video? - ```sql SELECT concat('< ', formatReadableQuantity(view_range)) AS views, @@ -261,6 +268,7 @@ ORDER BY ``` ```response + ┌─views─────────────┬─is_comments_enabled─┬────prob_like_dislike─┐ │ < 10.00 │ false │ 0.08224180712685371 │ │ < 100.00 │ false │ 0.06346337759167248 │ @@ -285,12 +293,11 @@ ORDER BY └───────────────────┴─────────────────────┴──────────────────────┘ 22 rows in set. Elapsed: 8.460 sec. Processed 4.56 billion rows, 77.48 GB (538.73 million rows/s., 9.16 GB/s.) + ``` Enabling comments seems to be correlated with a higher rate of engagement. -<<<<<<< Updated upstream -======= ### How does the number of videos change over time - notable events? @@ -301,9 +308,8 @@ SELECT count() as num_videos, sum(view_count) as view_count FROM youtube -WHERE (month >= '2005-01-01') AND (month < '2021-12-01') GROUP BY month -ORDER BY month ASC +ORDER BY month ASC; ``` ```response @@ -340,11 +346,12 @@ With advances in speech recognition, it’s easier than ever to create subtitles SELECT toStartOfMonth(upload_date) AS month, countIf(has_subtitles) / count() AS percent_subtitles, - percent_subtitles - any(percent_subtitles) OVER (ORDER BY month ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS previous + percent_subtitles - any(percent_subtitles) OVER ( + ORDER BY month ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING + ) AS previous FROM youtube -WHERE (month >= '2015-01-01') AND (month < '2021-12-02') GROUP BY month -ORDER BY month ASC +ORDER BY month ASC; ``` ```response @@ -392,9 +399,7 @@ GROUP BY uploader ORDER BY month ASC, - total_views DESC - -1001 rows in set. Elapsed: 34.917 sec. Processed 4.58 billion rows, 69.08 GB (131.15 million rows/s., 1.98 GB/s.) + total_views DESC; ``` ```response @@ -414,7 +419,6 @@ ORDER BY │ 2008-09-01 │ WWE │ 3717092 │ 0.07872802579349912 │ ``` ->>>>>>> Stashed changes ### How do like ratio changes as views go up? ```sql @@ -435,13 +439,7 @@ GROUP BY ORDER BY view_range ASC, is_comments_enabled ASC -<<<<<<< Updated upstream ); -======= -) - -20 rows in set. Elapsed: 9.043 sec. Processed 4.56 billion rows, 77.48 GB (503.99 million rows/s., 8.57 GB/s.) ->>>>>>> Stashed changes ``` ```response @@ -467,8 +465,6 @@ ORDER BY │ < 10.00 billion │ false │ 1.77 │ │ < 10.00 billion │ true │ 19.5 │ └───────────────────┴─────────────────────┴────────────┘ - -20 rows in set. Elapsed: 63.664 sec. Processed 4.56 billion rows, 113.93 GB (71.59 million rows/s., 1.79 GB/s.) ``` ### How are views distributed? @@ -504,6 +500,4 @@ ARRAY JOIN │ 20th │ 16 │ │ 10th │ 6 │ └────────────┴─────────┘ - -12 rows in set. Elapsed: 1.864 sec. Processed 4.56 billion rows, 36.46 GB (2.45 billion rows/s., 19.56 GB/s.) ``` \ No newline at end of file From 1693e4664045ce44013ac43e465be49091424ca4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 31 Mar 2023 19:03:20 +0000 Subject: [PATCH 175/233] Do not remove inputs from maybe compiled DAG. --- src/Interpreters/ActionsDAG.cpp | 2 +- .../0_stateless/02705_grouping_keys_equal_keys.reference | 6 ++++++ .../queries/0_stateless/02705_grouping_keys_equal_keys.sql | 7 +++++++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02705_grouping_keys_equal_keys.reference create mode 100644 tests/queries/0_stateless/02705_grouping_keys_equal_keys.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index ad809dca022..fb6f3cda99d 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1051,7 +1051,7 @@ ActionsDAGPtr ActionsDAG::clone() const void ActionsDAG::compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes) { compileFunctions(min_count_to_compile_expression, lazy_executed_nodes); - removeUnusedActions(); + removeUnusedActions(/*allow_remove_inputs = */ false); } #endif diff --git a/tests/queries/0_stateless/02705_grouping_keys_equal_keys.reference b/tests/queries/0_stateless/02705_grouping_keys_equal_keys.reference new file mode 100644 index 00000000000..a9e2f17562a --- /dev/null +++ b/tests/queries/0_stateless/02705_grouping_keys_equal_keys.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02705_grouping_keys_equal_keys.sql b/tests/queries/0_stateless/02705_grouping_keys_equal_keys.sql new file mode 100644 index 00000000000..fcf5b4d2ce5 --- /dev/null +++ b/tests/queries/0_stateless/02705_grouping_keys_equal_keys.sql @@ -0,0 +1,7 @@ +SELECT count() +FROM numbers(2) +GROUP BY +GROUPING SETS ( + (number, number + 0, number + 1), + (number % 1048576, number % -9223372036854775808), + (number / 2, number / 2)); From 04513504be5d9bf6a19d791399a6a30bd69969c8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 31 Mar 2023 19:25:46 +0000 Subject: [PATCH 176/233] Update version_date.tsv and changelogs after v22.3.20.29-lts --- docs/changelogs/v22.3.20.29-lts.md | 29 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 30 insertions(+) create mode 100644 docs/changelogs/v22.3.20.29-lts.md diff --git a/docs/changelogs/v22.3.20.29-lts.md b/docs/changelogs/v22.3.20.29-lts.md new file mode 100644 index 00000000000..a54a320c4c1 --- /dev/null +++ b/docs/changelogs/v22.3.20.29-lts.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.3.20.29-lts (297b4dd5e55) FIXME as compared to v22.3.19.6-lts (467e0a7bd77) + +#### Improvement +* Backported in [#46979](https://github.com/ClickHouse/ClickHouse/issues/46979): Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix incorrect alias recursion in QueryNormalizer [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). +* Fix arithmetic operations in aggregate optimization [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). +* Fix MSan report in `maxIntersections` function [#46847](https://github.com/ClickHouse/ClickHouse/pull/46847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix wrong results of some LIKE searches when the LIKE pattern contains quoted non-quotable characters [#46875](https://github.com/ClickHouse/ClickHouse/pull/46875) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix possible deadlock in QueryStatus [#47161](https://github.com/ClickHouse/ClickHouse/pull/47161) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add a fuse for backport branches w/o a created PR [#47760](https://github.com/ClickHouse/ClickHouse/pull/47760) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Only valid Reviews.STATES overwrite existing reviews [#47789](https://github.com/ClickHouse/ClickHouse/pull/47789) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Place short return before big block, improve logging [#47822](https://github.com/ClickHouse/ClickHouse/pull/47822) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tsan error lock-order-inversion [#47953](https://github.com/ClickHouse/ClickHouse/pull/47953) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f46a422446e..f993c90d1b2 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -73,6 +73,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.20.29-lts 2023-03-31 v22.3.19.6-lts 2023-02-27 v22.3.18.37-lts 2023-02-15 v22.3.17.13-lts 2023-01-12 From 648f80ee4e403fecbf99f8273391c798c8cb7798 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 31 Mar 2023 19:50:47 +0000 Subject: [PATCH 177/233] Update version_date.tsv and changelogs after v22.12.6.22-stable --- docs/changelogs/v22.12.6.22-stable.md | 26 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 28 insertions(+) create mode 100644 docs/changelogs/v22.12.6.22-stable.md diff --git a/docs/changelogs/v22.12.6.22-stable.md b/docs/changelogs/v22.12.6.22-stable.md new file mode 100644 index 00000000000..f0bf7c92340 --- /dev/null +++ b/docs/changelogs/v22.12.6.22-stable.md @@ -0,0 +1,26 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.12.6.22-stable (10d87f90261) FIXME as compared to v22.12.5.34-stable (b82d6401ca1) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix changing an expired role [#46772](https://github.com/ClickHouse/ClickHouse/pull/46772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix bug in zero-copy replication disk choice during fetch [#47010](https://github.com/ClickHouse/ClickHouse/pull/47010) ([alesapin](https://github.com/alesapin)). +* Fix NOT_IMPLEMENTED error with CROSS JOIN and algorithm = auto [#47068](https://github.com/ClickHouse/ClickHouse/pull/47068) ([Vladimir C](https://github.com/vdimir)). +* Fix query parameters [#47488](https://github.com/ClickHouse/ClickHouse/pull/47488) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Hotfix for too verbose warnings in HTTP [#47903](https://github.com/ClickHouse/ClickHouse/pull/47903) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Better error messages in ReplicatedMergeTreeAttachThread [#47454](https://github.com/ClickHouse/ClickHouse/pull/47454) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a fuse for backport branches w/o a created PR [#47760](https://github.com/ClickHouse/ClickHouse/pull/47760) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Only valid Reviews.STATES overwrite existing reviews [#47789](https://github.com/ClickHouse/ClickHouse/pull/47789) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Place short return before big block, improve logging [#47822](https://github.com/ClickHouse/ClickHouse/pull/47822) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Artifacts s3 prefix [#47945](https://github.com/ClickHouse/ClickHouse/pull/47945) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tsan error lock-order-inversion [#47953](https://github.com/ClickHouse/ClickHouse/pull/47953) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f46a422446e..0cd3416f44c 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -8,6 +8,7 @@ v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 +v22.12.6.22-stable 2023-03-31 v22.12.5.34-stable 2023-03-10 v22.12.4.76-stable 2023-03-01 v22.12.3.5-stable 2023-01-10 @@ -73,6 +74,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.20.29-lts 2023-03-31 v22.3.19.6-lts 2023-02-27 v22.3.18.37-lts 2023-02-15 v22.3.17.13-lts 2023-01-12 From 3f4aadfe7dc0a1795ce55d256696b6756704dc34 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 31 Mar 2023 23:50:35 +0200 Subject: [PATCH 178/233] Add logging for concurrency checks for backups. --- src/Backups/BackupCoordinationLocal.cpp | 10 ++++++++-- src/Backups/BackupCoordinationLocal.h | 2 ++ src/Backups/BackupCoordinationRemote.cpp | 8 ++++++-- src/Backups/BackupCoordinationRemote.h | 1 + src/Backups/RestoreCoordinationLocal.cpp | 13 +++++++++++-- src/Backups/RestoreCoordinationLocal.h | 2 ++ src/Backups/RestoreCoordinationRemote.cpp | 6 +++++- src/Backups/RestoreCoordinationRemote.h | 1 + 8 files changed, 36 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index a96ed443b9c..f3a6c02d228 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -8,7 +8,8 @@ namespace DB { -BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) : file_infos(plain_backup_) +BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) + : log(&Poco::Logger::get("BackupCoordinationLocal")), file_infos(plain_backup_) { } @@ -125,7 +126,12 @@ bool BackupCoordinationLocal::startWritingFile(size_t data_file_index) bool BackupCoordinationLocal::hasConcurrentBackups(const std::atomic & num_active_backups) const { - return (num_active_backups > 1); + if (num_active_backups > 1) + { + LOG_WARNING(log, "Found concurrent backups: num_active_backups={}", num_active_backups); + return true; + } + return false; } } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index db2070fa891..60fcc014720 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -52,6 +52,8 @@ public: bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; private: + Poco::Logger * const log; + BackupCoordinationReplicatedTables TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables; BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access; BackupCoordinationReplicatedSQLObjects TSA_GUARDED_BY(replicated_sql_objects_mutex) replicated_sql_objects; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 9b4343a1d3b..d6463d08909 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -164,17 +164,18 @@ BackupCoordinationRemote::BackupCoordinationRemote( , current_host_index(findCurrentHostIndex(all_hosts, current_host)) , plain_backup(plain_backup_) , is_internal(is_internal_) + , log(&Poco::Logger::get("BackupCoordinationRemote")) { zookeeper_retries_info = ZooKeeperRetriesInfo( "BackupCoordinationRemote", - &Poco::Logger::get("BackupCoordinationRemote"), + log, keeper_settings.keeper_max_retries, keeper_settings.keeper_retry_initial_backoff_ms, keeper_settings.keeper_retry_max_backoff_ms); createRootNodes(); stage_sync.emplace( - zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination")); + zookeeper_path + "/stage", [this] { return getZooKeeper(); }, log); } BackupCoordinationRemote::~BackupCoordinationRemote() @@ -664,7 +665,10 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage"); if (status != Stage::COMPLETED) + { + LOG_WARNING(log, "Found a concurrent backup: {}, current backup: {}", existing_backup_uuid, toString(backup_uuid)); return true; + } } zk->createIfNotExists(backup_stage_path, ""); diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index e7f5ff3a211..5155f21c27a 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -104,6 +104,7 @@ private: const size_t current_host_index; const bool plain_backup; const bool is_internal; + Poco::Logger * const log; mutable ZooKeeperRetriesInfo zookeeper_retries_info; std::optional stage_sync; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 191cde40aa1..068c4fe7e52 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -1,10 +1,14 @@ #include +#include namespace DB { -RestoreCoordinationLocal::RestoreCoordinationLocal() = default; +RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("RestoreCoordinationLocal")) +{ +} + RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; void RestoreCoordinationLocal::setStage(const String &, const String &) @@ -49,7 +53,12 @@ bool RestoreCoordinationLocal::acquireReplicatedSQLObjects(const String &, UserD bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic & num_active_restores) const { - return (num_active_restores > 1); + if (num_active_restores > 1) + { + LOG_WARNING(log, "Found concurrent backups: num_active_restores={}", num_active_restores); + return true; + } + return false; } } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index bbe76cdf5fd..e27f0d1ef88 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -42,6 +42,8 @@ public: bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; private: + Poco::Logger * const log; + std::set> acquired_tables_in_replicated_databases; std::unordered_set acquired_data_in_replicated_tables; mutable std::mutex mutex; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 10d085a696a..d93f99a3f2a 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -25,11 +25,12 @@ RestoreCoordinationRemote::RestoreCoordinationRemote( , current_host(current_host_) , current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host)) , is_internal(is_internal_) + , log(&Poco::Logger::get("RestoreCoordinationRemote")) { createRootNodes(); stage_sync.emplace( - zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination")); + zookeeper_path + "/stage", [this] { return getZooKeeper(); }, log); } RestoreCoordinationRemote::~RestoreCoordinationRemote() @@ -197,7 +198,10 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic const auto status = zk->get(root_zookeeper_path + "/" + existing_restore_path + "/stage"); if (status != Stage::COMPLETED) + { + LOG_WARNING(log, "Found a concurrent restore: {}, current restore: {}", existing_restore_uuid, toString(restore_uuid)); return true; + } } zk->createIfNotExists(path, ""); diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index b78c2e96f9e..e524e42c440 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -59,6 +59,7 @@ private: const String current_host; const size_t current_host_index; const bool is_internal; + Poco::Logger * const log; std::optional stage_sync; From 3b6c5992377d00de0f0159960be090c6b957f060 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 31 Mar 2023 16:03:44 -0600 Subject: [PATCH 179/233] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index c0a54906212..ea12042c635 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -142,12 +142,6 @@ Some comments about our `INSERT` command: - We used `ifNull` to avoid getting `NULL` values in our table. If an incoming value is `NULL`, the `ifNull` function is setting the value to an empty string - It takes a long time to download the data, so we added a `SETTINGS` clause to spread out the work over more threads while making sure the block sizes stayed fairly large -Here is the response when the data is fully loaded: - -```response - -``` - 4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase. It will take a while to insert 4.56B rows, depending on your server resources. (Without any tweaking of settings, it takes about 4.5 hours.) ```sql @@ -221,7 +215,7 @@ FROM youtube WHERE (title ILIKE '%ClickHouse%') OR (description ILIKE '%ClickHouse%') ORDER BY like_count DESC, - view_count DESC + view_count DESC; ``` This query has to process every row, and also parse through two columns of strings. Even then, we get decent performance at 4.15M rows/second: @@ -268,7 +262,6 @@ ORDER BY ``` ```response - ┌─views─────────────┬─is_comments_enabled─┬────prob_like_dislike─┐ │ < 10.00 │ false │ 0.08224180712685371 │ │ < 100.00 │ false │ 0.06346337759167248 │ @@ -293,7 +286,6 @@ ORDER BY └───────────────────┴─────────────────────┴──────────────────────┘ 22 rows in set. Elapsed: 8.460 sec. Processed 4.56 billion rows, 77.48 GB (538.73 million rows/s., 9.16 GB/s.) - ``` Enabling comments seems to be correlated with a higher rate of engagement. From f96e7b59a282ec18961bfb4de25e181776d9ff9d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 Apr 2023 13:19:07 +0200 Subject: [PATCH 180/233] Better --- .../ClickHouseDictionarySource.cpp | 120 +++++++++--------- .../ExternalDataSourceConfiguration.cpp | 101 +++++---------- 2 files changed, 89 insertions(+), 132 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 6c1304ea027..f02e2995876 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -7,11 +7,11 @@ #include #include #include -#include #include #include #include #include +#include #include #include #include "DictionarySourceFactory.h" @@ -29,10 +29,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static const std::unordered_set dictionary_allowed_keys = { - "host", "port", "user", "password", "quota_key", "db", "database", "table", - "update_field", "update_lag", "invalidate_query", "query", "where", "name", "secure"}; - namespace { constexpr size_t MAX_CONNECTIONS = 16; @@ -214,73 +210,75 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, ContextPtr global_context, - const std::string & default_database [[maybe_unused]], + const std::string & default_database, bool created_from_ddl) -> DictionarySourcePtr { - bool secure = config.getBool(config_prefix + ".secure", false); - - UInt16 default_port = getPortFromContext(global_context, secure); + using Configuration = ClickHouseDictionarySource::Configuration; + std::optional configuration; std::string settings_config_prefix = config_prefix + ".clickhouse"; - - std::string host = config.getString(settings_config_prefix + ".host", "localhost"); - std::string user = config.getString(settings_config_prefix + ".user", "default"); - std::string password = config.getString(settings_config_prefix + ".password", ""); - std::string quota_key = config.getString(settings_config_prefix + ".quota_key", ""); - std::string db = config.getString(settings_config_prefix + ".db", default_database); - std::string table = config.getString(settings_config_prefix + ".table", ""); - UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); - auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); }; - bool secure_from_named = false; - - auto named_collection = created_from_ddl - ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context, has_config_key) - : std::nullopt; + auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix) : nullptr; if (named_collection) { - const auto & configuration = named_collection->configuration; - host = configuration.host; - user = configuration.username; - password = configuration.password; - quota_key = configuration.quota_key; - db = configuration.database; - table = configuration.table; - port = configuration.port; + validateNamedCollection( + *named_collection, {}, ValidateKeysMultiset{ + "secure", "host", "hostnmae", "port", "user", "username", "password", "quota_key", "name", + "db", "database", "table","query", "where", "invalidate_query", "update_field", "update_lag"}); - const auto & storage_specific_args = named_collection->specific_args; - for (const auto & [arg_name, arg_value] : storage_specific_args) - { - if (arg_name == "secure") - { - secure_from_named = checkAndGetLiteralArgument(arg_value, "secure"); - } - } + const auto secure = named_collection->getOrDefault("secure", false); + const auto default_port = getPortFromContext(global_context, secure); + const auto host = named_collection->getAnyOrDefault({"host", "hostname"}, "localhost"); + const auto port = static_cast(named_collection->getOrDefault("port", default_port)); + + configuration.emplace(Configuration{ + .host = host, + .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), + .password = named_collection->getOrDefault("password", ""), + .quota_key = named_collection->getOrDefault("quota_key", ""), + .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), + .table = named_collection->getOrDefault("table", ""), + .query = named_collection->getOrDefault("query", ""), + .where = named_collection->getOrDefault("where", ""), + .invalidate_query = named_collection->getOrDefault("invalidate_query", ""), + .update_field = named_collection->getOrDefault("update_field", ""), + .update_lag = named_collection->getOrDefault("update_lag", 1), + .port = port, + .is_local = isLocalAddress({host, port}, default_port), + .secure = secure, + }); + } + else + { + const auto secure = config.getBool(settings_config_prefix + ".secure", false); + const auto default_port = getPortFromContext(global_context, secure); + const auto host = config.getString(settings_config_prefix + ".host", "localhost"); + const auto port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); + + configuration.emplace(Configuration{ + .host = host, + .user = config.getString(settings_config_prefix + ".user", "default"), + .password = config.getString(settings_config_prefix + ".password", ""), + .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), + .db = config.getString(settings_config_prefix + ".db", default_database), + .table = config.getString(settings_config_prefix + ".table", ""), + .query = config.getString(settings_config_prefix + ".query", ""), + .where = config.getString(settings_config_prefix + ".where", ""), + .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), + .update_field = config.getString(settings_config_prefix + ".update_field", ""), + .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), + .port = port, + .is_local = isLocalAddress({host, port}, default_port), + .secure = secure, + }); } - ClickHouseDictionarySource::Configuration configuration{ - .host = host, - .user = user, - .password = password, - .quota_key = quota_key, - .db = db, - .table = table, - .query = config.getString(settings_config_prefix + ".query", ""), - .where = config.getString(settings_config_prefix + ".where", ""), - .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), - .update_field = config.getString(settings_config_prefix + ".update_field", ""), - .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), - .port = port, - .is_local = isLocalAddress({host, port}, default_port), - .secure = config.getBool(settings_config_prefix + ".secure", secure_from_named)}; - - ContextMutablePtr context; - if (configuration.is_local) + if (configuration->is_local) { /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). Session session(global_context, ClientInfo::Interface::LOCAL); - session.authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress{}); + session.authenticate(configuration->user, configuration->password, Poco::Net::SocketAddress{}); context = session.makeQueryContext(); } else @@ -288,7 +286,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) context = Context::createCopy(global_context); if (created_from_ddl) - context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + context->getRemoteHostFilter().checkHostAndPort(configuration->host, toString(configuration->port)); } context->applySettingsChanges(readSettingsFromDictionaryConfig(config, config_prefix)); @@ -296,10 +294,10 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) String dictionary_name = config.getString(".dictionary.name", ""); String dictionary_database = config.getString(".dictionary.database", ""); - if (dictionary_name == configuration.table && dictionary_database == configuration.db) + if (dictionary_name == configuration->table && dictionary_database == configuration->db) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table"); - return std::make_unique(dict_struct, configuration, sample_block, context); + return std::make_unique(dict_struct, *configuration, sample_block, context); }; factory.registerSource("clickhouse", create_table_source); diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 33e5772ed3e..e503c5edaab 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -21,6 +21,13 @@ namespace ErrorCodes IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) +static const std::unordered_set dictionary_allowed_keys = { + "host", "port", "user", "password", "quota_key", "db", + "database", "table", "schema", "replica", + "update_field", "update_lag", "invalidate_query", "query", + "where", "name", "secure", "uri", "collection"}; + + template SettingsChanges getSettingsChangesFromConfig( const BaseSettings & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) @@ -72,53 +79,6 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration addresses_expr = conf.addresses_expr; } -namespace -{ -void initExternalDataSourceConfiguration(ExternalDataSourceConfiguration & configuration) -{ - configuration = ExternalDataSourceConfiguration(); - configuration.username = ""; -} - -void readNamedCollection(const Poco::Util::AbstractConfiguration & config, - std::string_view collection_prefix, - ExternalDataSourceConfiguration & configuration) -{ - auto get_path = [collection_prefix](std::string_view fname) - { - return fmt::format("{}.{}", collection_prefix, fname); - }; - - configuration.host = config.getString(get_path("host"), configuration.host); - configuration.port = config.getInt(get_path("port"), configuration.port); - configuration.username = config.getString(get_path("user"), configuration.username); - configuration.password = config.getString(get_path("password"), configuration.password); - configuration.quota_key = config.getString(get_path("quota_key"), configuration.quota_key); - configuration.database = config.getString(get_path("db"), config.getString(get_path("database"), configuration.database)); - configuration.table = config.getString(get_path("table"), config.getString(get_path("collection"), configuration.table)); - configuration.schema = config.getString(get_path("schema"), configuration.schema); - configuration.addresses_expr = config.getString(get_path("addresses_expr"), configuration.addresses_expr); -} - -using ConfigWithPrefix = std::pair; - -/// Logical priority is from left to right. -/// If first element of config_with_prefix_vect does not have a particular field, -/// second element is used, etc. -/// Technically values are overwritten from right to left. -/// If no luck, default values come into play. -void readNamedCollection(const std::vector & config_with_prefix_vect, - ExternalDataSourceConfiguration & configuration) -{ - initExternalDataSourceConfiguration(configuration); - - for (auto it = std::crbegin(config_with_prefix_vect); it != std::crend(config_with_prefix_vect); ++it) - { - readNamedCollection((*it).first, (*it).second, configuration); - } -} -} - static void validateConfigKeys( const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, HasConfigKeyFunc has_config_key_func) @@ -139,7 +99,6 @@ std::optional getExternalDataSourceConfiguration( { validateConfigKeys(dict_config, dict_config_prefix, has_config_key); ExternalDataSourceConfiguration configuration; - StorageSpecificArgs non_common_args; auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); if (!collection_name.empty()) @@ -155,16 +114,15 @@ std::optional getExternalDataSourceConfiguration( if (!config.has(collection_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); - readNamedCollection({{dict_config, dict_config_prefix}, {config, collection_prefix}}, configuration); - - - if (dict_config.has(dict_config_prefix + ".secure") || config.has(collection_prefix + ".secure")) - { - uint64_t secure = dict_config.getBool(dict_config_prefix + ".secure", config.getBool(collection_prefix + ".secure", false)); - - non_common_args.emplace_back(std::make_pair("secure", std::make_shared(secure))); - } - + configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(collection_prefix + ".host", "")); + configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0)); + configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", "")); + configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", "")); + configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", config.getString(collection_prefix + ".quota_key", "")); + configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database", + config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", "")))); + configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", "")); + configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", "")); if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.table.empty()) { @@ -172,7 +130,7 @@ std::optional getExternalDataSourceConfiguration( "Named collection of connection parameters is missing some " "of the parameters and dictionary parameters are not added"); } - return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = non_common_args, .settings_changes = config_settings }; + return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = {}, .settings_changes = config_settings }; } return std::nullopt; } @@ -251,7 +209,14 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( } else { - readNamedCollection({{dict_config, dict_config_prefix}}, common_configuration); + common_configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); + common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); + common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); + common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); + common_configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", ""); + common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", "")); + common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); + common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); } ExternalDataSourcesByPriority configuration @@ -276,10 +241,11 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( validateConfigKeys(dict_config, replica_name, has_config_key); size_t priority = dict_config.getInt(replica_name + ".priority", 0); - - readNamedCollection({{dict_config, replica_name}, - {dict_config, dict_config_prefix}}, replica_configuration); - + replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); + replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port); + replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username); + replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password); + replica_configuration.quota_key = dict_config.getString(replica_name + ".quota_key", common_configuration.quota_key); if (replica_configuration.host.empty() || replica_configuration.port == 0 || replica_configuration.username.empty() || replica_configuration.password.empty()) @@ -288,13 +254,6 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( "Named collection of connection parameters is missing some " "of the parameters and no other dictionary parameters are added"); } - if (replica_configuration.database != common_configuration.database - || replica_configuration.table != common_configuration.table - || replica_configuration.schema != common_configuration.schema) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Named collection of connection parameters is not consistent"); - } configuration.replicas_configurations[priority].emplace_back(replica_configuration); } From 48b23dd012d95d2b2d477d4fa448f4e88f628c8e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 1 Apr 2023 15:26:00 +0300 Subject: [PATCH 181/233] Fix race between DROP MatView and RESTART REPLICAS (#47863) * fix race between drop mv and restart replicas * unrelated: fix bad exception messages * fix test * fix * fix * fix * fix * fix test * fix * fix test * Update 02437_drop_mv_restart_replicas.sh * fix tests --- src/DataTypes/DataTypeDecimalBase.h | 7 ++- src/DataTypes/DataTypesDecimal.h | 8 ++- src/Functions/array/arrayAggregation.cpp | 6 +- src/Functions/pointInEllipses.cpp | 5 +- src/Functions/pointInPolygon.cpp | 4 +- src/Functions/svg.cpp | 11 +--- src/Interpreters/InterpreterDropQuery.cpp | 22 ++++--- src/Interpreters/InterpreterDropQuery.h | 3 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- .../StorageMaterializedPostgreSQL.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 17 ++++- src/Storages/WindowView/StorageWindowView.cpp | 4 +- tests/ci/stress_tests.lib | 7 ++- tests/clickhouse-test | 13 ++-- .../test.py | 4 +- .../02437_drop_mv_restart_replicas.reference | 0 .../02437_drop_mv_restart_replicas.sh | 63 +++++++++++++++++++ 17 files changed, 139 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/02437_drop_mv_restart_replicas.reference create mode 100755 tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index 0be345ba879..adbe9c95b14 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -71,9 +71,12 @@ public: scale(scale_) { if (unlikely(precision < 1 || precision > maxPrecision())) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Precision {} is out of bounds", std::to_string(precision)); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Precision {} is out of bounds (precision range: [1, {}])", + std::to_string(precision), maxPrecision()); if (unlikely(scale > maxPrecision())) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", std::to_string(scale)); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds (max scale: {})", + std::to_string(scale), maxPrecision()); } TypeIndex getTypeId() const override { return TypeToTypeIndex; } diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 7a49238b5be..583f7ea804a 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -116,7 +116,8 @@ inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & v if (common::mulOverflow(static_cast(value.value), converted_value, converted_value)) { if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow", std::string(ToDataType::family_name)); + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow while multiplying {} by scale {}", + std::string(ToDataType::family_name), toString(value.value), toString(converted_value)); else return ReturnType(false); } @@ -136,7 +137,10 @@ inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & v converted_value > std::numeric_limits::max()) { if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow", std::string(ToDataType::family_name)); + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow: {} is not in range ({}, {})", + std::string(ToDataType::family_name), toString(converted_value), + toString(std::numeric_limits::min()), + toString(std::numeric_limits::max())); else return ReturnType(false); } diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 59991b7b313..8818ebde9f1 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -223,7 +223,8 @@ struct ArrayAggregateImpl auto result_scale = column_typed->getScale() * array_size; if (unlikely(result_scale > DecimalUtils::max_precision)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds (max scale: {})", + result_scale, DecimalUtils::max_precision); res[i] = DecimalUtils::convertTo(product, static_cast(result_scale)); } @@ -332,7 +333,8 @@ struct ArrayAggregateImpl auto result_scale = column->getScale() * count; if (unlikely(result_scale > DecimalUtils::max_precision)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds (max scale: {})", + result_scale, DecimalUtils::max_precision); res[i] = DecimalUtils::convertTo(aggregate_value, static_cast(result_scale)); } diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index 208f2ad2f82..2147428cee3 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -71,8 +71,9 @@ private: /// For array on stack, see below. if (arguments.size() > 10000) { - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments of function {} is too large.", - getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + "Number of arguments of function {} is too large (maximum: 10000).", + getName()); } for (const auto arg_idx : collections::range(0, arguments.size())) diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index 24ad1d20611..0e4467a8210 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -37,7 +37,7 @@ namespace DB { namespace ErrorCodes { - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; @@ -87,7 +87,7 @@ public: { if (arguments.size() < 2) { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Too few arguments"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments", getName()); } /** We allow function invocation in one of the following forms: diff --git a/src/Functions/svg.cpp b/src/Functions/svg.cpp index 69e619df901..f8f85216b3f 100644 --- a/src/Functions/svg.cpp +++ b/src/Functions/svg.cpp @@ -13,8 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionSvg : public IFunction @@ -48,13 +47,9 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 2) + if (arguments.empty() || arguments.size() > 2) { - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Too many arguments"); - } - else if (arguments.empty()) - { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Too few arguments"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Incorrect number of arguments: expected 1 or 2 arguments"); } else if (arguments.size() == 2 && checkAndGetDataType(arguments[1].get()) == nullptr) { diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index e16403bed67..0beb4492aef 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -355,19 +355,22 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, /// Flush should not be done if shouldBeEmptyOnDetach() == false, /// since in this case getTablesIterator() may do some additional work, /// see DatabaseMaterializedMySQL::getTablesIterator() - for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) - { - iterator->table()->flush(); - } - auto table_context = Context::createCopy(getContext()); table_context->setInternalQuery(true); + /// Do not hold extra shared pointers to tables + std::vector> tables_to_drop; for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { + iterator->table()->flush(); + tables_to_drop.push_back({iterator->name(), iterator->table()->isDictionary()}); + } + + for (const auto & table : tables_to_drop) + { + query_for_table.setTable(table.first); + query_for_table.is_dictionary = table.second; DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; - query_for_table.setTable(iterator->name()); - query_for_table.is_dictionary = iterator->table()->isDictionary(); executeToTableImpl(table_context, query_for_table, db, table_to_wait); uuids_to_wait.push_back(table_to_wait); } @@ -428,7 +431,8 @@ AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() co return required_access; } -void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync) +void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, + const StorageID & target_table_id, bool sync, bool ignore_sync_setting) { if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) { @@ -445,6 +449,8 @@ void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr /// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant /// looks like expected behaviour and we have tests for it. auto drop_context = Context::createCopy(global_context); + if (ignore_sync_setting) + drop_context->setSetting("database_atomic_wait_for_drop_and_detach_synchronously", false); drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; if (auto txn = current_context->getZooKeeperMetadataTransaction()) { diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index afec26424ba..af7a4ddef25 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -24,7 +24,8 @@ public: /// Drop table or database. BlockIO execute() override; - static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync); + static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, + const StorageID & target_table_id, bool sync, bool ignore_sync_setting = false); bool supportsTransactions() const override; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b45618be1f8..e9905821fd1 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -604,6 +604,7 @@ void InterpreterSystemQuery::restoreReplica() StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard) { + LOG_TRACE(log, "Restarting replica {}", replica); auto table_ddl_guard = need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) : nullptr; @@ -647,6 +648,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, database->attachTable(system_context, replica.table_name, table, data_path); table->startup(); + LOG_TRACE(log, "Restarted replica {}", replica); return table; } @@ -693,11 +695,11 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context) guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name); size_t threads = std::min(static_cast(getNumberOfPhysicalCPUCores()), replica_names.size()); + LOG_DEBUG(log, "Will restart {} replicas using {} threads", replica_names.size(), threads); ThreadPool pool(CurrentMetrics::RestartReplicaThreads, CurrentMetrics::RestartReplicaThreadsActive, threads); for (auto & replica : replica_names) { - LOG_TRACE(log, "Restarting replica on {}", replica.getNameForLogs()); pool.scheduleOrThrowOnError([&]() { tryRestartReplica(replica, system_context, false); }); } pool.wait(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index d194c8b8201..78e72564ab7 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -250,7 +250,7 @@ void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool sync, ContextPtr lo auto nested_table = tryGetNested() != nullptr; if (nested_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), sync); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), sync, /* ignore_sync_setting */ true); } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ae3fa62b38c..b96c132d601 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -212,13 +212,26 @@ void StorageMaterializedView::drop() if (!select_query.select_table_id.empty()) DatabaseCatalog::instance().removeViewDependency(select_query.select_table_id, table_id); - dropInnerTableIfAny(true, getContext()); + /// Sync flag and the setting make sense for Atomic databases only. + /// However, with Atomic databases, IStorage::drop() can be called only from a background task in DatabaseCatalog. + /// Running synchronous DROP from that task leads to deadlock. + /// Usually dropInnerTableIfAny is no-op, because the inner table is dropped before enqueueing a drop task for the MV itself. + /// But there's a race condition with SYSTEM RESTART REPLICA: the inner table might be detached due to RESTART. + /// In this case, dropInnerTableIfAny will not find the inner table and will not drop it during executions of DROP query for the MV itself. + /// DDLGuard does not protect from that, because RESTART REPLICA acquires DDLGuard for the inner table name, + /// but DROP acquires DDLGuard for the name of MV. And we cannot acquire second DDLGuard for the inner name in DROP, + /// because it may lead to lock-order-inversion (DDLGuards must be acquired in lexicographical order). + dropInnerTableIfAny(/* sync */ false, getContext()); } void StorageMaterializedView::dropInnerTableIfAny(bool sync, ContextPtr local_context) { + /// We will use `sync` argument wneh this function is called from a DROP query + /// and will ignore database_atomic_wait_for_drop_and_detach_synchronously when it's called from drop task. + /// See the comment in StorageMaterializedView::drop if (has_inner_table && tryGetTargetTable()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, + sync, /* ignore_sync_setting */ true); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3471e4ea6bf..bfa126c3525 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1609,7 +1609,7 @@ void StorageWindowView::drop() { /// Must be guaranteed at this point for database engine Atomic that has_inner_table == false, /// because otherwise will be a deadlock. - dropInnerTableIfAny(true, getContext()); + dropInnerTableIfAny(false, getContext()); } void StorageWindowView::dropInnerTableIfAny(bool sync, ContextPtr local_context) @@ -1623,7 +1623,7 @@ void StorageWindowView::dropInnerTableIfAny(bool sync, ContextPtr local_context) ASTDropQuery::Kind::Drop, getContext(), local_context, inner_table_id, sync); if (has_inner_target_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync, /* ignore_sync_setting */ true); } catch (...) { diff --git a/tests/ci/stress_tests.lib b/tests/ci/stress_tests.lib index 75195baaeeb..04df50b3248 100644 --- a/tests/ci/stress_tests.lib +++ b/tests/ci/stress_tests.lib @@ -149,9 +149,14 @@ function stop() if [ $check_hang == true ] then # We failed to stop the server with SIGTERM. Maybe it hang, let's collect stacktraces. - echo -e "Possible deadlock on shutdown (see gdb.log)$FAIL" >> /test_output/test_results.tsv + # Add a special status just in case, so it will be possible to find in the CI DB + echo -e "Warning: server did not stop yet$OK" >> /test_output/test_results.tsv kill -TERM "$(pidof gdb)" ||: sleep 5 + + # The server could finally stop while we were terminating gdb, let's recheck if it's still running + kill -s 0 $pid || return + echo -e "Possible deadlock on shutdown (see gdb.log)$FAIL" >> /test_output/test_results.tsv echo "thread apply all backtrace (on stop)" >> /test_output/gdb.log timeout 30m gdb -batch -ex 'thread apply all backtrace' -p "$pid" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log clickhouse stop --force diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a355c2f8e73..d407f73d033 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -278,7 +278,7 @@ def need_retry(args, stdout, stderr, total_time): def get_processlist_with_stacktraces(args): try: if args.replicated_database: - return clickhouse_execute_json( + return clickhouse_execute( args, """ SELECT materialize(hostName() || '::' || tcpPort()::String) as host_port, * @@ -295,14 +295,14 @@ def get_processlist_with_stacktraces(args): WHERE query NOT LIKE '%system.processes%' GROUP BY p.* )) - ORDER BY elapsed DESC + ORDER BY elapsed DESC FORMAT Vertical """, settings={ "allow_introspection_functions": 1, }, ) else: - return clickhouse_execute_json( + return clickhouse_execute( args, """ SELECT @@ -315,7 +315,7 @@ def get_processlist_with_stacktraces(args): JOIN system.stack_trace s USING (query_id) WHERE query NOT LIKE '%system.processes%' GROUP BY p.* - ORDER BY elapsed DESC + ORDER BY elapsed DESC FORMAT Vertical """, settings={ "allow_introspection_functions": 1, @@ -2058,7 +2058,10 @@ def reportLogStats(args): 'Table {} is not replicated', '{} {}.{} already exists', 'Attempt to read after eof', 'Replica {} already exists', 'Convert overflow', 'key must be a tuple', 'Division by zero', 'No part {} in committed state', 'Files set to {}', 'Bytes set to {}', 'Sharding key {} is not used', - 'Cannot parse datetime', 'Bad get: has {}, requested {}', 'There is no {} in {}', 'Numeric overflow' + 'Cannot parse datetime', 'Bad get: has {}, requested {}', 'There is no {} in {}', 'Numeric overflow', + 'Polygon is not valid: {}', 'Decimal math overflow', '{} only accepts maps', 'Dictionary ({}) not found', + 'Unknown format {}', 'Invalid IPv4 value', 'Invalid IPv6 value', 'Unknown setting {}', + 'Unknown table function {}' ) AS known_short_messages SELECT count() AS c, message_format_string, substr(any(message), 1, 120) FROM system.text_log diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 6b27c69462a..67f7ce47451 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -91,8 +91,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("OPTIMIZE TABLE mt FINAL") - assert node1.query("SELECT id FROM mt") == "1\n4\n" - assert node2.query("SELECT id FROM mt") == "1\n4\n" + assert node1.query("SELECT id FROM mt ORDER BY id") == "1\n4\n" + assert node2.query("SELECT id FROM mt ORDER BY id") == "1\n4\n" for node in [node1, node2]: node.query("DROP TABLE mt") diff --git a/tests/queries/0_stateless/02437_drop_mv_restart_replicas.reference b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh new file mode 100755 index 00000000000..e4f52b6e4ad --- /dev/null +++ b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash +# Tags: long, zookeeper, race, no-ordinary-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create user u_$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT -q "grant all on db_$CLICKHOUSE_DATABASE.* to u_$CLICKHOUSE_DATABASE" + +# For tests with Replicated +ENGINE=$($CLICKHOUSE_CLIENT -q "select replace(engine_full, '$CLICKHOUSE_DATABASE', 'db_$CLICKHOUSE_DATABASE') from system.databases where name='$CLICKHOUSE_DATABASE' format TSVRaw") +export ENGINE + +function thread_ddl() +{ + while true; do + $CLICKHOUSE_CLIENT -q "create database if not exists db_$CLICKHOUSE_DATABASE engine=$ENGINE" + $CLICKHOUSE_CLIENT -q "CREATE TABLE if not exists db_$CLICKHOUSE_DATABASE.test (test String, A Int64, B Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') ORDER BY tuple();" + $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW if not exists db_$CLICKHOUSE_DATABASE.test_mv_a Engine=ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') order by tuple() AS SELECT test, A, count() c FROM db_$CLICKHOUSE_DATABASE.test group by test, A;" + $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW if not exists db_$CLICKHOUSE_DATABASE.test_mv_b Engine=ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') partition by A order by tuple() AS SELECT test, A, count() c FROM db_$CLICKHOUSE_DATABASE.test group by test, A;" + $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW if not exists db_$CLICKHOUSE_DATABASE.test_mv_c Engine=ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') order by tuple() AS SELECT test, A, count() c FROM db_$CLICKHOUSE_DATABASE.test group by test, A;" + sleep 0.$RANDOM; + + # A kind of backoff + timeout 5s $CLICKHOUSE_CLIENT -q "select sleepEachRow(0.1) from system.dropped_tables format Null" 2>/dev/null ||: + + $CLICKHOUSE_CLIENT -q "drop database if exists db_$CLICKHOUSE_DATABASE" + done +} + +function thread_insert() +{ + while true; do + $CLICKHOUSE_CLIENT -q "INSERT INTO db_$CLICKHOUSE_DATABASE.test SELECT 'case1', number%3, rand() FROM numbers(5)" + sleep 0.$RANDOM; + done +} + +function thread_restart() +{ + while true; do + # The simplest way to restart only replicas from a specific database is to use a special user + $CLICKHOUSE_CLIENT --user "u_$CLICKHOUSE_DATABASE" -q "system restart replicas" + sleep 0.$RANDOM; + done +} + +export -f thread_ddl; +export -f thread_insert; +export -f thread_restart; + +TIMEOUT=15 + +timeout $TIMEOUT bash -c thread_ddl 2>&1| grep -Fa "Exception: " | grep -Fv -e "TABLE_IS_DROPPED" -e "UNKNOWN_TABLE" -e "DATABASE_NOT_EMPTY" & +timeout $TIMEOUT bash -c thread_insert 2> /dev/null & +timeout $TIMEOUT bash -c thread_restart 2>&1| grep -Fa "Exception: " | grep -Fv -e "is currently dropped or renamed" & + +wait + +timeout 45s $CLICKHOUSE_CLIENT -q "select sleepEachRow(0.3) from system.dropped_tables format Null" 2>/dev/null ||: + +$CLICKHOUSE_CLIENT -q "drop database if exists db_$CLICKHOUSE_DATABASE" 2>&1| grep -Fa "Exception: " | grep -Fv -e "TABLE_IS_DROPPED" -e "UNKNOWN_TABLE" -e "DATABASE_NOT_EMPTY" ||: From ba6ecd2d4ea063366097ed5bad05a358c4283b74 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 15:02:58 +0200 Subject: [PATCH 182/233] Fix ThreadPool for DistributedSink Signed-off-by: Azat Khuzhin --- src/Common/CurrentMetrics.cpp | 2 ++ src/Storages/Distributed/DistributedSink.cpp | 9 ++++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 542c48148c8..cde020509ee 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -126,6 +126,8 @@ M(DDLWorkerThreadsActive, "Number of threads in the DDLWORKER thread pool for ON CLUSTER queries running a task.") \ M(StorageDistributedThreads, "Number of threads in the StorageDistributed thread pool.") \ M(StorageDistributedThreadsActive, "Number of threads in the StorageDistributed thread pool running a task.") \ + M(DistributedInsertThreads, "Number of threads used for INSERT into Distributed.") \ + M(DistributedInsertThreadsActive, "Number of threads used for INSERT into Distributed running a task.") \ M(StorageS3Threads, "Number of threads in the StorageS3 thread pool.") \ M(StorageS3ThreadsActive, "Number of threads in the StorageS3 thread pool running a task.") \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 11b938cd722..720a951299a 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -41,6 +41,8 @@ namespace CurrentMetrics { extern const Metric DistributedSend; + extern const Metric DistributedInsertThreads; + extern const Metric DistributedInsertThreadsActive; } namespace ProfileEvents @@ -460,9 +462,10 @@ void DistributedSink::writeSync(const Block & block) size_t jobs_count = random_shard_insert ? 1 : (remote_jobs_count + local_jobs_count); size_t max_threads = std::min(settings.max_distributed_connections, jobs_count); - pool.emplace(/* max_threads_= */ max_threads, - /* max_free_threads_= */ max_threads, - /* queue_size_= */ jobs_count); + pool.emplace( + CurrentMetrics::DistributedInsertThreads, + CurrentMetrics::DistributedInsertThreadsActive, + max_threads, max_threads, jobs_count); if (!throttler && (settings.max_network_bandwidth || settings.max_network_bytes)) { From f69441b6339f79fae29c969f54f6cd687077a2eb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 15:22:13 +0200 Subject: [PATCH 183/233] Fix ThreadPool usage in gtest_thread_pool_limit Signed-off-by: Azat Khuzhin --- src/Common/tests/gtest_thread_pool_limit.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_thread_pool_limit.cpp b/src/Common/tests/gtest_thread_pool_limit.cpp index bc67ffd0bc1..17f79d17894 100644 --- a/src/Common/tests/gtest_thread_pool_limit.cpp +++ b/src/Common/tests/gtest_thread_pool_limit.cpp @@ -1,16 +1,23 @@ #include #include #include +#include #include +namespace CurrentMetrics +{ + extern const Metric LocalThread; + extern const Metric LocalThreadActive; +} + /// Test for thread self-removal when number of free threads in pool is too large. /// Just checks that nothing weird happens. template int test() { - Pool pool(10, 2, 10); + Pool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 10, 2, 10); std::atomic counter{0}; for (size_t i = 0; i < 10; ++i) From 3ede50ccfc54e32228e19bca415491940d00f051 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 15:05:35 +0200 Subject: [PATCH 184/233] Fix MaxPushedDDLEntryID --- src/Interpreters/DDLWorker.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 22bece0ef04..c4529af2c51 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -1022,10 +1022,10 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry) { String str_buf = node_path.substr(query_path_prefix.length()); DB::ReadBufferFromString in(str_buf); - CurrentMetrics::Metric id; - readText(id, in); - id = std::max(*max_pushed_entry_metric, id); - CurrentMetrics::set(*max_pushed_entry_metric, id); + CurrentMetrics::Value pushed_entry; + readText(pushed_entry, in); + pushed_entry = std::max(CurrentMetrics::get(*max_pushed_entry_metric), pushed_entry); + CurrentMetrics::set(*max_pushed_entry_metric, pushed_entry); } /// We cannot create status dirs in a single transaction with previous request, From 81e635a293805a0d59ace0f47a596f686b0ef347 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 11:48:47 +0200 Subject: [PATCH 185/233] Mark operator constexpr in Strongtypedef --- base/base/strong_typedef.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/strong_typedef.h b/base/base/strong_typedef.h index 2ddea6412f5..b3b8bced688 100644 --- a/base/base/strong_typedef.h +++ b/base/base/strong_typedef.h @@ -35,7 +35,7 @@ public: Self & operator=(T && rhs) { t = std::move(rhs); return *this;} // NOLINTBEGIN(google-explicit-constructor) - operator const T & () const { return t; } + constexpr operator const T & () const { return t; } operator T & () { return t; } // NOLINTEND(google-explicit-constructor) From 8d0e516310edcf9bb4202113d15c4ab42f75c31b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 11:17:23 +0200 Subject: [PATCH 186/233] Use StrongTypedef for CurrentMetrics to avoid possible incorrect usage Signed-off-by: Azat Khuzhin --- programs/server/MetricsTransmitter.cpp | 2 +- src/Common/CurrentMetrics.cpp | 4 ++-- src/Common/CurrentMetrics.h | 3 ++- src/Disks/TemporaryFileOnDisk.cpp | 2 +- src/Disks/TemporaryFileOnDisk.h | 2 +- src/Interpreters/MetricLog.cpp | 2 +- src/Interpreters/TemporaryDataOnDisk.cpp | 2 +- src/Interpreters/TemporaryDataOnDisk.h | 4 ++-- 8 files changed, 11 insertions(+), 10 deletions(-) diff --git a/programs/server/MetricsTransmitter.cpp b/programs/server/MetricsTransmitter.cpp index 2f28f0a1d16..1672b0cd4dc 100644 --- a/programs/server/MetricsTransmitter.cpp +++ b/programs/server/MetricsTransmitter.cpp @@ -110,7 +110,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count if (send_metrics) { - for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + for (CurrentMetrics::Metric i = CurrentMetrics::Metric(0), end = CurrentMetrics::end(); i < end; ++i) { const auto value = CurrentMetrics::values[i].load(std::memory_order_relaxed); diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index cde020509ee..cfe9f41befe 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -186,10 +186,10 @@ namespace CurrentMetrics { - #define M(NAME, DOCUMENTATION) extern const Metric NAME = __COUNTER__; + #define M(NAME, DOCUMENTATION) extern const Metric NAME = Metric(__COUNTER__); APPLY_FOR_METRICS(M) #undef M - constexpr Metric END = __COUNTER__; + constexpr Metric END = Metric(__COUNTER__); std::atomic values[END] {}; /// Global variable, initialized by zeros. diff --git a/src/Common/CurrentMetrics.h b/src/Common/CurrentMetrics.h index 0ae16e2d08d..a1ef254485d 100644 --- a/src/Common/CurrentMetrics.h +++ b/src/Common/CurrentMetrics.h @@ -6,6 +6,7 @@ #include #include #include +#include /** Allows to count number of simultaneously happening processes or current value of some metric. * - for high-level profiling. @@ -22,7 +23,7 @@ namespace CurrentMetrics { /// Metric identifier (index in array). - using Metric = size_t; + using Metric = StrongTypedef; using Value = DB::Int64; /// Get name of metric by identifier. Returns statically allocated string. diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index d31b09b2185..6fe6fd5a1c9 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -27,7 +27,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_) : TemporaryFileOnDisk(disk_, "") {} -TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope) +TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Metric metric_scope) : TemporaryFileOnDisk(disk_) { sub_metric_increment.emplace(metric_scope); diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index 9ba59c3eaf0..4c376383087 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -17,7 +17,7 @@ class TemporaryFileOnDisk { public: explicit TemporaryFileOnDisk(const DiskPtr & disk_); - explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope); + explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Metric metric_scope); explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix); ~TemporaryFileOnDisk(); diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 6e98f84bc82..14a190b41ef 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -50,7 +50,7 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(profile_events[i]); for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) - columns[column_idx++]->insert(current_metrics[i]); + columns[column_idx++]->insert(current_metrics[i].toUnderType()); } diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 25252f8226b..c57de88d964 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -49,7 +49,7 @@ TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_) : TemporaryDataOnDiskScope(std::move(parent_), /* limit_ = */ 0) {} -TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Value metric_scope) +TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Metric metric_scope) : TemporaryDataOnDiskScope(std::move(parent_), /* limit_ = */ 0) , current_metric_scope(metric_scope) {} diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index f0e02f16fb6..f7a6249a1ee 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -85,7 +85,7 @@ public: explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_); - explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Value metric_scope); + explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Metric metric_scope); /// If max_file_size > 0, then check that there's enough space on the disk and throw an exception in case of lack of free space TemporaryFileStream & createStream(const Block & header, size_t max_file_size = 0); @@ -102,7 +102,7 @@ private: mutable std::mutex mutex; std::vector streams TSA_GUARDED_BY(mutex); - typename CurrentMetrics::Value current_metric_scope = CurrentMetrics::TemporaryFilesUnknown; + typename CurrentMetrics::Metric current_metric_scope = CurrentMetrics::TemporaryFilesUnknown; }; /* From c64f9e6f07f57491ef3950d779b403e9cc2982b5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 11:52:23 +0200 Subject: [PATCH 187/233] Use StrongTypedef for ProfileEvents Signed-off-by: Azat Khuzhin --- programs/server/MetricsTransmitter.cpp | 4 ++-- src/Common/ProfileEvents.cpp | 10 +++++----- src/Common/ProfileEvents.h | 5 +++-- src/Interpreters/MetricLog.cpp | 2 +- src/Interpreters/ProfileEventsExt.cpp | 4 ++-- src/Server/PrometheusMetricsWriter.cpp | 2 +- src/Storages/System/StorageSystemEvents.cpp | 2 +- 7 files changed, 15 insertions(+), 14 deletions(-) diff --git a/programs/server/MetricsTransmitter.cpp b/programs/server/MetricsTransmitter.cpp index 1672b0cd4dc..ae9fa5ecc2c 100644 --- a/programs/server/MetricsTransmitter.cpp +++ b/programs/server/MetricsTransmitter.cpp @@ -87,7 +87,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count if (send_events) { - for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); const auto counter_increment = counter - prev_counters[i]; @@ -100,7 +100,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count if (send_events_cumulative) { - for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); std::string key{ProfileEvents::getName(static_cast(i))}; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 3cee4a8e718..1d035952f13 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -497,10 +497,10 @@ The server successfully detected this situation and will download merged part fr namespace ProfileEvents { -#define M(NAME, DOCUMENTATION) extern const Event NAME = __COUNTER__; +#define M(NAME, DOCUMENTATION) extern const Event NAME = Event(__COUNTER__); APPLY_FOR_EVENTS(M) #undef M -constexpr Event END = __COUNTER__; +constexpr Event END = Event(__COUNTER__); /// Global variable, initialized by zeros. Counter global_counters_array[END] {}; @@ -522,7 +522,7 @@ void Counters::resetCounters() { if (counters) { - for (Event i = 0; i < num_counters; ++i) + for (Event i = Event(0); i < num_counters; ++i) counters[i].store(0, std::memory_order_relaxed); } } @@ -540,7 +540,7 @@ Counters::Snapshot::Snapshot() Counters::Snapshot Counters::getPartiallyAtomicSnapshot() const { Snapshot res; - for (Event i = 0; i < num_counters; ++i) + for (Event i = Event(0); i < num_counters; ++i) res.counters_holder[i] = counters[i].load(std::memory_order_relaxed); return res; } @@ -616,7 +616,7 @@ CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot) CountersIncrement::CountersIncrement(Counters::Snapshot const & after, Counters::Snapshot const & before) { init(); - for (Event i = 0; i < Counters::num_counters; ++i) + for (Event i = Event(0); i < Counters::num_counters; ++i) increment_holder[i] = static_cast(after[i]) - static_cast(before[i]); } diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 867b5b551c6..a36e68742cf 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -1,7 +1,8 @@ #pragma once #include -#include "base/types.h" +#include +#include #include #include #include @@ -14,7 +15,7 @@ namespace ProfileEvents { /// Event identifier (index in array). - using Event = size_t; + using Event = StrongTypedef; using Count = size_t; using Increment = Int64; using Counter = std::atomic; diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 14a190b41ef..578cc118a6b 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -97,7 +97,7 @@ void MetricLog::metricThreadFunction() elem.milliseconds = timeInMilliseconds(current_time) - timeInSeconds(current_time) * 1000; elem.profile_events.resize(ProfileEvents::end()); - for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); auto & old_value = prev_profile_events[i]; diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 7fbbe3c662b..44977524c56 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -32,7 +32,7 @@ void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, auto & value_column = tuple_column.getColumn(1); size_t size = 0; - for (Event event = 0; event < Counters::num_counters; ++event) + for (Event event = Event(0); event < Counters::num_counters; ++event) { UInt64 value = counters[event]; @@ -54,7 +54,7 @@ static void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::Mutabl size_t rows = 0; auto & name_column = columns[NAME_COLUMN_INDEX]; auto & value_column = columns[VALUE_COLUMN_INDEX]; - for (Event event = 0; event < Counters::num_counters; ++event) + for (Event event = Event(0); event < Counters::num_counters; ++event) { Int64 value = snapshot.counters[event]; diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index abf2a2c0b6b..2331e455225 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -59,7 +59,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const { if (send_events) { - for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index be2d3f8d49e..b9b07cfe0ac 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -18,7 +18,7 @@ NamesAndTypesList StorageSystemEvents::getNamesAndTypes() void StorageSystemEvents::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { UInt64 value = ProfileEvents::global_counters[i]; From fbc99d778860355ee7e246a3503d0a02c43ddff5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 11:55:31 +0200 Subject: [PATCH 188/233] Use StrongTypedef for StatusInfo Signed-off-by: Azat Khuzhin --- src/Common/StatusInfo.cpp | 4 ++-- src/Common/StatusInfo.h | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Common/StatusInfo.cpp b/src/Common/StatusInfo.cpp index 32afc833001..1f9ddfaf4b9 100644 --- a/src/Common/StatusInfo.cpp +++ b/src/Common/StatusInfo.cpp @@ -8,10 +8,10 @@ namespace CurrentStatusInfo { - #define M(NAME, DOCUMENTATION, ENUM) extern const Status NAME = __COUNTER__; + #define M(NAME, DOCUMENTATION, ENUM) extern const Status NAME = Status(__COUNTER__); APPLY_FOR_STATUS(M) #undef M - constexpr Status END = __COUNTER__; + constexpr Status END = Status(__COUNTER__); std::mutex locks[END] {}; std::unordered_map values[END] {}; diff --git a/src/Common/StatusInfo.h b/src/Common/StatusInfo.h index 9aa185cd0c3..91e6d4d3b85 100644 --- a/src/Common/StatusInfo.h +++ b/src/Common/StatusInfo.h @@ -6,13 +6,14 @@ #include #include #include +#include #include #include namespace CurrentStatusInfo { - using Status = size_t; + using Status = StrongTypedef; using Key = std::string; const char * getName(Status event); From 179450542879d11711cd2415c3fa7eeab18188be Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 1 Apr 2023 16:02:24 +0200 Subject: [PATCH 189/233] check-style: do not count CurrentMetrics::get as metric --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 7a1fa6ce123..a6cc20bb7c8 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -78,6 +78,7 @@ EXTERN_TYPES_EXCLUDES=( CurrentMetrics::add CurrentMetrics::sub + CurrentMetrics::get CurrentMetrics::set CurrentMetrics::end CurrentMetrics::Increment From ccb5b257248eae05ffc404ec4acbf1fd4cd9b075 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 Apr 2023 18:21:49 +0200 Subject: [PATCH 190/233] One more test --- .../configs/named_coll.xml | 8 ++++++++ .../integration/test_dictionaries_ddl/test.py | 19 +++++++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/tests/integration/test_dictionaries_ddl/configs/named_coll.xml b/tests/integration/test_dictionaries_ddl/configs/named_coll.xml index b14113a301d..ba450ae014f 100644 --- a/tests/integration/test_dictionaries_ddl/configs/named_coll.xml +++ b/tests/integration/test_dictionaries_ddl/configs/named_coll.xml @@ -17,5 +17,13 @@ default 9440 + + localhost + 9000 + test + default + + select id, SomeValue1 from test.xml_dictionary_table + diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 29da9d7af7b..7dda6fc245a 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -592,3 +592,22 @@ def test_secure(started_cluster): node1.query("DROP DICTIONARY test.clickhouse_secure") node1.query("DROP TABLE test.foo_dict") + + +def test_named_collection(started_cluster): + node1.query( + """ + CREATE DICTIONARY test.clickhouse_named_collection( + id UInt64, + SomeValue1 UInt8 + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(NAME click1)) + LIFETIME(MIN 1 MAX 10) + """ + ) + + node1.query( + "select dictGetUInt8('test.clickhouse_named_collection', 'SomeValue1', toUInt64(23))" + ) == "0\n" From 230adac9f6bfeaf8d86f33396e147910eab930fe Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 1 Apr 2023 18:23:59 +0200 Subject: [PATCH 191/233] fixes --- .../AggregateFunctionGroupArray.h | 9 ++-- .../AggregateFunctionGroupArrayInsertAt.h | 6 ++- .../AggregateFunctionGroupArrayMoving.h | 3 +- .../AggregateFunctionGroupBitmapData.h | 3 +- .../AggregateFunctionHistogram.h | 3 +- .../AggregateFunctionIntervalLengthSum.h | 2 +- .../AggregateFunctionMaxIntersections.h | 3 +- .../AggregateFunctionSequenceNextNode.h | 3 +- src/AggregateFunctions/QuantileExact.h | 3 +- src/AggregateFunctions/ReservoirSampler.h | 3 +- .../ReservoirSamplerDeterministic.h | 3 +- src/IO/ReadHelpers.h | 3 +- src/Server/TCPHandler.cpp | 3 +- tests/clickhouse-test | 12 +++++- ...nd_exception_messages_formatting.reference | 2 +- ..._log_and_exception_messages_formatting.sql | 2 +- .../02437_drop_mv_restart_replicas.sh | 3 +- ...replacing_merge_tree_is_deleted_column.sql | 42 +++++++++---------- 18 files changed, 66 insertions(+), 42 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 5a799dc3641..7a5e6a8cb2d 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -288,7 +288,8 @@ public: readVarUInt(size, buf); if (unlikely(size > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE); if (limit_num_elems && unlikely(size > max_elems)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size, it should not exceed {}", max_elems); @@ -367,7 +368,8 @@ struct GroupArrayNodeBase UInt64 size; readVarUInt(size, buf); if (unlikely(size > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE); Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; @@ -621,7 +623,8 @@ public: return; if (unlikely(elems > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE); if (limit_num_elems && unlikely(elems > max_elems)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size, it should not exceed {}", max_elems); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index dbd7b32b9ce..439bb613337 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -79,7 +79,8 @@ public: { length_to_resize = applyVisitor(FieldVisitorConvertToNumber(), params[1]); if (length_to_resize > AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE); } } @@ -167,7 +168,8 @@ public: readVarUInt(size, buf); if (size > AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE); Array & arr = data(place).value; diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index 8d7010c10db..e6f79d7bca1 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -144,7 +144,8 @@ public: readVarUInt(size, buf); if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE); if (size > 0) { diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 62017251108..d99f0bf16ee 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -127,7 +127,8 @@ public: if (size == 0) throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect size (0) in groupBitmap."); if (size > max_size) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in groupBitmap."); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size in groupBitmap (maximum: {})", max_size); /// TODO: this is unnecessary copying - it will be better to read and deserialize in one pass. std::unique_ptr buf(new char[size]); diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 62ed071856a..3a98737f199 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -294,7 +294,8 @@ public: throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too many bins"); static constexpr size_t max_size = 1_GiB; if (size > max_size) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in histogram."); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size in histogram (maximum: {})", max_size); buf.readStrict(reinterpret_cast(points), size * sizeof(WeightedValue)); } diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index 625a2511b0d..e31c62802f1 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -117,7 +117,7 @@ struct AggregateFunctionIntervalLengthSumData readBinary(size, buf); if (unlikely(size > MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {})", MAX_ARRAY_SIZE); segments.clear(); segments.reserve(size); diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 563d56c6f40..2c54293eeec 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -140,7 +140,8 @@ public: readVarUInt(size, buf); if (unlikely(size > AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE); auto & value = this->data(place).value; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 4fd7db4160e..77bd590ebbb 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -324,7 +324,8 @@ public: return; if (unlikely(size > max_node_size_deserialize)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", max_node_size_deserialize); auto & value = data(place).value; diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index c67621a99ce..a92d1979bab 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -58,7 +58,8 @@ struct QuantileExactBase size_t size = 0; readVarUInt(size, buf); if (unlikely(size > QUANTILE_EXACT_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", QUANTILE_EXACT_MAX_ARRAY_SIZE); array.resize(size); buf.readStrict(reinterpret_cast(array.data()), size * sizeof(array[0])); } diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index ef0e7c6566e..3d723d5aace 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -213,7 +213,8 @@ public: size_t size = std::min(total_values, sample_count); static constexpr size_t MAX_RESERVOIR_SIZE = 1_GiB; if (unlikely(size > MAX_RESERVOIR_SIZE)) - throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", MAX_RESERVOIR_SIZE); samples.resize(size); diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 5e1d23ed2c2..bde33260f5a 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -166,7 +166,8 @@ public: static constexpr size_t MAX_RESERVOIR_SIZE = 1_GiB; if (unlikely(size > MAX_RESERVOIR_SIZE)) - throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", MAX_RESERVOIR_SIZE); samples.resize(size); for (size_t i = 0; i < size; ++i) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index cac42c198b1..20ba73e0fa7 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -165,7 +165,8 @@ void readVectorBinary(std::vector & v, ReadBuffer & buf) readVarUInt(size, buf); if (size > DEFAULT_MAX_STRING_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size."); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", DEFAULT_MAX_STRING_SIZE); v.resize(size); for (size_t i = 0; i < size; ++i) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 501478c1163..a608219ed63 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1202,7 +1202,8 @@ void TCPHandler::receiveHello() throw Exception(ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT, "Client has connected to wrong port"); } else - throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet from client"); + throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, + "Unexpected packet from client (expected Hello, got {})", packet_type); } readStringBinary(client_name, *in); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d407f73d033..d85cd308702 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2053,7 +2053,7 @@ def reportLogStats(args): 'No sharding key', 'No tables', 'Query: {}', 'Removed', 'Removed part {}', 'Removing parts.', 'Request URI: {}', 'Sending part {}', 'Sent handshake', 'Starting {}', 'Will mimic {}', 'Writing to {}', 'dropIfEmpty', 'loadAll {}', '{} ({}:{})', '{} -> {}', '{} {}', '{}: {}', 'Query was cancelled', - 'Table {} already exists', '{}%', 'Cancelled merging parts', 'All replicas are lost', + 'Table {} already exists.', '{}%', 'Cancelled merging parts', 'All replicas are lost', 'Cancelled mutating parts', 'Read object: {}', 'New segment: {}', 'Unknown geometry type {}', 'Table {} is not replicated', '{} {}.{} already exists', 'Attempt to read after eof', 'Replica {} already exists', 'Convert overflow', 'key must be a tuple', 'Division by zero', @@ -2061,7 +2061,15 @@ def reportLogStats(args): 'Cannot parse datetime', 'Bad get: has {}, requested {}', 'There is no {} in {}', 'Numeric overflow', 'Polygon is not valid: {}', 'Decimal math overflow', '{} only accepts maps', 'Dictionary ({}) not found', 'Unknown format {}', 'Invalid IPv4 value', 'Invalid IPv6 value', 'Unknown setting {}', - 'Unknown table function {}' + 'Unknown table function {}', 'Database {} already exists.', 'Table {} doesn''t exist', + 'Invalid credentials', 'Part {} already exists', 'Invalid mode: {}', 'Log pulling is cancelled', + 'JOIN {} cannot get JOIN keys', 'Unknown function {}{}', 'Cannot parse IPv6 {}', + 'Not found address of host: {}', '{} must contain a tuple', 'Unknown codec family: {}', + 'Expected const String column', 'Invalid partition format: {}', 'Cannot parse IPv4 {}', + 'AST is too deep. Maximum: {}', 'Array sizes are too large: {}', 'Unable to connect to HDFS: {}', + 'Shutdown is called for table', 'File is not inside {}', + 'Table {} doesn''t exist', 'Database {} doesn''t exist', 'Table {}.{} doesn''t exist', + 'File {} doesn''t exist', 'No such attribute ''{}''', 'User name ''{}'' is reserved' ) AS known_short_messages SELECT count() AS c, message_format_string, substr(any(message), 1, 120) FROM system.text_log diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 032d7e396ff..fddfbd49de3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -2,7 +2,7 @@ runtime messages 0.001 runtime exceptions 0.05 messages shorter than 10 10 messages shorter than 16 40 -exceptions shorter than 30 125 +exceptions shorter than 30 120 noisy messages 0.3 noisy Trace messages 0.16 noisy Debug messages 0.09 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 2bcceec8cbe..71f41c7a9d2 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -21,7 +21,7 @@ select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_st select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 40) from logs where length(message_format_string) < 16; -- Same as above, but exceptions must be more informative. Feel free to update the threshold or remove this query if really necessary -select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 125) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%'; +select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 120) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%'; -- Avoid too noisy messages: top 1 message frequency must be less than 30%. We should reduce the threshold diff --git a/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh index e4f52b6e4ad..ca5e1245046 100755 --- a/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh +++ b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, race, no-ordinary-database +# Tags: long, zookeeper, race, no-ordinary-database, no-replicated-database +# FIXME remove no-replicated-database tag CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql b/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql index 615d42f12fa..8549300d49f 100644 --- a/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql +++ b/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql @@ -8,33 +8,33 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version) Order by (uid); INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); SELECT '== Test SELECT ... FINAL - no is_deleted =='; -select * from test FINAL; +select * from test FINAL order by uid; OPTIMIZE TABLE test FINAL CLEANUP; -select * from test; +select * from test order by uid; DROP TABLE IF EXISTS test; CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version) Order by (uid) SETTINGS clean_deleted_rows='Always'; INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); SELECT '== Test SELECT ... FINAL - no is_deleted SETTINGS clean_deleted_rows=Always =='; -select * from test FINAL; +select * from test FINAL order by uid; OPTIMIZE TABLE test FINAL CLEANUP; -select * from test; +select * from test order by uid; -- Test the new behaviour DROP TABLE IF EXISTS test; CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid); INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); SELECT '== Test SELECT ... FINAL =='; -select * from test FINAL; -select * from test; +select * from test FINAL order by uid; +select * from test order by uid; SELECT '== Insert backups =='; INSERT INTO test (*) VALUES ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1); -select * from test FINAL; +select * from test FINAL order by uid; SELECT '== Insert a second batch with overlaping data =='; INSERT INTO test (*) VALUES ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 1), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0), ('d2', 2, 1), ('d2', 3, 0), ('d3', 2, 1), ('d3', 3, 0); -select * from test FINAL; +select * from test FINAL order by uid; DROP TABLE IF EXISTS test; CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid); @@ -45,7 +45,7 @@ INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 2, 1); SELECT '== Only last version remains after OPTIMIZE W/ CLEANUP =='; OPTIMIZE TABLE test FINAL CLEANUP; -select * from test; +select * from test order by uid; -- insert d6 v=3 is_deleted=true (timestamp more recent so this version should be the one take into acount) INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 1); @@ -53,7 +53,7 @@ INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, SELECT '== OPTIMIZE W/ CLEANUP (remove d6) =='; OPTIMIZE TABLE test FINAL CLEANUP; -- No d6 anymore -select * from test; +select * from test order by uid; DROP TABLE IF EXISTS test; CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) SETTINGS clean_deleted_rows='Always'; @@ -61,12 +61,12 @@ CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = Replac SELECT '== Test of the SETTINGS clean_deleted_rows as Always =='; INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); -- Even if the setting is set to Always, the SELECT FINAL doesn't delete rows -select * from test FINAL; -select * from test; +select * from test FINAL order by uid; +select * from test order by uid; OPTIMIZE TABLE test FINAL; -- d6 has to be removed since we set clean_deleted_rows as 'Always' -select * from test; +select * from test order by uid; SELECT '== Test of the SETTINGS clean_deleted_rows as Never =='; ALTER TABLE test MODIFY SETTING clean_deleted_rows='Never'; @@ -74,7 +74,7 @@ INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); OPTIMIZE TABLE test FINAL; -- d6 has NOT to be removed since we set clean_deleted_rows as 'Never' -select * from test; +select * from test order by uid; DROP TABLE IF EXISTS testCleanupR1; @@ -92,7 +92,7 @@ OPTIMIZE TABLE testCleanupR1 FINAL CLEANUP; -- Only d3 to d5 remain SELECT '== (Replicas) Test optimize =='; -SELECT * FROM testCleanupR1; +SELECT * FROM testCleanupR1 order by uid; ------------------------------ @@ -110,7 +110,7 @@ OPTIMIZE TABLE testSettingsR1 FINAL; -- Only d3 to d5 remain SELECT '== (Replicas) Test settings =='; -SELECT * FROM testSettingsR1; +SELECT * FROM testSettingsR1 order by col1; ------------------------------ @@ -133,28 +133,28 @@ CREATE TABLE testMT (uid String, version UInt32, is_deleted UInt8) ENGINE = Merg INSERT INTO testMT (*) VALUES ('d1', 1, 1); OPTIMIZE TABLE testMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } OPTIMIZE TABLE testMT FINAL; -SELECT * FROM testMT; +SELECT * FROM testMT order by uid; CREATE TABLE testSummingMT (uid String, version UInt32, is_deleted UInt8) ENGINE = SummingMergeTree() Order by (uid) SETTINGS clean_deleted_rows='Always'; INSERT INTO testSummingMT (*) VALUES ('d1', 1, 1); OPTIMIZE TABLE testSummingMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } OPTIMIZE TABLE testSummingMT FINAL; -SELECT * FROM testSummingMT; +SELECT * FROM testSummingMT order by uid; CREATE TABLE testAggregatingMT (uid String, version UInt32, is_deleted UInt8) ENGINE = AggregatingMergeTree() Order by (uid) SETTINGS clean_deleted_rows='Always'; INSERT INTO testAggregatingMT (*) VALUES ('d1', 1, 1); OPTIMIZE TABLE testAggregatingMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } OPTIMIZE TABLE testAggregatingMT FINAL; -SELECT * FROM testAggregatingMT; +SELECT * FROM testAggregatingMT order by uid; CREATE TABLE testCollapsingMT (uid String, version UInt32, is_deleted UInt8, sign Int8) ENGINE = CollapsingMergeTree(sign) Order by (uid) SETTINGS clean_deleted_rows='Always'; INSERT INTO testCollapsingMT (*) VALUES ('d1', 1, 1, 1); OPTIMIZE TABLE testCollapsingMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } OPTIMIZE TABLE testCollapsingMT FINAL; -SELECT * FROM testCollapsingMT; +SELECT * FROM testCollapsingMT order by uid; CREATE TABLE testVersionedCMT (uid String, version UInt32, is_deleted UInt8, sign Int8) ENGINE = VersionedCollapsingMergeTree(sign, version) Order by (uid) SETTINGS clean_deleted_rows='Always'; INSERT INTO testVersionedCMT (*) VALUES ('d1', 1, 1, 1); OPTIMIZE TABLE testVersionedCMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } OPTIMIZE TABLE testVersionedCMT FINAL; -SELECT * FROM testVersionedCMT; +SELECT * FROM testVersionedCMT order by uid; From 225fee16dc82a079bee7517948742c04c396b656 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Apr 2023 19:46:53 +0200 Subject: [PATCH 192/233] Fix test "test_backup_all" and add more test cases. --- .../test_backup_restore_new/test.py | 35 ++++++++++++++----- 1 file changed, 27 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 7c3374ffe7d..424799e2402 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1184,7 +1184,8 @@ def test_restore_partition(): ) -def test_backup_all(): +@pytest.mark.parametrize("exclude_system_log_tables", [False, True]) +def test_backup_all(exclude_system_log_tables): create_and_fill_table() session_id = new_session_id() @@ -1201,20 +1202,34 @@ def test_backup_all(): instance.query("CREATE USER u1 IDENTIFIED BY 'qwe123' SETTINGS custom_a = 1") backup_name = new_backup_name() - instance.http_query( - f"BACKUP ALL TO {backup_name}", - params={"session_id": session_id}, - ) + + exclude_from_backup = [] + if exclude_system_log_tables: + system_log_tables = ( + instance.query( + "SELECT concat('system.', table) FROM system.tables WHERE (database = 'system') AND (table LIKE '%_log')" + ) + .rstrip("\n") + .split("\n") + ) + exclude_from_backup += system_log_tables + + backup_command = f"BACKUP ALL {'EXCEPT TABLES ' + ','.join(exclude_from_backup) if exclude_from_backup else ''} TO {backup_name}" + + instance.http_query(backup_command, params={"session_id": session_id}) instance.query("DROP TABLE test.table") instance.query("DROP FUNCTION two_and_half") instance.query("DROP USER u1") + restore_settings = [] + if not exclude_system_log_tables: + restore_settings.append("allow_non_empty_tables=true") + restore_command = f"RESTORE ALL FROM {backup_name} {'SETTINGS '+ ', '.join(restore_settings) if restore_settings else ''}" + session_id = new_session_id() instance.http_query( - f"RESTORE ALL FROM {backup_name}", - params={"session_id": session_id}, - method="POST", + restore_command, params={"session_id": session_id}, method="POST" ) assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -1230,6 +1245,10 @@ def test_backup_all(): == "CREATE USER u1 IDENTIFIED WITH sha256_password SETTINGS custom_a = 1\n" ) + instance.query("DROP TABLE test.table") + instance.query("DROP FUNCTION two_and_half") + instance.query("DROP USER u1") + def test_operation_id(): create_and_fill_table(n=30) From a28996d0477d32c6f1aac7de54a62a2f03c10ada Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Sat, 1 Apr 2023 18:50:49 -0400 Subject: [PATCH 193/233] Update docs/en/operations/tips.md --- docs/en/operations/tips.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index c5b489e92c5..6aff89fcff8 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -55,7 +55,7 @@ For Linux, software RAID is better (with `mdadm`). When creating RAID-10, select the `far` layout. If your budget allows, choose RAID-10. -LVM by itself (without RAID or mdadm) is ok, but making RAID with it or combining it with mdadm is a less explored option, and there will be more chances for mistakes +LVM by itself (without RAID or `mdadm`) is ok, but making RAID with it or combining it with mdadm is a less explored option, and there will be more chances for mistakes (selecting wrong chunk size; misalignment of chunks; choosing a wrong raid type; forgetting to cleanup disks). If you are confident in using LVM, there is nothing against using it. From 9061b039171db0e58dc6d28c1cf1cdaf80dc9ad5 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Sat, 1 Apr 2023 18:51:18 -0400 Subject: [PATCH 194/233] Update docs/en/operations/tips.md --- docs/en/operations/tips.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 6aff89fcff8..693dcd0d21b 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -55,7 +55,7 @@ For Linux, software RAID is better (with `mdadm`). When creating RAID-10, select the `far` layout. If your budget allows, choose RAID-10. -LVM by itself (without RAID or `mdadm`) is ok, but making RAID with it or combining it with mdadm is a less explored option, and there will be more chances for mistakes +LVM by itself (without RAID or `mdadm`) is ok, but making RAID with it or combining it with `mdadm` is a less explored option, and there will be more chances for mistakes (selecting wrong chunk size; misalignment of chunks; choosing a wrong raid type; forgetting to cleanup disks). If you are confident in using LVM, there is nothing against using it. From 62ecac805ffc22660335d7fc710f38480902f9ea Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Sat, 1 Apr 2023 18:58:48 -0400 Subject: [PATCH 195/233] Add mdadm to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3c5af582692..8f72f07d7ec 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -374,6 +374,7 @@ llvm localhost macOS mariadb +mdadm miniselect msgpack msgpk From 740a5ef777e0271603c03f1c4dc9ace4a712577e Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Sun, 12 Mar 2023 17:03:17 +0800 Subject: [PATCH 196/233] enable return null and complext type --- .../sql-reference/functions/json-functions.md | 2 +- src/Functions/FunctionSQLJSON.h | 17 +++++++++++------ .../01889_sql_json_functions.reference | 16 +++++++++------- .../0_stateless/01889_sql_json_functions.sql | 1 + 4 files changed, 22 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index bfe2a541647..4c2372561b7 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -401,7 +401,7 @@ Before version 21.11 the order of arguments was wrong, i.e. JSON_QUERY(path, jso Parses a JSON and extract a value as JSON scalar. -If the value does not exist, an empty string will be returned. +If the value does not exist, NULL will be returned. Example: diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index d649752768c..94d38d1951b 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -3,9 +3,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -233,7 +235,11 @@ class JSONValueImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + DataTypePtr string_type = std::make_shared(); + return std::make_shared(string_type); + } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } @@ -247,10 +253,7 @@ public: { if (status == VisitorStatus::Ok) { - if (!(current_element.isArray() || current_element.isObject())) - { - break; - } + break; } else if (status == VisitorStatus::Error) { @@ -267,7 +270,8 @@ public: std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out << current_element.getElement(); auto output_str = out.str(); - ColumnString & col_str = assert_cast(dest); + ColumnNullable & col_null = assert_cast(dest); + ColumnString & col_str = assert_cast(col_null.getNestedColumn()); ColumnString::Chars & data = col_str.getChars(); ColumnString::Offsets & offsets = col_str.getOffsets(); @@ -282,6 +286,7 @@ public: { col_str.insertData(output_str.data(), output_str.size()); } + col_null.getNullMapColumn().insertValue(0); return true; } }; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index c2c106e8632..be055e64ae7 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -1,8 +1,8 @@ -- { echo } SELECT '--JSON_VALUE--'; --JSON_VALUE-- -SELECT JSON_VALUE('{"hello":1}', '$'); -- root is a complex object => default value (empty string) - +SELECT JSON_VALUE('{"hello":1}', '$'); +{"hello":1} SELECT JSON_VALUE('{"hello":1}', '$.hello'); 1 SELECT JSON_VALUE('{"hello":1.2}', '$.hello'); @@ -14,13 +14,13 @@ world SELECT JSON_VALUE('{"hello":null}', '$.hello'); null SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); - +["world","world2"] SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); - -SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) - +{"world":"!"} +SELECT JSON_VALUE('{hello:world}', '$.hello'); +null SELECT JSON_VALUE('', '$.hello'); - +null SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); bar SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, World \\uD83C\\uDF37 \\uD83C\\uDF38 \\uD83C\\uDF3A"}', '$.hello'); @@ -31,6 +31,8 @@ select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); \n\0 select JSON_VALUE('{"a":"\\u263a"}', '$.a'); ☺ +select JSON_VALUE('{"a":"b"}', "$.b"); +null SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index e816443382c..622c4e3cb86 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -17,6 +17,7 @@ SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); select JSON_VALUE('{"a":"\\u263a"}', '$.a'); +select JSON_VALUE('{"a":"b"}', "$.b"); SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From f971b544e138dfd1166e073cded75fe8640d78ae Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Sun, 12 Mar 2023 17:25:30 +0800 Subject: [PATCH 197/233] modify test --- tests/queries/0_stateless/01889_sql_json_functions.reference | 2 +- tests/queries/0_stateless/01889_sql_json_functions.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index be055e64ae7..8275828e990 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -31,7 +31,7 @@ select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); \n\0 select JSON_VALUE('{"a":"\\u263a"}', '$.a'); ☺ -select JSON_VALUE('{"a":"b"}', "$.b"); +select JSON_VALUE('{"a":"b"}', '$.b'); null SELECT '--JSON_QUERY--'; --JSON_QUERY-- diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 622c4e3cb86..290706ff15d 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -17,7 +17,7 @@ SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); select JSON_VALUE('{"a":"\\u263a"}', '$.a'); -select JSON_VALUE('{"a":"b"}', "$.b"); +select JSON_VALUE('{"a":"b"}', '$.b'); SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From e0a10dc62fd18f0621a2b3d9eb872c11f7125fde Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Mon, 13 Mar 2023 18:19:23 +0800 Subject: [PATCH 198/233] enable nullable return type --- src/Core/Settings.h | 5 +++ src/Functions/FunctionSQLJSON.h | 38 ++++++++++++++++--- .../01889_sql_json_functions.reference | 6 +-- .../0_stateless/01889_sql_json_functions.sql | 2 +- 4 files changed, 41 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e9db155fb12..a9a210a6c58 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -941,10 +941,15 @@ class IColumn; M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \ \ M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \ +<<<<<<< f971b544e138dfd1166e073cded75fe8640d78ae M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ +======= + M(Bool, regexp_dict_allow_hyperscan, false, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ + M(Bool, function_return_type_allow_nullable, false, "Allow function to return nullable type.", 0) \ +>>>>>>> enable nullable return type // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 94d38d1951b..39c22bea73f 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -23,6 +23,7 @@ #include #include #include +#include #include "config.h" @@ -117,7 +118,6 @@ public: /// Parse JSON for every row Impl impl; - for (const auto i : collections::range(0, input_rows_count)) { std::string_view json{ @@ -156,7 +156,14 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - return Impl::getReturnType(Name::name, arguments); + if constexpr (has_static_member_function_getReturnType, DataTypePtr(const char *, const ColumnsWithTypeAndName &, const bool &)>::value) + { + return Impl::getReturnType(Name::name, arguments, getContext()->getSettingsRef().function_return_type_allow_nullable); + } + else + { + return Impl::getReturnType(Name::name, arguments); + } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -173,6 +180,9 @@ public: #endif return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth); } + +private: + BOOST_TTI_HAS_STATIC_MEMBER_FUNCTION(getReturnType) }; struct NameJSONExists @@ -235,10 +245,22 @@ class JSONValueImpl public: using Element = typename JSONParser::Element; + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool & allow_nullable) + { + if (allow_nullable) + { + DataTypePtr string_type = std::make_shared(); + return std::make_shared(string_type); + } + else + { + return std::make_shared(); + } + } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { - DataTypePtr string_type = std::make_shared(); - return std::make_shared(string_type); + return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } @@ -270,8 +292,9 @@ public: std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out << current_element.getElement(); auto output_str = out.str(); + bool is_nullable_col = typeid(dest) == typeid(ColumnNullable); ColumnNullable & col_null = assert_cast(dest); - ColumnString & col_str = assert_cast(col_null.getNestedColumn()); + ColumnString & col_str = is_nullable_col ? assert_cast(col_null.getNestedColumn()) : assert_cast(dest); ColumnString::Chars & data = col_str.getChars(); ColumnString::Offsets & offsets = col_str.getOffsets(); @@ -286,7 +309,10 @@ public: { col_str.insertData(output_str.data(), output_str.size()); } - col_null.getNullMapColumn().insertValue(0); + if (is_nullable_col) + { + col_null.getNullMapColumn().insertValue(0); + } return true; } }; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 8275828e990..d85decedda6 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -18,9 +18,9 @@ SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); {"world":"!"} SELECT JSON_VALUE('{hello:world}', '$.hello'); -null + SELECT JSON_VALUE('', '$.hello'); -null + SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); bar SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, World \\uD83C\\uDF37 \\uD83C\\uDF38 \\uD83C\\uDF3A"}', '$.hello'); @@ -31,7 +31,7 @@ select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); \n\0 select JSON_VALUE('{"a":"\\u263a"}', '$.a'); ☺ -select JSON_VALUE('{"a":"b"}', '$.b'); +select JSON_VALUE('{"a":"b"}', '$.b') settings function_return_type_allow_nullable=true; null SELECT '--JSON_QUERY--'; --JSON_QUERY-- diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 290706ff15d..c428f959050 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -17,7 +17,7 @@ SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); select JSON_VALUE('{"a":"\\u263a"}', '$.a'); -select JSON_VALUE('{"a":"b"}', '$.b'); +select JSON_VALUE('{"a":"b"}', '$.b') settings function_return_type_allow_nullable=true; SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From 05c508047adcfae00e74b74d479af6ee1a0f2989 Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Mon, 13 Mar 2023 18:22:02 +0800 Subject: [PATCH 199/233] docs fix --- docs/en/sql-reference/functions/json-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 4c2372561b7..bfe2a541647 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -401,7 +401,7 @@ Before version 21.11 the order of arguments was wrong, i.e. JSON_QUERY(path, jso Parses a JSON and extract a value as JSON scalar. -If the value does not exist, NULL will be returned. +If the value does not exist, an empty string will be returned. Example: From 212619919307599476ab372392b93ebd82f820d7 Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Mon, 13 Mar 2023 19:12:44 +0800 Subject: [PATCH 200/233] check style --- src/Functions/FunctionSQLJSON.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 39c22bea73f..e3f5592db03 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -156,7 +156,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if constexpr (has_static_member_function_getReturnType, DataTypePtr(const char *, const ColumnsWithTypeAndName &, const bool &)>::value) + if constexpr (has_static_member_function_getReturnType, DataTypePtr(const char *, const ColumnsWithTypeAndName &, const bool &)>::value) { return Impl::getReturnType(Name::name, arguments, getContext()->getSettingsRef().function_return_type_allow_nullable); } From 0935ccf0e08aa7f4dc1615bee2186d8300ea62aa Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Tue, 14 Mar 2023 10:26:53 +0800 Subject: [PATCH 201/233] ci fix --- src/Functions/FunctionSQLJSON.h | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index e3f5592db03..a6eaa155a94 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -292,9 +292,20 @@ public: std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out << current_element.getElement(); auto output_str = out.str(); + auto cast_to_column_string = [&] (IColumn & col, bool & is_nullable_col) -> ColumnString & + { + if (is_nullable_col) + { + ColumnNullable & col_null = assert_cast(col); + return assert_cast(col_null.getNestedColumn()); + } + else + { + return assert_cast(dest); + } + }; bool is_nullable_col = typeid(dest) == typeid(ColumnNullable); - ColumnNullable & col_null = assert_cast(dest); - ColumnString & col_str = is_nullable_col ? assert_cast(col_null.getNestedColumn()) : assert_cast(dest); + ColumnString & col_str = cast_to_column_string(dest, is_nullable_col); ColumnString::Chars & data = col_str.getChars(); ColumnString::Offsets & offsets = col_str.getOffsets(); @@ -311,6 +322,7 @@ public: } if (is_nullable_col) { + ColumnNullable & col_null = assert_cast(dest); col_null.getNullMapColumn().insertValue(0); } return true; From 3adc42af5c23bf1c58188a77d8ec2b998dad2a4f Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Wed, 15 Mar 2023 11:28:02 +0800 Subject: [PATCH 202/233] ci fix --- tests/queries/0_stateless/01889_sql_json_functions.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index d85decedda6..9f0e9cd64c2 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -17,7 +17,7 @@ SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); ["world","world2"] SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); {"world":"!"} -SELECT JSON_VALUE('{hello:world}', '$.hello'); +SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) SELECT JSON_VALUE('', '$.hello'); @@ -32,7 +32,7 @@ select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); select JSON_VALUE('{"a":"\\u263a"}', '$.a'); ☺ select JSON_VALUE('{"a":"b"}', '$.b') settings function_return_type_allow_nullable=true; -null +\N SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); From ce5f4cf4ab7c431755d2a63d17dda3f5386127ab Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Sat, 25 Mar 2023 21:54:45 +0800 Subject: [PATCH 203/233] code review fix --- docs/en/operations/settings/settings.md | 41 +++++++++ .../sql-reference/functions/json-functions.md | 4 +- src/Core/Settings.h | 5 -- src/Functions/FunctionSQLJSON.h | 86 ++++++++----------- .../01889_sql_json_functions.reference | 14 +-- .../0_stateless/01889_sql_json_functions.sql | 4 +- 6 files changed, 92 insertions(+), 62 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5ea555aa56a..1ebf13d36b9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4049,3 +4049,44 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Possible values: `true`, `false` Default value: `false` +## function_return_type_allow_nullable + +Control whether allow to return `NULL` when value is not exist for JSON_VALUE function. + +```sql +SELECT JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; + +┌─JSON_VALUE('{"hello":"world"}', '$.b')─┐ +│ ᴺᵁᴸᴸ │ +└────────────────────────────────────────┘ + +1 row in set. Elapsed: 0.001 sec. +``` + +Possible values: + +- true — Allow. +- false — Disallow. + +Default value: `false`. + +## function_json_value_return_type_allow_complex + +Control whether allow to return complex type (such as: struct, array, map) for json_value function. + +```sql +SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true + +┌─JSON_VALUE('{"hello":{"world":"!"}}', '$.hello')─┐ +│ {"world":"!"} │ +└──────────────────────────────────────────────────┘ + +1 row in set. Elapsed: 0.001 sec. +``` + +Possible values: + +- true — Allow. +- false — Disallow. + +Default value: `false`. diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index bfe2a541647..81697f901c1 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -401,7 +401,7 @@ Before version 21.11 the order of arguments was wrong, i.e. JSON_QUERY(path, jso Parses a JSON and extract a value as JSON scalar. -If the value does not exist, an empty string will be returned. +If the value does not exist, an empty string will be returned by default, and by SET `function_return_type_allow_nullable` = `true`, `NULL` will be returned. If the value is complex type (such as: struct, array, map), an empty string will be returned by default, and by SET `function_json_value_return_type_allow_complex` = `true`, the complex value will be returned. Example: @@ -410,6 +410,8 @@ SELECT JSON_VALUE('{"hello":"world"}', '$.hello'); SELECT JSON_VALUE('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); SELECT JSON_VALUE('{"hello":2}', '$.hello'); SELECT toTypeName(JSON_VALUE('{"hello":2}', '$.hello')); +select JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; ``` Result: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a9a210a6c58..7271cb2c84f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -941,15 +941,10 @@ class IColumn; M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \ \ M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \ -<<<<<<< f971b544e138dfd1166e073cded75fe8640d78ae M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - -======= M(Bool, regexp_dict_allow_hyperscan, false, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ - M(Bool, function_return_type_allow_nullable, false, "Allow function to return nullable type.", 0) \ ->>>>>>> enable nullable return type // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index a6eaa155a94..8684a5e39e5 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -23,7 +23,6 @@ #include #include #include -#include #include "config.h" @@ -43,7 +42,8 @@ public: class Executor { public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth) + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, + const bool & return_type_allow_complex) { MutableColumnPtr to{result_type->createColumn()}; to->reserve(input_rows_count); @@ -127,7 +127,7 @@ public: bool added_to_column = false; if (document_ok) { - added_to_column = impl.insertResultToColumn(*to, document, res); + added_to_column = impl.insertResultToColumn(*to, document, res, return_type_allow_complex); } if (!added_to_column) { @@ -156,14 +156,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if constexpr (has_static_member_function_getReturnType, DataTypePtr(const char *, const ColumnsWithTypeAndName &, const bool &)>::value) - { - return Impl::getReturnType(Name::name, arguments, getContext()->getSettingsRef().function_return_type_allow_nullable); - } - else - { - return Impl::getReturnType(Name::name, arguments); - } + return Impl::getReturnType(Name::name, arguments, getContext()); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -176,13 +169,12 @@ public: unsigned parse_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); #if USE_SIMDJSON if (getContext()->getSettingsRef().allow_simdjson) - return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth); + return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth, + getContext()->getSettingsRef().function_json_value_return_type_allow_complex); #endif - return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth); + return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth, + getContext()->getSettingsRef().function_json_value_return_type_allow_complex); } - -private: - BOOST_TTI_HAS_STATIC_MEMBER_FUNCTION(getReturnType) }; struct NameJSONExists @@ -206,11 +198,11 @@ class JSONExistsImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr &) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr) + static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const bool &) { GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; @@ -245,9 +237,9 @@ class JSONValueImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool & allow_nullable) + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr & context) { - if (allow_nullable) + if (context->getSettingsRef().function_return_type_allow_nullable) { DataTypePtr string_type = std::make_shared(); return std::make_shared(string_type); @@ -258,14 +250,9 @@ public: } } - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr) + static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const bool & return_type_allow_complex) { GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; @@ -275,7 +262,14 @@ public: { if (status == VisitorStatus::Ok) { - break; + if (return_type_allow_complex) + { + break; + } + else if (!(current_element.isArray() || current_element.isObject())) + { + break; + } } else if (status == VisitorStatus::Error) { @@ -292,22 +286,19 @@ public: std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out << current_element.getElement(); auto output_str = out.str(); - auto cast_to_column_string = [&] (IColumn & col, bool & is_nullable_col) -> ColumnString & + ColumnString * col_str; + if (isColumnNullable(dest)) { - if (is_nullable_col) - { - ColumnNullable & col_null = assert_cast(col); - return assert_cast(col_null.getNestedColumn()); - } - else - { - return assert_cast(dest); - } - }; - bool is_nullable_col = typeid(dest) == typeid(ColumnNullable); - ColumnString & col_str = cast_to_column_string(dest, is_nullable_col); - ColumnString::Chars & data = col_str.getChars(); - ColumnString::Offsets & offsets = col_str.getOffsets(); + ColumnNullable & col_null = assert_cast(dest); + col_null.getNullMapData().push_back(0); + col_str = assert_cast(&col_null.getNestedColumn()); + } + else + { + col_str = assert_cast(&dest); + } + ColumnString::Chars & data = col_str->getChars(); + ColumnString::Offsets & offsets = col_str->getOffsets(); if (current_element.isString()) { @@ -318,12 +309,7 @@ public: } else { - col_str.insertData(output_str.data(), output_str.size()); - } - if (is_nullable_col) - { - ColumnNullable & col_null = assert_cast(dest); - col_null.getNullMapColumn().insertValue(0); + col_str->insertData(output_str.data(), output_str.size()); } return true; } @@ -339,11 +325,11 @@ class JSONQueryImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr &) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr) + static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const bool &) { GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 9f0e9cd64c2..0d7fb270009 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -1,8 +1,8 @@ -- { echo } SELECT '--JSON_VALUE--'; --JSON_VALUE-- -SELECT JSON_VALUE('{"hello":1}', '$'); -{"hello":1} +SELECT JSON_VALUE('{"hello":1}', '$'); -- root is a complex object => default value (empty string) + SELECT JSON_VALUE('{"hello":1}', '$.hello'); 1 SELECT JSON_VALUE('{"hello":1.2}', '$.hello'); @@ -14,9 +14,9 @@ world SELECT JSON_VALUE('{"hello":null}', '$.hello'); null SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); -["world","world2"] + SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); -{"world":"!"} + SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) SELECT JSON_VALUE('', '$.hello'); @@ -31,8 +31,12 @@ select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); \n\0 select JSON_VALUE('{"a":"\\u263a"}', '$.a'); ☺ -select JSON_VALUE('{"a":"b"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; \N +select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; +{"world":"!"} +SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello') settings function_json_value_return_type_allow_complex=true; +["world","world2"] SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index c428f959050..71ef6ee2ada 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -17,7 +17,9 @@ SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); select JSON_VALUE('{"a":"\\u263a"}', '$.a'); -select JSON_VALUE('{"a":"b"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; +SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello') settings function_json_value_return_type_allow_complex=true; SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From 7d59f36336eb71fc42865ca890dc9e9b1928240b Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Sat, 25 Mar 2023 21:59:09 +0800 Subject: [PATCH 204/233] code style fix --- src/Functions/FunctionSQLJSON.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 8684a5e39e5..f31a1980acc 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -42,8 +42,7 @@ public: class Executor { public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, - const bool & return_type_allow_complex) + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, const bool & return_type_allow_complex) { MutableColumnPtr to{result_type->createColumn()}; to->reserve(input_rows_count); From da4ff587af1dc4c137912d9cc643b86cddfdee32 Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Sun, 2 Apr 2023 00:18:36 +0800 Subject: [PATCH 205/233] review fix --- docs/en/operations/settings/settings.md | 4 ++-- src/Functions/FunctionSQLJSON.h | 20 +++++++++---------- .../01889_sql_json_functions.reference | 2 +- .../0_stateless/01889_sql_json_functions.sql | 2 +- 4 files changed, 13 insertions(+), 15 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1ebf13d36b9..bb134c4d9d5 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4049,12 +4049,12 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Possible values: `true`, `false` Default value: `false` -## function_return_type_allow_nullable +## function_json_value_return_type_allow_nullable Control whether allow to return `NULL` when value is not exist for JSON_VALUE function. ```sql -SELECT JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; +SELECT JSON_VALUE('{"hello":"world"}', '$.b') settings function_json_value_return_type_allow_nullable=true; ┌─JSON_VALUE('{"hello":"world"}', '$.b')─┐ │ ᴺᵁᴸᴸ │ diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index f31a1980acc..9565ca5b242 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -42,7 +42,7 @@ public: class Executor { public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, const bool & return_type_allow_complex) + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, const ContextPtr & context) { MutableColumnPtr to{result_type->createColumn()}; to->reserve(input_rows_count); @@ -126,7 +126,7 @@ public: bool added_to_column = false; if (document_ok) { - added_to_column = impl.insertResultToColumn(*to, document, res, return_type_allow_complex); + added_to_column = impl.insertResultToColumn(*to, document, res, context); } if (!added_to_column) { @@ -168,11 +168,9 @@ public: unsigned parse_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); #if USE_SIMDJSON if (getContext()->getSettingsRef().allow_simdjson) - return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth, - getContext()->getSettingsRef().function_json_value_return_type_allow_complex); + return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth, getContext()); #endif - return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth, - getContext()->getSettingsRef().function_json_value_return_type_allow_complex); + return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth, getContext()); } }; @@ -201,7 +199,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const bool &) + static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &) { GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; @@ -238,7 +236,7 @@ public: static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr & context) { - if (context->getSettingsRef().function_return_type_allow_nullable) + if (context->getSettingsRef().function_json_value_return_type_allow_nullable) { DataTypePtr string_type = std::make_shared(); return std::make_shared(string_type); @@ -251,7 +249,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const bool & return_type_allow_complex) + static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr & context) { GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; @@ -261,7 +259,7 @@ public: { if (status == VisitorStatus::Ok) { - if (return_type_allow_complex) + if (context->getSettingsRef().function_json_value_return_type_allow_complex) { break; } @@ -328,7 +326,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const bool &) + static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &) { GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 0d7fb270009..5ac1ff501e5 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -31,7 +31,7 @@ select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); \n\0 select JSON_VALUE('{"a":"\\u263a"}', '$.a'); ☺ -select JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":"world"}', '$.b') settings function_json_value_return_type_allow_nullable=true; \N select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; {"world":"!"} diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 71ef6ee2ada..f174d04933c 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -17,7 +17,7 @@ SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); select JSON_VALUE('{"a":"\\u263a"}', '$.a'); -select JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":"world"}', '$.b') settings function_json_value_return_type_allow_nullable=true; select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello') settings function_json_value_return_type_allow_complex=true; From c2687c05443b0dd575d2e6b96322f47dfe124a85 Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Sun, 2 Apr 2023 14:14:35 +0800 Subject: [PATCH 206/233] review fix --- src/Core/Settings.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7271cb2c84f..7f6fa19d660 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -722,6 +722,8 @@ class IColumn; M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ M(UInt64, http_max_request_param_data_size, 10_MiB, "Limit on size of request data used as a query parameter in predefined HTTP requests.", 0) \ M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ + M(Bool, function_json_value_return_type_allow_nullable, false, "Allow function to return nullable type.", 0) \ + M(Bool, function_json_value_return_type_allow_complex, false, "Allow function to return complex type, such as: struct, array, map.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. @@ -944,7 +946,6 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, regexp_dict_allow_hyperscan, false, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. From 907ed27ae2e04db05f9fdaccbed6d9d8baa2c7c6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 2 Apr 2023 10:09:39 +0200 Subject: [PATCH 207/233] Fix crash in EXPLAIN PIPELINE for Merge over Distributed CI: https://s3.amazonaws.com/clickhouse-test-reports/48314/179450542879d11711cd2415c3fa7eeab18188be/fuzzer_astfuzzerasan/report.html Signed-off-by: Azat Khuzhin --- src/Interpreters/IInterpreterUnionOrSelectQuery.cpp | 5 ++++- src/Interpreters/IInterpreterUnionOrSelectQuery.h | 1 + src/Interpreters/InterpreterSelectQueryAnalyzer.h | 1 + src/Storages/StorageMerge.cpp | 8 +++++--- src/Storages/StorageMerge.h | 2 +- .../02704_storage_merge_explain_graph_crash.sql | 9 ++++++++- 6 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index b6e910eac94..4aa87346e80 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -18,9 +18,12 @@ namespace DB QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() { QueryPlan query_plan; + return buildQueryPipeline(query_plan); +} +QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline(QueryPlan & query_plan) +{ buildQueryPlan(query_plan); - return std::move(*query_plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context))); } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 1147070f48a..e4425a73505 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -35,6 +35,7 @@ public: virtual void buildQueryPlan(QueryPlan & query_plan) = 0; QueryPipelineBuilder buildQueryPipeline(); + QueryPipelineBuilder buildQueryPipeline(QueryPlan & query_plan); virtual void ignoreWithTotals() = 0; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index de97400e01b..33497c3ea16 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -72,6 +72,7 @@ public: void setProperClientInfo(size_t replica_number, size_t count_participating_replicas); const Planner & getPlanner() const { return planner; } + Planner & getPlanner() { return planner; } private: ASTPtr query; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 00ccfebff25..a76126c7879 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -649,14 +649,13 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryProcessingStage::Complete, storage_snapshot, modified_query_info); + if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { /// If there are only virtual columns in query, you must request at least one other column. if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); - /// Steps for reading from child tables should have the same lifetime as the current step - /// because `builder` can have references to them (mainly for EXPLAIN PIPELINE). QueryPlan & plan = child_plans.emplace_back(); StorageView * view = dynamic_cast(storage.get()); @@ -709,12 +708,15 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( modified_context->setSetting("max_threads", streams_num); modified_context->setSetting("max_streams_to_max_threads_ratio", 1); + QueryPlan & plan = child_plans.emplace_back(); + if (allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()); builder = std::make_unique(interpreter.buildQueryPipeline()); + plan = std::move(interpreter.getPlanner()).extractQueryPlan(); } else { @@ -723,7 +725,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()}; - builder = std::make_unique(interpreter.buildQueryPipeline()); + builder = std::make_unique(interpreter.buildQueryPipeline(plan)); } /** Materialization is needed, since from distributed storage the constants come materialized. diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index d53dcd34f5f..c4b6d815935 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -160,7 +160,7 @@ private: StorageSnapshotPtr merge_storage_snapshot; /// Store read plan for each child table. - /// It's needed to guarantee lifetime for child steps to be the same as for this step. + /// It's needed to guarantee lifetime for child steps to be the same as for this step (mainly for EXPLAIN PIPELINE). std::vector child_plans; SelectQueryInfo query_info; diff --git a/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql b/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql index b1725da6e82..44a8fe4f049 100644 --- a/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql +++ b/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql @@ -1,9 +1,16 @@ DROP TABLE IF EXISTS foo; +DROP TABLE IF EXISTS foo2; +DROP TABLE IF EXISTS foo2_dist; DROP TABLE IF EXISTS merge1; CREATE TABLE foo (`Id` Int32, `Val` Int32) ENGINE = MergeTree ORDER BY Id; INSERT INTO foo SELECT number, number FROM numbers(100); -CREATE TABLE merge1 AS foo ENGINE = Merge(currentDatabase(), '^foo'); +CREATE TABLE foo2 (`Id` Int32, `Val` Int32) ENGINE = MergeTree ORDER BY Id; +INSERT INTO foo2 SELECT number, number FROM numbers(100); +CREATE TABLE foo2_dist (`Id` UInt32, `Val` String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), foo2); + +CREATE TABLE merge1 AS foo ENGINE = Merge(currentDatabase(), '^(foo|foo2_dist)$'); EXPLAIN PIPELINE graph = 1, compact = 1 SELECT * FROM merge1 FORMAT Null; +EXPLAIN PIPELINE graph = 1, compact = 1 SELECT * FROM merge1 FORMAT Null SETTINGS allow_experimental_analyzer=1; From 9a051bc27ec20213e351216bc8c32c9c91d76adc Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 2 Apr 2023 12:30:36 -0300 Subject: [PATCH 208/233] some complex query (it fails with analyzer enabled --- ...707_complex_query_fails_analyzer.reference | 10 ++ .../02707_complex_query_fails_analyzer.sql | 117 ++++++++++++++++++ 2 files changed, 127 insertions(+) create mode 100644 tests/queries/0_stateless/02707_complex_query_fails_analyzer.reference create mode 100644 tests/queries/0_stateless/02707_complex_query_fails_analyzer.sql diff --git a/tests/queries/0_stateless/02707_complex_query_fails_analyzer.reference b/tests/queries/0_stateless/02707_complex_query_fails_analyzer.reference new file mode 100644 index 00000000000..192f8aa904a --- /dev/null +++ b/tests/queries/0_stateless/02707_complex_query_fails_analyzer.reference @@ -0,0 +1,10 @@ +1 1 -59.952 +1 2 59.952 +1 3 -100 +2 1 -93.7611 +2 2 93.7611 +3 1 0 +3 2 0 +--------- +0 +0 diff --git a/tests/queries/0_stateless/02707_complex_query_fails_analyzer.sql b/tests/queries/0_stateless/02707_complex_query_fails_analyzer.sql new file mode 100644 index 00000000000..a9d83479d50 --- /dev/null +++ b/tests/queries/0_stateless/02707_complex_query_fails_analyzer.sql @@ -0,0 +1,117 @@ +DROP TABLE IF EXISTS srv_account_parts; +DROP TABLE IF EXISTS etl_batch; + +CREATE TABLE srv_account_parts( + shard_num UInt16, + account_ids Array(Int64) +)ENGINE = ReplacingMergeTree +ORDER BY shard_num +as select * from values ((0,[]),(1,[1,2,3]),(2,[1,2,3]),(3,[1])); + +CREATE TABLE etl_batch( + batch_id UInt64, + batch_start DateTime, + batch_start_day Date DEFAULT toDate(batch_start), + batch_load DateTime, + total_num_records UInt32, + etl_server_id Int32, + account_id UInt64, + shard_num UInt16 +)ENGINE = ReplacingMergeTree +PARTITION BY toYYYYMM(batch_start_day) +ORDER BY (batch_id, etl_server_id, account_id); + +insert into etl_batch(batch_id, batch_start, batch_load, total_num_records, etl_server_id, account_id, shard_num) +select number batch_id, + toDateTime('2022-01-01') + INTERVAL 23 HOUR batch_start, + batch_start batch_load, + 333 total_num_records, + 1 etl_server_id, + number%3+1 account_id, + 1 shard_num +from numbers(1000); + +insert into etl_batch(batch_id, batch_start, batch_load, total_num_records, etl_server_id, account_id, shard_num) +select number+2000 batch_id, + toDateTime('2022-01-01') + INTERVAL 23 HOUR batch_start, + batch_start batch_load, + 333 total_num_records, + 1 etl_server_id, + number%3+1 account_id, + 2 shard_num +from numbers(1000); + +insert into etl_batch(batch_id, batch_start, batch_load, total_num_records, etl_server_id, account_id, shard_num) +select number+4000 batch_id, + toDateTime('2022-01-01') + INTERVAL 3 HOUR batch_start, + batch_start batch_load, + 3333 total_num_records, + 1 etl_server_id, + 2 account_id, + 2 shard_num +from numbers(1000); + +insert into etl_batch(batch_id, batch_start, batch_load, total_num_records, etl_server_id, account_id, shard_num) +select number+6000 batch_id, + toDateTime('2022-01-01') + INTERVAL 23 HOUR batch_start, + batch_start batch_load, + 333 total_num_records, + 1 etl_server_id, + 1 account_id, + 2 shard_num +from numbers(1000); + +insert into etl_batch(batch_id, batch_start, batch_load, total_num_records, etl_server_id, account_id, shard_num) +select number+8000 batch_id, + toDateTime('2022-01-01') + INTERVAL 23 HOUR batch_start, + batch_start batch_load, + 1000 total_num_records, + 1 etl_server_id, + 3 account_id, + 3 shard_num +from numbers(1000); + +CREATE OR REPLACE VIEW v_num_records_by_node_bias_acc as +SELECT shard_num, + arrayJoin(account_ids) AS account_id, + records_24h, + records_12h, + IF (b = '',-100,xbias) AS bias, + IF (bias > 10,0,IF (bias > 0,1,IF (bias < -10,301,300))) AS sbias +FROM srv_account_parts + LEFT JOIN (SELECT account_id, + shard_num, + records_24h, + records_12h, + xbias, + 'b' AS b + FROM (SELECT account_id, + groupArray((shard_num,records_24h,records_12h)) AS ga, + arraySum(ga.2) AS tot24, + arraySum(ga.3) AS tot12, + arrayMap(i ->(((((i.2)*LENGTH(ga))*100) / tot24) - 100),ga) AS bias24, + arrayMap(i ->(((((i.3)*LENGTH(ga))*100) / tot12) - 100),ga) AS bias12, + arrayMap((i,j,k) ->(i,IF (tot12 = 0,0,IF (ABS(j) > ABS(k),j,k))),ga,bias24,bias12) AS a_bias + FROM (SELECT shard_num, + toInt64(account_id) AS account_id, + SUM(total_num_records) AS records_24h, + sumIf(total_num_records,batch_load >(toDateTime('2022-01-02') -(3600*12))) AS records_12h + FROM etl_batch FINAL PREWHERE (batch_start_day >= (toDate('2022-01-02') - 2)) AND (batch_load > (toDateTime('2022-01-02') - (3600*24))) + where (shard_num, account_id) in (select shard_num, arrayJoin(account_ids) from srv_account_parts) + GROUP BY shard_num,account_id) + GROUP BY account_id) + ARRAY JOIN (a_bias.1).1 AS shard_num,a_bias.2 AS xbias, (a_bias.1).2 AS records_24h, (a_bias.1).3 AS records_12h + ) s USING (shard_num,account_id); + +select account_id, shard_num, round(bias,4) +from v_num_records_by_node_bias_acc +order by account_id, shard_num, bias; + +select '---------'; + +SELECT a AS b FROM (SELECT 0 a) s LEFT JOIN (SELECT 0 b) t USING (b); + +SELECT arrayJoin(a) AS b FROM (SELECT [0] a) s LEFT JOIN (SELECT 0 b) t USING (b); + +DROP TABLE srv_account_parts; +DROP TABLE etl_batch; From ff1cc5598f4406e22fef41c2f7018363ad356817 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 2 Apr 2023 22:21:10 +0000 Subject: [PATCH 209/233] fix clang-tidy --- src/Storages/StorageS3.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 0bccc4a419f..f4d915e9c55 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -299,7 +299,7 @@ public: protected: static StorageS3::Configuration copyAndUpdateConfiguration(ContextPtr local_context, const Configuration & configuration); - static void updateConfiguration(ContextPtr local_context, Configuration & configuration); + static void updateConfiguration(ContextPtr ctx, StorageS3::Configuration & upd); private: friend class StorageS3Cluster; From 21d22bcd3b08b50d51ac85cbe0af24938c52d674 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Apr 2023 01:06:26 +0200 Subject: [PATCH 210/233] Fix tests for constraints after merge --- .../01622_constraints_simple_optimization.sql | 6 ++--- .../01622_constraints_where_optimization.sql | 12 ++++----- .../01623_constraints_column_swap.sql | 26 +++++++++---------- .../01625_constraints_index_append.sh | 8 +++--- 4 files changed, 26 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 21d75a48587..a4d0035c590 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -100,10 +100,10 @@ SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); -- EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); ---> the order of the generated checks is not consistent EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100) SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100) SETTINGS allow_experimental_analyzer = 1; DROP TABLE constraint_test_constants; diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index 562c8705a51..2818351a120 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -8,15 +8,15 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15 SETTINGS allow_experimental_analyzer = 1; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20 SETTINGS allow_experimental_analyzer = 1; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2 SETTINGS allow_experimental_analyzer = 1; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8 SETTINGS allow_experimental_analyzer = 1; -- assumption -> remove (b < 20) EXPLAIN SYNTAX SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8 SETTINGS allow_experimental_analyzer = 1; -- assumption -> remove (b < 20) DROP TABLE t_constraints_where; @@ -25,6 +25,6 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5 SETTINGS allow_experimental_analyzer = 1; -- assumption -> (b < 20) -> 0; DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 6d70b78194d..3219ee3cda7 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -13,22 +13,22 @@ INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2); INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test_test WHERE b = 0; -EXPLAIN QUERY TREE SELECT b + 10, a FROM column_swap_test_test WHERE b = 0; +EXPLAIN QUERY TREE SELECT b + 10, a FROM column_swap_test_test WHERE b = 0 SETTINGS allow_experimental_analyzer = 1; DROP TABLE column_swap_test_test; @@ -36,13 +36,13 @@ CREATE TABLE column_swap_test_test (i Int64, a String, b String, CONSTRAINT c1 A INSERT INTO column_swap_test_test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c' SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS allow_experimental_analyzer = 1; DROP TABLE column_swap_test_test; @@ -53,6 +53,6 @@ CREATE TABLE t_bad_constraint(a UInt32, s String, CONSTRAINT c1 ASSUME a = toUIn INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000); EXPLAIN SYNTAX SELECT a FROM t_bad_constraint; -EXPLAIN QUERY TREE SELECT a FROM t_bad_constraint; +EXPLAIN QUERY TREE SELECT a FROM t_bad_constraint SETTINGS allow_experimental_analyzer = 1; DROP TABLE t_bad_constraint; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sh b/tests/queries/0_stateless/01625_constraints_index_append.sh index f17ea422409..6f2dfab7b14 100755 --- a/tests/queries/0_stateless/01625_constraints_index_append.sh +++ b/tests/queries/0_stateless/01625_constraints_index_append.sh @@ -24,12 +24,12 @@ function run_with_settings() } run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" $CLICKHOUSE_CLIENT --query "DROP TABLE index_append_test_test;" From ddf4ceda903effac920f96b711f6b4e5afbf08f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Apr 2023 01:23:06 +0200 Subject: [PATCH 211/233] Update references --- .../01622_constraints_simple_optimization.reference | 3 +++ .../01622_constraints_where_optimization.reference | 6 ++++++ .../01623_constraints_column_swap.reference | 13 +++++++++++++ 3 files changed, 22 insertions(+) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 529351180b3..a375c35ca3e 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -52,6 +52,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 @@ -69,6 +70,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() AS `count()` FROM constraint_test_constants QUERY id: 0 @@ -79,3 +81,4 @@ QUERY id: 0 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE TABLE id: 3, table_name: default.constraint_test_constants + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index 9bb42ed1c27..b5520d75b0e 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -11,6 +11,7 @@ QUERY id: 0 TABLE id: 3, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() FROM t_constraints_where WHERE 0 @@ -24,6 +25,7 @@ QUERY id: 0 TABLE id: 3, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() FROM t_constraints_where WHERE 0 @@ -37,6 +39,7 @@ QUERY id: 0 TABLE id: 3, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() FROM t_constraints_where WHERE b < 8 @@ -54,6 +57,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() FROM t_constraints_where PREWHERE (b > 20) OR (b < 8) @@ -71,6 +75,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT count() FROM t_constraints_where QUERY id: 0 @@ -81,3 +86,4 @@ QUERY id: 0 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE TABLE id: 3, table_name: default.t_constraints_where + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 124b5d06bed..3639ad47228 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -27,6 +27,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT cityHash64(a) + 10, b + 3 @@ -56,6 +57,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` @@ -85,6 +87,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` @@ -114,6 +117,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` @@ -143,6 +147,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` FROM column_swap_test_test WHERE b = 0 @@ -164,6 +169,7 @@ QUERY id: 0 LIST id: 8, nodes: 2 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT (cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, a AS a @@ -195,6 +201,7 @@ QUERY id: 0 LIST id: 13, nodes: 1 COLUMN id: 14, column_name: a, result_type: String, source_id: 7 CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT (cityHash64(a) AS b) + 10 AS `plus(b, 10)`, a AS a @@ -226,6 +233,7 @@ QUERY id: 0 LIST id: 13, nodes: 1 COLUMN id: 14, column_name: a, result_type: String, source_id: 7 CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT a AS `substring(reverse(b), 1, 1)`, a AS a @@ -247,6 +255,7 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'c\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 SELECT a AS `substring(reverse(b), 1, 1)`, a AS a @@ -268,6 +277,7 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'c\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 SELECT a AS t1, a AS t2 @@ -289,6 +299,7 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'c\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 SELECT a AS `substring(reverse(b), 1, 1)` FROM column_swap_test_test WHERE a = \'c\' @@ -306,6 +317,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: a, result_type: String, source_id: 3 CONSTANT id: 7, constant_value: \'c\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_bad_constraint QUERY id: 0 @@ -316,3 +328,4 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3 JOIN TREE TABLE id: 3, table_name: default.t_bad_constraint + SETTINGS allow_experimental_analyzer=1 From 34f1be92183c6aea3126547bc9421c0a8972ed40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Apr 2023 01:25:48 +0200 Subject: [PATCH 212/233] Fix test --- .../0_stateless/01625_constraints_index_append.sh | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sh b/tests/queries/0_stateless/01625_constraints_index_append.sh index 6f2dfab7b14..acceedbb1d1 100755 --- a/tests/queries/0_stateless/01625_constraints_index_append.sh +++ b/tests/queries/0_stateless/01625_constraints_index_append.sh @@ -19,17 +19,19 @@ function run_with_settings() , optimize_substitute_columns = 1\ , optimize_append_index = 1" + if [[ $query =~ "EXPLAIN QUERY TREE" ]]; then query="${query}, allow_experimental_analyzer = 1"; fi + $CLICKHOUSE_CLIENT --query="$query" } run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0" | grep -Fac "indexHint" run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0" | grep -Fac "indexHint" run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0" | grep -Fac "indexHint" run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100" -run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100 SETTINGS allow_experimental_analyzer = 1" | grep -Fac "indexHint" +run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100" | grep -Fac "indexHint" $CLICKHOUSE_CLIENT --query "DROP TABLE index_append_test_test;" From d7ee31638027f0fe9256cc5306f2e593281c0271 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 3 Apr 2023 07:24:08 +0000 Subject: [PATCH 213/233] Minor: Suggest ClickHouse-native function name over MySQL-compatibility alias --- .../functions/date-time-functions.md | 24 +++++++++---------- src/Functions/formatDateTime.cpp | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index d06ab253cf7..71b7fa07f18 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1463,28 +1463,28 @@ Result: └───────────────────────┘ ``` -## FROM\_UNIXTIME +## fromUnixTimestamp Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. -FROM_UNIXTIME uses MySQL datetime format style, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format. +fromUnixTimestamp uses MySQL datetime format style, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format. -Alias: `fromUnixTimestamp`. +Alias: `FROM_UNIXTIME`. **Example:** Query: ```sql -SELECT FROM_UNIXTIME(423543535); +SELECT fromUnixTimestamp(423543535); ``` Result: ```text -┌─FROM_UNIXTIME(423543535)─┐ -│ 1983-06-04 10:58:55 │ -└──────────────────────────┘ +┌─fromUnixTimestamp(423543535)─┐ +│ 1983-06-04 10:58:55 │ +└──────────────────────────────┘ ``` When there are two or three arguments, the first an [Integer](../../sql-reference/data-types/int-uint.md), [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md), the second a constant format string and the third an optional constant time zone string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. @@ -1492,7 +1492,7 @@ When there are two or three arguments, the first an [Integer](../../sql-referenc For example: ```sql -SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; +SELECT fromUnixTimestamp(1234334543, '%Y-%m-%d %R:%S') AS DateTime; ``` ```text @@ -1505,11 +1505,12 @@ SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; - [fromUnixTimestampInJodaSyntax](##fromUnixTimestampInJodaSyntax) - ## fromUnixTimestampInJodaSyntax -Similar to FROM_UNIXTIME, except that it formats time in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. + +Similar to fromUnixTimestamp, except that it formats time in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. **Example:** + Query: ``` sql SELECT fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC'); @@ -1517,12 +1518,11 @@ SELECT fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC'); Result: ``` -┌─fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC')─┐ +┌─fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC')────┐ │ 2022-11-30 10:41:12 │ └────────────────────────────────────────────────────────────────────────────┘ ``` - ## toModifiedJulianDay Converts a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) date in text form `YYYY-MM-DD` to a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) number in Int32. This function supports date from `0000-01-01` to `9999-12-31`. It raises an exception if the argument cannot be parsed as a date, or the date is invalid. diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index bbb4c3ba5b0..dd96a44c17b 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -1405,7 +1405,7 @@ REGISTER_FUNCTION(FormatDateTime) factory.registerAlias("DATE_FORMAT", FunctionFormatDateTime::name); factory.registerFunction(); - factory.registerAlias("FROM_UNIXTIME", "fromUnixTimestamp"); + factory.registerAlias("FROM_UNIXTIME", FunctionFromUnixTimestamp::name); factory.registerFunction(); factory.registerFunction(); From b4ea2268ca38603d12d4b0ade370924be3fb1930 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 3 Apr 2023 10:54:47 +0200 Subject: [PATCH 214/233] Adapt unit tests to the new exception --- src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp b/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp index 36b4ec10de0..583ba9c97de 100644 --- a/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp +++ b/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp @@ -198,7 +198,7 @@ TEST(MemoryWriteBuffer, WriteAndReread) if (s > 1) { MemoryWriteBuffer buf(s - 1); - EXPECT_THROW(buf.write(data.data(), data.size()), DB::Exception); + EXPECT_THROW(buf.write(data.data(), data.size()), MemoryWriteBuffer::CurrentBufferExhausted); } } From 40a0ecf66a35f23addcf9211642ff301eb4c897a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 09:30:39 +0000 Subject: [PATCH 215/233] Fix --- src/Storages/StorageKeeperMap.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 58d02372f2a..aeb206f1e05 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -93,7 +93,7 @@ class StorageKeeperMapSink : public SinkToStorage public: StorageKeeperMapSink(StorageKeeperMap & storage_, Block header, ContextPtr context_) - : SinkToStorage(std::move(header)), storage(storage_), context(std::move(context_)) + : SinkToStorage(header), storage(storage_), context(std::move(context_)) { auto primary_key = storage.getPrimaryKey(); assert(primary_key.size() == 1); @@ -171,7 +171,10 @@ public: zkutil::ZooKeeper::MultiExistsResponse results; if constexpr (!for_update) - results = zookeeper->exists(key_paths); + { + if (!strict) + results = zookeeper->exists(key_paths); + } Coordination::Requests requests; requests.reserve(key_paths.size()); @@ -189,11 +192,8 @@ public: } else { - if (results[i].error == Coordination::Error::ZOK) + if (!strict && results[i].error == Coordination::Error::ZOK) { - if (strict) - throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Value for key '{}' already exists", key); - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); } else @@ -937,8 +937,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca while (executor.pull(block)) sink->consume(Chunk{block.getColumns(), block.rows()}); - sink->finalize(local_context->getSettingsRef().keeper_map_strict_mode); - sink->onFinish(); + sink->finalize(strict); } namespace From 95661c13bcf3fb5576b80d3afe9b760e508ccf41 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 09:55:11 +0000 Subject: [PATCH 216/233] Add tests for strict update/delete --- ..._keeper_map_delete_update_strict.reference | 32 ++++++++++++++ .../02707_keeper_map_delete_update_strict.sql | 44 +++++++++++++++++++ 2 files changed, 76 insertions(+) create mode 100644 tests/queries/0_stateless/02707_keeper_map_delete_update_strict.reference create mode 100644 tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql diff --git a/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.reference b/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.reference new file mode 100644 index 00000000000..8ca8c0ca5a2 --- /dev/null +++ b/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.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/02707_keeper_map_delete_update_strict.sql b/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql new file mode 100644 index 00000000000..1e14675d353 --- /dev/null +++ b/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql @@ -0,0 +1,44 @@ +-- Tags: no-ordinary-database, no-fasttest + +DROP TABLE IF EXISTS 02661_keepermap_delete_update; + +SET keeper_map_strict_mode = 1; + +CREATE TABLE 02661_keepermap_delete_update (key UInt64, value String, value2 UInt64) ENGINE=KeeperMap('/' || currentDatabase() || '/test02661_keepermap_delete_update') PRIMARY KEY(key); + +INSERT INTO 02661_keepermap_delete_update VALUES (1, 'Some string', 0), (2, 'Some other string', 0), (3, 'random', 0), (4, 'random2', 0); + +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +DELETE FROM 02661_keepermap_delete_update WHERE value LIKE 'Some%string'; + +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02661_keepermap_delete_update DELETE WHERE key >= 4; + +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +DELETE FROM 02661_keepermap_delete_update WHERE 1 = 1; +SELECT count() FROM 02661_keepermap_delete_update; +SELECT '-----------'; + +INSERT INTO 02661_keepermap_delete_update VALUES (1, 'String', 10), (2, 'String', 20), (3, 'String', 30), (4, 'String', 40); +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02661_keepermap_delete_update UPDATE value = 'Another' WHERE key > 2; +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02661_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError 36 } +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +ALTER TABLE 02661_keepermap_delete_update UPDATE value2 = value2 * 10 + 2 WHERE value2 < 100; +SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT '-----------'; + +DROP TABLE IF EXISTS 02661_keepermap_delete_update; From e79343c169fb0534526ee11353d497681edec7e5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 3 Apr 2023 10:33:10 +0000 Subject: [PATCH 217/233] Make the column order in system.query_cache more intutitive --- .../System/StorageSystemQueryCache.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 5e4a1e662e2..612322e4225 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -13,12 +13,12 @@ NamesAndTypesList StorageSystemQueryCache::getNamesAndTypes() { return { {"query", std::make_shared()}, - {"key_hash", std::make_shared()}, - {"expires_at", std::make_shared()}, + {"result_size", std::make_shared()}, {"stale", std::make_shared()}, - {"compressed", std::make_shared()}, {"shared", std::make_shared()}, - {"result_size", std::make_shared()} + {"compressed", std::make_shared()}, + {"expires_at", std::make_shared()}, + {"key_hash", std::make_shared()} }; } @@ -45,12 +45,12 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr continue; res_columns[0]->insert(key.queryStringFromAst()); /// approximates the original query string - res_columns[1]->insert(key.ast->getTreeHash().first); - res_columns[2]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); - res_columns[3]->insert(key.expires_at < std::chrono::system_clock::now()); + res_columns[1]->insert(QueryCache::QueryResultWeight()(*query_result)); + res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); + res_columns[3]->insert(!key.username.has_value()); res_columns[4]->insert(key.is_compressed); - res_columns[5]->insert(!key.username.has_value()); - res_columns[6]->insert(QueryCache::QueryResultWeight()(*query_result)); + res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); + res_columns[6]->insert(key.ast->getTreeHash().first); } } From adfedf692a68cf9e93a4d6103c22aea19d7c019f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 13:13:19 +0200 Subject: [PATCH 218/233] Update table name --- .../02707_keeper_map_delete_update_strict.sql | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql b/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql index 1e14675d353..aaf4f2fd838 100644 --- a/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql +++ b/tests/queries/0_stateless/02707_keeper_map_delete_update_strict.sql @@ -1,44 +1,44 @@ -- Tags: no-ordinary-database, no-fasttest -DROP TABLE IF EXISTS 02661_keepermap_delete_update; +DROP TABLE IF EXISTS 02707_keepermap_delete_update; SET keeper_map_strict_mode = 1; -CREATE TABLE 02661_keepermap_delete_update (key UInt64, value String, value2 UInt64) ENGINE=KeeperMap('/' || currentDatabase() || '/test02661_keepermap_delete_update') PRIMARY KEY(key); +CREATE TABLE 02707_keepermap_delete_update (key UInt64, value String, value2 UInt64) ENGINE=KeeperMap('/' || currentDatabase() || '/test02707_keepermap_delete_update') PRIMARY KEY(key); -INSERT INTO 02661_keepermap_delete_update VALUES (1, 'Some string', 0), (2, 'Some other string', 0), (3, 'random', 0), (4, 'random2', 0); +INSERT INTO 02707_keepermap_delete_update VALUES (1, 'Some string', 0), (2, 'Some other string', 0), (3, 'random', 0), (4, 'random2', 0); -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -DELETE FROM 02661_keepermap_delete_update WHERE value LIKE 'Some%string'; +DELETE FROM 02707_keepermap_delete_update WHERE value LIKE 'Some%string'; -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -ALTER TABLE 02661_keepermap_delete_update DELETE WHERE key >= 4; +ALTER TABLE 02707_keepermap_delete_update DELETE WHERE key >= 4; -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -DELETE FROM 02661_keepermap_delete_update WHERE 1 = 1; -SELECT count() FROM 02661_keepermap_delete_update; +DELETE FROM 02707_keepermap_delete_update WHERE 1 = 1; +SELECT count() FROM 02707_keepermap_delete_update; SELECT '-----------'; -INSERT INTO 02661_keepermap_delete_update VALUES (1, 'String', 10), (2, 'String', 20), (3, 'String', 30), (4, 'String', 40); -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +INSERT INTO 02707_keepermap_delete_update VALUES (1, 'String', 10), (2, 'String', 20), (3, 'String', 30), (4, 'String', 40); +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -ALTER TABLE 02661_keepermap_delete_update UPDATE value = 'Another' WHERE key > 2; -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +ALTER TABLE 02707_keepermap_delete_update UPDATE value = 'Another' WHERE key > 2; +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -ALTER TABLE 02661_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError 36 } -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +ALTER TABLE 02707_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError 36 } +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -ALTER TABLE 02661_keepermap_delete_update UPDATE value2 = value2 * 10 + 2 WHERE value2 < 100; -SELECT * FROM 02661_keepermap_delete_update ORDER BY key; +ALTER TABLE 02707_keepermap_delete_update UPDATE value2 = value2 * 10 + 2 WHERE value2 < 100; +SELECT * FROM 02707_keepermap_delete_update ORDER BY key; SELECT '-----------'; -DROP TABLE IF EXISTS 02661_keepermap_delete_update; +DROP TABLE IF EXISTS 02707_keepermap_delete_update; From 11556a23adbc462d74d80eb2505c2133f8bc761d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 3 Apr 2023 10:47:29 +0000 Subject: [PATCH 219/233] Extend tests for compression of query cache entries --- .../02494_query_cache_compression.reference | 6532 +++++++++++++++++ .../02494_query_cache_compression.sql | 444 ++ ...query_cache_disabled_compression.reference | 2 - ...02494_query_cache_disabled_compression.sql | 12 - ...ery_cache_squash_partial_results.reference | 7 +- ...494_query_cache_squash_partial_results.sql | 23 +- 6 files changed, 6994 insertions(+), 26 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_compression.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_compression.sql delete mode 100644 tests/queries/0_stateless/02494_query_cache_disabled_compression.reference delete mode 100644 tests/queries/0_stateless/02494_query_cache_disabled_compression.sql diff --git a/tests/queries/0_stateless/02494_query_cache_compression.reference b/tests/queries/0_stateless/02494_query_cache_compression.reference new file mode 100644 index 00000000000..1d206bbc4d2 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_compression.reference @@ -0,0 +1,6532 @@ +-- insert with enabled compression +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +-- read from cache +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +-- insert with disabled compression +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +-- read from cache +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +def +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl +jkl diff --git a/tests/queries/0_stateless/02494_query_cache_compression.sql b/tests/queries/0_stateless/02494_query_cache_compression.sql new file mode 100644 index 00000000000..0f527dfde5e --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_compression.sql @@ -0,0 +1,444 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SET allow_experimental_query_cache = true; + +SYSTEM DROP QUERY CACHE; + +DROP TABLE IF EXISTS t; + +-- Create test table with lot's of rows +CREATE TABLE t(c String) ENGINE=MergeTree ORDER BY c; +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +INSERT INTO t values ('abc') ('def') ('abc') ('jkl'); +OPTIMIZE TABLE t FINAL; + +-- Run query which, store *compressed* result in query cache +SELECT '-- insert with enabled compression'; +SELECT * FROM t ORDER BY c +SETTINGS use_query_cache = true, query_cache_compress_entries = true; + +-- Run again to check that no bad things happen and that the result is as expected +SELECT '-- read from cache'; +SELECT * FROM t ORDER BY c +SETTINGS use_query_cache = true; + +SYSTEM DROP QUERY CACHE; + +-- Run query which, store *uncompressed* result in query cache +SELECT '-- insert with disabled compression'; +SELECT * FROM t ORDER BY c +SETTINGS use_query_cache = true, query_cache_compress_entries = false; + +-- Run again to check that no bad things happen and that the result is as expected +SELECT '-- read from cache'; +SELECT * FROM t ORDER BY c +SETTINGS use_query_cache = true; + +DROP TABLE t; diff --git a/tests/queries/0_stateless/02494_query_cache_disabled_compression.reference b/tests/queries/0_stateless/02494_query_cache_disabled_compression.reference deleted file mode 100644 index 6ed281c757a..00000000000 --- a/tests/queries/0_stateless/02494_query_cache_disabled_compression.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 diff --git a/tests/queries/0_stateless/02494_query_cache_disabled_compression.sql b/tests/queries/0_stateless/02494_query_cache_disabled_compression.sql deleted file mode 100644 index ca95ffd918d..00000000000 --- a/tests/queries/0_stateless/02494_query_cache_disabled_compression.sql +++ /dev/null @@ -1,12 +0,0 @@ --- Tags: no-parallel --- Tag no-parallel: Messes with internal cache - -SET allow_experimental_query_cache = true; - -SYSTEM DROP QUERY CACHE; - --- Run query and store result in query cache but without compression which is on by default -SELECT 1 SETTINGS use_query_cache = true, query_cache_compress_entries = false; - --- Run again to check that no bad things happen and that the result is as expected -SELECT 1 SETTINGS use_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference index e3ffe57ae3e..2c4fa587dfc 100644 --- a/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference +++ b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.reference @@ -1,3 +1,4 @@ +-- insert with enabled squashing abc abc abc @@ -66,7 +67,7 @@ jkl jkl jkl jkl -- +-- read from cache abc abc abc @@ -135,7 +136,7 @@ jkl jkl jkl jkl --------------------- +-- insert with disabled squashing abc abc abc @@ -204,7 +205,7 @@ jkl jkl jkl jkl -- +-- read from cache abc abc abc diff --git a/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql index eee633b747e..d57773b51f8 100644 --- a/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql +++ b/tests/queries/0_stateless/02494_query_cache_squash_partial_results.sql @@ -9,6 +9,7 @@ DROP TABLE IF EXISTS t; -- Create test table with "many" rows CREATE TABLE t(c String) ENGINE=MergeTree ORDER BY c; +SYSTEM STOP MERGES t; -- retain multiple parts to make the SELECT process multiple chunks INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); @@ -28,21 +29,25 @@ INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); INSERT INTO t values ('abc') ('def') ('ghi') ('jkl'); -- Run query which reads multiple chunks (small max_block_size), cache result in query cache, force squashing of partial results -SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true, query_cache_squash_partial_results = true; - -SELECT '-'; +SELECT '-- insert with enabled squashing'; +SELECT * FROM t ORDER BY c +SETTINGS max_block_size = 2, use_query_cache = true, query_cache_squash_partial_results = true; -- Run again to check that no bad things happen and that the result is as expected -SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true; +SELECT '-- read from cache'; +SELECT * FROM t ORDER BY c +SETTINGS max_block_size = 2, use_query_cache = true; -SELECT '--------------------'; +SYSTEM DROP QUERY CACHE; -- Run query which reads multiple chunks (small max_block_size), cache result in query cache, but **disable** squashing of partial results -SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true, query_cache_squash_partial_results = false; - -SELECT '-'; +SELECT '-- insert with disabled squashing'; +SELECT * FROM t ORDER BY c +SETTINGS max_block_size = 2, use_query_cache = true, query_cache_squash_partial_results = false; -- Run again to check that no bad things happen and that the result is as expected -SELECT * FROM t ORDER BY c SETTINGS max_block_size = 2, use_query_cache = true; +SELECT '-- read from cache'; +SELECT * FROM t ORDER BY c +SETTINGS max_block_size = 2, use_query_cache = true; DROP TABLE t; From 781907bda87a314828c26b6c61ea8cf1da891b21 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 3 Apr 2023 12:10:45 +0000 Subject: [PATCH 220/233] Update version_date.tsv and changelogs after v23.1.6.42-stable --- docs/changelogs/v23.1.6.42-stable.md | 34 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 36 insertions(+) create mode 100644 docs/changelogs/v23.1.6.42-stable.md diff --git a/docs/changelogs/v23.1.6.42-stable.md b/docs/changelogs/v23.1.6.42-stable.md new file mode 100644 index 00000000000..21fb9220443 --- /dev/null +++ b/docs/changelogs/v23.1.6.42-stable.md @@ -0,0 +1,34 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.1.6.42-stable (783ddf67991) FIXME as compared to v23.1.5.24-stable (0e51b53ba99) + +#### Build/Testing/Packaging Improvement +* Backported in [#48215](https://github.com/ClickHouse/ClickHouse/issues/48215): Use sccache as a replacement for ccache and using S3 as cache backend. [#46240](https://github.com/ClickHouse/ClickHouse/pull/46240) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#48254](https://github.com/ClickHouse/ClickHouse/issues/48254): The `clickhouse/clickhouse-keeper` image used to be pushed only with tags `-alpine`, e.g. `latest-alpine`. As it was suggested in https://github.com/ClickHouse/examples/pull/2, now it will be pushed as suffixless too. [#48236](https://github.com/ClickHouse/ClickHouse/pull/48236) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix changing an expired role [#46772](https://github.com/ClickHouse/ClickHouse/pull/46772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix bug in zero-copy replication disk choice during fetch [#47010](https://github.com/ClickHouse/ClickHouse/pull/47010) ([alesapin](https://github.com/alesapin)). +* Fix NOT_IMPLEMENTED error with CROSS JOIN and algorithm = auto [#47068](https://github.com/ClickHouse/ClickHouse/pull/47068) ([Vladimir C](https://github.com/vdimir)). +* Disable logical expression optimizer for expression with aliases. [#47451](https://github.com/ClickHouse/ClickHouse/pull/47451) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix query parameters [#47488](https://github.com/ClickHouse/ClickHouse/pull/47488) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Parameterized view bug fix 47287 47247 [#47495](https://github.com/ClickHouse/ClickHouse/pull/47495) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix wait for zero copy lock during move [#47631](https://github.com/ClickHouse/ClickHouse/pull/47631) ([alesapin](https://github.com/alesapin)). +* Hotfix for too verbose warnings in HTTP [#47903](https://github.com/ClickHouse/ClickHouse/pull/47903) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Better error messages in ReplicatedMergeTreeAttachThread [#47454](https://github.com/ClickHouse/ClickHouse/pull/47454) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `00933_test_fix_extra_seek_on_compressed_cache` in releases. [#47490](https://github.com/ClickHouse/ClickHouse/pull/47490) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a fuse for backport branches w/o a created PR [#47760](https://github.com/ClickHouse/ClickHouse/pull/47760) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Only valid Reviews.STATES overwrite existing reviews [#47789](https://github.com/ClickHouse/ClickHouse/pull/47789) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Place short return before big block, improve logging [#47822](https://github.com/ClickHouse/ClickHouse/pull/47822) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Artifacts s3 prefix [#47945](https://github.com/ClickHouse/ClickHouse/pull/47945) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tsan error lock-order-inversion [#47953](https://github.com/ClickHouse/ClickHouse/pull/47953) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0cd3416f44c..24af79eef2f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,8 +1,10 @@ v23.3.1.2823-lts 2023-03-31 +v23.2.5.46-stable 2023-04-03 v23.2.4.12-stable 2023-03-10 v23.2.3.17-stable 2023-03-06 v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 +v23.1.6.42-stable 2023-04-03 v23.1.5.24-stable 2023-03-10 v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 From 00e335530e690f996a26f566ecee032fe9489bbc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 3 Apr 2023 08:15:18 -0400 Subject: [PATCH 221/233] close client --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 9067a8142bc..d0dfdd783f1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -144,7 +144,7 @@ def clickhouse_execute_http( except Exception as ex: if i == max_http_retries - 1: raise ex - + client.close() sleep(i + 1) if res.status != 200: From 0af16e4b7a5a253e5f4f42a86fd318908283d73c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 3 Apr 2023 12:16:27 +0000 Subject: [PATCH 222/233] Update version_date.tsv and changelogs after v23.2.5.46-stable --- docs/changelogs/v23.2.5.46-stable.md | 40 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 42 insertions(+) create mode 100644 docs/changelogs/v23.2.5.46-stable.md diff --git a/docs/changelogs/v23.2.5.46-stable.md b/docs/changelogs/v23.2.5.46-stable.md new file mode 100644 index 00000000000..b3ce585848b --- /dev/null +++ b/docs/changelogs/v23.2.5.46-stable.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.2.5.46-stable (b50faecbb12) FIXME as compared to v23.2.4.12-stable (8fe866cb035) + +#### Improvement +* Backported in [#48164](https://github.com/ClickHouse/ClickHouse/issues/48164): Fixed `UNKNOWN_TABLE` exception when attaching to a materialized view that has dependent tables that are not available. This might be useful when trying to restore state from a backup. [#47975](https://github.com/ClickHouse/ClickHouse/pull/47975) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#48216](https://github.com/ClickHouse/ClickHouse/issues/48216): Use sccache as a replacement for ccache and using S3 as cache backend. [#46240](https://github.com/ClickHouse/ClickHouse/pull/46240) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#48256](https://github.com/ClickHouse/ClickHouse/issues/48256): The `clickhouse/clickhouse-keeper` image used to be pushed only with tags `-alpine`, e.g. `latest-alpine`. As it was suggested in https://github.com/ClickHouse/examples/pull/2, now it will be pushed as suffixless too. [#48236](https://github.com/ClickHouse/ClickHouse/pull/48236) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix changing an expired role [#46772](https://github.com/ClickHouse/ClickHouse/pull/46772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix bug in zero-copy replication disk choice during fetch [#47010](https://github.com/ClickHouse/ClickHouse/pull/47010) ([alesapin](https://github.com/alesapin)). +* Fix NOT_IMPLEMENTED error with CROSS JOIN and algorithm = auto [#47068](https://github.com/ClickHouse/ClickHouse/pull/47068) ([Vladimir C](https://github.com/vdimir)). +* Disable logical expression optimizer for expression with aliases. [#47451](https://github.com/ClickHouse/ClickHouse/pull/47451) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix query parameters [#47488](https://github.com/ClickHouse/ClickHouse/pull/47488) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Parameterized view bug fix 47287 47247 [#47495](https://github.com/ClickHouse/ClickHouse/pull/47495) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Proper fix for bug in parquet, revert reverted [#45878](https://github.com/ClickHouse/ClickHouse/issues/45878) [#47538](https://github.com/ClickHouse/ClickHouse/pull/47538) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix wait for zero copy lock during move [#47631](https://github.com/ClickHouse/ClickHouse/pull/47631) ([alesapin](https://github.com/alesapin)). +* Hotfix for too verbose warnings in HTTP [#47903](https://github.com/ClickHouse/ClickHouse/pull/47903) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* fix: keeper systemd service file include invalid inline comment [#47105](https://github.com/ClickHouse/ClickHouse/pull/47105) ([SuperDJY](https://github.com/cmsxbc)). +* Better error messages in ReplicatedMergeTreeAttachThread [#47454](https://github.com/ClickHouse/ClickHouse/pull/47454) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `00933_test_fix_extra_seek_on_compressed_cache` in releases. [#47490](https://github.com/ClickHouse/ClickHouse/pull/47490) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix startup on older systemd versions [#47689](https://github.com/ClickHouse/ClickHouse/pull/47689) ([Thomas Casteleyn](https://github.com/Hipska)). +* Add a fuse for backport branches w/o a created PR [#47760](https://github.com/ClickHouse/ClickHouse/pull/47760) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Only valid Reviews.STATES overwrite existing reviews [#47789](https://github.com/ClickHouse/ClickHouse/pull/47789) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Place short return before big block, improve logging [#47822](https://github.com/ClickHouse/ClickHouse/pull/47822) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Artifacts s3 prefix [#47945](https://github.com/ClickHouse/ClickHouse/pull/47945) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tsan error lock-order-inversion [#47953](https://github.com/ClickHouse/ClickHouse/pull/47953) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0cd3416f44c..24af79eef2f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,8 +1,10 @@ v23.3.1.2823-lts 2023-03-31 +v23.2.5.46-stable 2023-04-03 v23.2.4.12-stable 2023-03-10 v23.2.3.17-stable 2023-03-06 v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 +v23.1.6.42-stable 2023-04-03 v23.1.5.24-stable 2023-03-10 v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 From fb7c8ca1575610a85523db927cb8acc05aaa41ae Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 3 Apr 2023 10:47:44 -0300 Subject: [PATCH 223/233] Update tuple-functions.md --- docs/en/sql-reference/functions/tuple-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index c248499be69..c4742d0bac7 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -208,7 +208,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-referen Query: ``` sql -CREATE TABLE tupletest (`col` Tuple(user_ID UInt64, session_ID UInt64) ENGINE = Memory; +CREATE TABLE tupletest (col Tuple(user_ID UInt64, session_ID UInt64)) ENGINE = Memory; INSERT INTO tupletest VALUES (tuple( 100, 2502)), (tuple(1,100)); @@ -227,11 +227,11 @@ Result: It is possible to transform colums to rows using this function: ``` sql -CREATE TABLE tupletest (`col` Tuple(CPU Float64, Memory Float64, Disk Float64)) ENGINE = Memory; +CREATE TABLE tupletest (col Tuple(CPU Float64, Memory Float64, Disk Float64)) ENGINE = Memory; INSERT INTO tupletest VALUES(tuple(3.3, 5.5, 6.6)); -SELECT arrayJoin(tupleToNameValuePairs(col))FROM tupletest; +SELECT arrayJoin(tupleToNameValuePairs(col)) FROM tupletest; ``` Result: From e7d00c8f340497dde2a84325d3f1baf8616f74c0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 13:56:03 +0000 Subject: [PATCH 224/233] Don't replicate mutations for KeeperMap tables --- src/Databases/DatabaseReplicated.cpp | 8 ++++++++ src/Interpreters/InterpreterAlterQuery.cpp | 12 +++++++++--- .../0_stateless/02577_keepermap_delete_update.sql | 4 +++- 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 76eea059174..efac04d9e15 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -34,6 +34,7 @@ #include #include #include +#include namespace DB { @@ -1390,6 +1391,13 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, /// Some ALTERs are not replicated on database level if (const auto * alter = query_ptr->as()) { + auto table_id = query_context->resolveStorageID(*alter, Context::ResolveOrdinary); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, query_context); + + /// we never replicate KeeperMap operations because it doesn't make sense + if (auto * keeper_map = table->as()) + return false; + return !alter->isAttachAlter() && !alter->isFetchAlter() && !alter->isDropPartitionAlter(); } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index fabcc6844e5..3dfa29fbb01 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -39,6 +40,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; + extern const int BAD_ARGUMENTS; } @@ -72,16 +74,21 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!UserDefinedSQLFunctionFactory::instance().empty()) UserDefinedSQLFunctionVisitor::visit(query_ptr); + auto table_id = getContext()->resolveStorageID(alter, Context::ResolveOrdinary); + query_ptr->as().setDatabase(table_id.database_name); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!alter.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { + if (table->as()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutations with ON CLUSTER are not allowed for KeeperMap tables"); + DDLQueryOnClusterParams params; params.access_to_check = getRequiredAccess(); return executeDDLQueryOnCluster(query_ptr, getContext(), params); } getContext()->checkAccess(getRequiredAccess()); - auto table_id = getContext()->resolveStorageID(alter, Context::ResolveOrdinary); - query_ptr->as().setDatabase(table_id.database_name); DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (database->shouldReplicateQuery(getContext(), query_ptr)) @@ -91,7 +98,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) return database->tryEnqueueReplicatedDDL(query_ptr, getContext()); } - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); checkStorageSupportsTransactionsIfNeeded(table, getContext()); if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); diff --git a/tests/queries/0_stateless/02577_keepermap_delete_update.sql b/tests/queries/0_stateless/02577_keepermap_delete_update.sql index 199a653822c..942dd28cd46 100644 --- a/tests/queries/0_stateless/02577_keepermap_delete_update.sql +++ b/tests/queries/0_stateless/02577_keepermap_delete_update.sql @@ -31,7 +31,7 @@ ALTER TABLE 02661_keepermap_delete_update UPDATE value = 'Another' WHERE key > 2 SELECT * FROM 02661_keepermap_delete_update ORDER BY key; SELECT '-----------'; -ALTER TABLE 02661_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError 36 } +ALTER TABLE 02661_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError BAD_ARGUMENTS } SELECT * FROM 02661_keepermap_delete_update ORDER BY key; SELECT '-----------'; @@ -39,4 +39,6 @@ ALTER TABLE 02661_keepermap_delete_update UPDATE value2 = value2 * 10 + 2 WHERE SELECT * FROM 02661_keepermap_delete_update ORDER BY key; SELECT '-----------'; +ALTER TABLE 02661_keepermap_delete_update ON CLUSTER test_shard_localhost UPDATE value2 = value2 * 10 + 2 WHERE value2 < 100; -- { serverError BAD_ARGUMENTS } + DROP TABLE IF EXISTS 02661_keepermap_delete_update; From 12bee0573fe3dcde656942181a0d234994930c48 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 14:54:14 +0000 Subject: [PATCH 225/233] Correctly check table --- src/Interpreters/InterpreterAlterQuery.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 3dfa29fbb01..c683296a2ba 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -76,11 +76,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) auto table_id = getContext()->resolveStorageID(alter, Context::ResolveOrdinary); query_ptr->as().setDatabase(table_id.database_name); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (!alter.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { - if (table->as()) + if (table && table->as()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutations with ON CLUSTER are not allowed for KeeperMap tables"); DDLQueryOnClusterParams params; @@ -98,6 +98,9 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) return database->tryEnqueueReplicatedDDL(query_ptr, getContext()); } + if (!table) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not find table: {}", table_id.table_name); + checkStorageSupportsTransactionsIfNeeded(table, getContext()); if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); From e81c2999a3ffe0733be88ced07456fca67119c4b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 16:56:45 +0200 Subject: [PATCH 226/233] Update src/Interpreters/InterpreterAlterQuery.cpp Co-authored-by: Alexander Tokmakov --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c683296a2ba..49bc18534a8 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -99,7 +99,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) } if (!table) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not find table: {}", table_id.table_name); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Could not find table: {}", table_id.table_name); checkStorageSupportsTransactionsIfNeeded(table, getContext()); if (table->isStaticStorage()) From f21c664744897be6da191b9e62d4768abc098194 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Apr 2023 15:17:06 +0000 Subject: [PATCH 227/233] Add error code --- src/Interpreters/InterpreterAlterQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 49bc18534a8..21f0fbadd09 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -41,6 +41,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TABLE; } From fea5fae5b00294fa0b0729cc7ee109a572510180 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 3 Apr 2023 17:42:57 +0000 Subject: [PATCH 228/233] Fix 02494_query_cache_drop.sql Failing with high rate in master after #45912 was merged --- tests/queries/0_stateless/02494_query_cache_drop_cache.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql index 1f61472fcb0..078057a834f 100644 --- a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql +++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql @@ -3,6 +3,9 @@ SET allow_experimental_query_cache = true; +-- (it's silly to use what will be tested below but we have to assume other tests cluttered the query cache) +SYSTEM DROP QUERY CACHE; + -- Cache query result in query cache SELECT 1 SETTINGS use_query_cache = true; SELECT count(*) FROM system.query_cache; From 1cb8a7c45cd5894facc0abeb17e7e89c9a4b5821 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Apr 2023 00:46:25 +0300 Subject: [PATCH 229/233] Update tips.md See Telegram. --- docs/en/operations/tips.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 693dcd0d21b..8f6cf6ad147 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -74,7 +74,7 @@ Never set the block size too small or too large. You can use RAID-0 on SSD. Regardless of RAID use, always use replication for data security. -Enable NCQ with a long queue. For HDD, choose the CFQ scheduler, and for SSD, choose noop. Don’t reduce the ‘readahead’ setting. +Enable NCQ with a long queue. For HDD, choose the mq-deadline or CFQ scheduler, and for SSD, choose noop. Don’t reduce the ‘readahead’ setting. For HDD, enable the write cache. Make sure that [`fstrim`](https://en.wikipedia.org/wiki/Trim_(computing)) is enabled for NVME and SSD disks in your OS (usually it's implemented using a cronjob or systemd service). From 558b1eb37226b3307d866abc47f7b1abfc665cff Mon Sep 17 00:00:00 2001 From: rfraposa Date: Mon, 3 Apr 2023 19:35:23 -0600 Subject: [PATCH 230/233] Update clickhouse-local.md --- .../operations/utilities/clickhouse-local.md | 146 +++++++++++++++++- 1 file changed, 144 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 6bf1269c1d9..a23e0745dec 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -8,10 +8,150 @@ sidebar_label: clickhouse-local The `clickhouse-local` program enables you to perform fast processing on local files, without having to deploy and configure the ClickHouse server. It accepts data that represent tables and queries them using [ClickHouse SQL dialect](../../sql-reference/index.md). `clickhouse-local` uses the same core as ClickHouse server, so it supports most of the features and the same set of formats and table engines. -By default `clickhouse-local` has access to data on the same host, and it does not depend on the server's configuration. It also supports loading server configuration using `--config-file` argument. For temporary data, a unique temporary data directory is created by default. +## Download clickhouse-local + +`clickhouse-local` is executed using the same `clickhouse` binary that runs the ClickHouse server and `clickhouse-client`. The easiest way to download the latest version is with the following command: + +```bash +curl https://clickhouse.com/ | sh +``` + +:::note +The binary you just downloaded can run all sorts of ClickHouse tools and utilities. If you want to run ClickHouse as a database server, check out the [Quick Start](../../quick-start.mdx). +::: + +## Query data in a CSV file using SQL + +A common use of `clickhouse-local` is to run ad-hoc queries on files: where you don't have to insert the data into a table. `clickhouse-local` can stream the data from a file into a temporary table and execute your SQL. + +If the file is sitting on the same machine as `clickhouse-local`, use the `file` table engine. The following `reviews.tsv` file contains a sampling of Amazon product reviews: + +```bash +./clickhouse local -q "SELECT * FROM file('reviews.tsv')" +``` + +ClickHouse knows the file uses a tab-separated format from filename extension. If you need to explicitly specify the format, simply add one of the [many ClickHouse input formats](../../interfaces/formats.md): + ```bash + ./clickhouse local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')" + ``` + +The `file` table function creates a table, and you can use `DESCRIBE` to see the inferred schema: + +```bash +./clickhouse local -q "DESCRIBE file('reviews.tsv')" +``` + +```response +marketplace Nullable(String) +customer_id Nullable(Int64) +review_id Nullable(String) +product_id Nullable(String) +product_parent Nullable(Int64) +product_title Nullable(String) +product_category Nullable(String) +star_rating Nullable(Int64) +helpful_votes Nullable(Int64) +total_votes Nullable(Int64) +vine Nullable(String) +verified_purchase Nullable(String) +review_headline Nullable(String) +review_body Nullable(String) +review_date Nullable(Date) +``` + +Let's find a product with the highest rating: + +```bash +./clickhouse local -q "SELECT + argMax(product_title,star_rating), + max(star_rating) +FROM file('reviews.tsv')" +``` + +```response +Monopoly Junior Board Game 5 +``` + +## Query data in a Parquet file in AWS S3 + +If you have a file in S3, use `clickhouse-local` and the `s3` table function to query the file in place (without inserting the data into a ClickHouse table). We have a file named `house_0.parquet` in a public bucket that contains home prices of property sold in the United Kingdom. Let's see how many rows it has: + +```bash +./clickhouse local -q " +SELECT count() +FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" +``` + +The file has 2.7M rows: + +```response +2772030 +``` + +It's always useful to see what the inferred schema that ClickHouse determines from the file: + +```bash +./clickhouse local -q "DESCRIBE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" +``` + +```response +price Nullable(Int64) +date Nullable(UInt16) +postcode1 Nullable(String) +postcode2 Nullable(String) +type Nullable(String) +is_new Nullable(UInt8) +duration Nullable(String) +addr1 Nullable(String) +addr2 Nullable(String) +street Nullable(String) +locality Nullable(String) +town Nullable(String) +district Nullable(String) +county Nullable(String) +``` + +Let's see what the most expensive neighborhoods are: + +```bash +./clickhouse local -q " +SELECT + town, + district, + count() AS c, + round(avg(price)) AS price, + bar(price, 0, 5000000, 100) +FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet') +GROUP BY + town, + district +HAVING c >= 100 +ORDER BY price DESC +LIMIT 10" +``` + +```response +LONDON CITY OF LONDON 886 2271305 █████████████████████████████████████████████▍ +LEATHERHEAD ELMBRIDGE 206 1176680 ███████████████████████▌ +LONDON CITY OF WESTMINSTER 12577 1108221 ██████████████████████▏ +LONDON KENSINGTON AND CHELSEA 8728 1094496 █████████████████████▉ +HYTHE FOLKESTONE AND HYTHE 130 1023980 ████████████████████▍ +CHALFONT ST GILES CHILTERN 113 835754 ████████████████▋ +AMERSHAM BUCKINGHAMSHIRE 113 799596 ███████████████▉ +VIRGINIA WATER RUNNYMEDE 356 789301 ███████████████▊ +BARNET ENFIELD 282 740514 ██████████████▊ +NORTHWOOD THREE RIVERS 184 731609 ██████████████▋ +``` + +:::tip +When you are ready to insert your files into ClickHouse, startup a ClickHouse server and insert the results of your `file` and `s3` table functions into a `MergeTree` table. View the [Quick Start](../../quick-start.mdx) for more details. +::: + ## Usage {#usage} +By default `clickhouse-local` has access to data of a ClickHouse server on the same host, and it does not depend on the server's configuration. It also supports loading server configuration using `--config-file` argument. For temporary data, a unique temporary data directory is created by default. + Basic usage (Linux): ``` bash @@ -24,7 +164,9 @@ Basic usage (Mac): $ ./clickhouse local --structure "table_structure" --input-format "format_of_incoming_data" --query "query" ``` -Also supported on Windows through WSL2. +:::note +`clickhouse-local` is also supported on Windows through WSL2. +::: Arguments: From 3bd29f0aa978a4d20f26a7593b8da0d05614be19 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Apr 2023 10:34:33 +0800 Subject: [PATCH 231/233] fix exception message --- src/Functions/map.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 14453de0646..21a4ab8de7d 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -179,7 +179,7 @@ public: if (const auto * keys_type = checkAndGetDataType(arguments[0].get())) key_type = keys_type->getNestedType(); else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be Array or Map", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName()); DataTypePtr value_type; if (const auto * value_array_type = checkAndGetDataType(arguments[1].get())) From 57a412745d39074cd3e3740706da0e12760eab8c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 4 Apr 2023 11:35:29 +0000 Subject: [PATCH 232/233] Update version_date.tsv and changelogs after v22.8.16.32-lts --- docs/changelogs/v22.8.16.32-lts.md | 29 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 30 insertions(+) create mode 100644 docs/changelogs/v22.8.16.32-lts.md diff --git a/docs/changelogs/v22.8.16.32-lts.md b/docs/changelogs/v22.8.16.32-lts.md new file mode 100644 index 00000000000..27ec1f1f145 --- /dev/null +++ b/docs/changelogs/v22.8.16.32-lts.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.8.16.32-lts (7c4be737bd0) FIXME as compared to v22.8.15.23-lts (d36fa168bbf) + +#### Build/Testing/Packaging Improvement +* Backported in [#48344](https://github.com/ClickHouse/ClickHouse/issues/48344): Use sccache as a replacement for ccache and using S3 as cache backend. [#46240](https://github.com/ClickHouse/ClickHouse/pull/46240) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#48250](https://github.com/ClickHouse/ClickHouse/issues/48250): The `clickhouse/clickhouse-keeper` image used to be pushed only with tags `-alpine`, e.g. `latest-alpine`. As it was suggested in https://github.com/ClickHouse/examples/pull/2, now it will be pushed as suffixless too. [#48236](https://github.com/ClickHouse/ClickHouse/pull/48236) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix bug in zero-copy replication disk choice during fetch [#47010](https://github.com/ClickHouse/ClickHouse/pull/47010) ([alesapin](https://github.com/alesapin)). +* Fix query parameters [#47488](https://github.com/ClickHouse/ClickHouse/pull/47488) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix wait for zero copy lock during move [#47631](https://github.com/ClickHouse/ClickHouse/pull/47631) ([alesapin](https://github.com/alesapin)). +* Fix crash in polygonsSymDifferenceCartesian [#47702](https://github.com/ClickHouse/ClickHouse/pull/47702) ([pufit](https://github.com/pufit)). +* Backport to 22.8: Fix moving broken parts to the detached for the object storage disk on startup [#48273](https://github.com/ClickHouse/ClickHouse/pull/48273) ([Aleksei Filatov](https://github.com/aalexfvk)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Add a fuse for backport branches w/o a created PR [#47760](https://github.com/ClickHouse/ClickHouse/pull/47760) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Only valid Reviews.STATES overwrite existing reviews [#47789](https://github.com/ClickHouse/ClickHouse/pull/47789) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Place short return before big block, improve logging [#47822](https://github.com/ClickHouse/ClickHouse/pull/47822) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Artifacts s3 prefix [#47945](https://github.com/ClickHouse/ClickHouse/pull/47945) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tsan error lock-order-inversion [#47953](https://github.com/ClickHouse/ClickHouse/pull/47953) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 24af79eef2f..16ae3007938 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -36,6 +36,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.16.32-lts 2023-04-04 v22.8.15.23-lts 2023-03-10 v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 From 2d4fbdf4b0cd60063f6b0eceb25ad2369d0d3607 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 4 Apr 2023 14:27:58 +0200 Subject: [PATCH 233/233] add script for a slack bot --- utils/ci-slack-bot/ci-slack-bot.py | 179 +++++++++++++++++++++++++++++ 1 file changed, 179 insertions(+) create mode 100755 utils/ci-slack-bot/ci-slack-bot.py diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py new file mode 100755 index 00000000000..83b6d8aad6b --- /dev/null +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -0,0 +1,179 @@ +#!/usr/bin/env python3 + +# A trivial stateless slack bot that notifies about new broken tests in ClickHouse CI. +# It checks what happened to our CI during the last check_period hours (1 hour) and notifies us in slack if necessary. +# This script should be executed once each check_period hours (1 hour). +# It will post duplicate messages if you run it more often; it will lose some messages if you run it less often. +# +# You can run it locally with no arguments, it will work in a dry-run mode. Or you can set your own SLACK_URL_DEFAULT. +# Feel free to add more checks, more details to messages, or better heuristics. +# NOTE There's no deployment automation for now, +# an AWS Lambda (slack-ci-bot-test lambda in CI-CD) has to be updated manually after changing this script. +# +# See also: https://aretestsgreenyet.com/ + +import os +import json +import base64 + +if os.environ.get("AWS_LAMBDA_ENV", "0") == "1": + # For AWS labmda (python 3.7) + from botocore.vendored import requests +else: + # For running locally + import requests + +DRY_RUN_MARK = "" + +MAX_FAILURES_DEFAULT = 50 +SLACK_URL_DEFAULT = DRY_RUN_MARK + +# Find tests that failed in master during the last check_period hours, +# but did not fail during the last 2 weeks. Assuming these tests were broken recently. +# NOTE: It may report flaky tests that fail too rarely. +NEW_BROKEN_TESTS_QUERY = """ +WITH + 1 AS check_period, + now() as now +SELECT test_name, any(report_url) +FROM checks +WHERE 1 + AND check_start_time >= now - INTERVAL 1 WEEK + AND (check_start_time + check_duration_ms / 1000) >= now - INTERVAL check_period HOUR + AND pull_request_number = 0 + AND test_status LIKE 'F%' + AND check_status != 'success' + AND test_name NOT IN ( + SELECT test_name FROM checks WHERE 1 + AND check_start_time >= now - INTERVAL 1 MONTH + AND (check_start_time + check_duration_ms / 1000) BETWEEN now - INTERVAL 2 WEEK AND now - INTERVAL check_period HOUR + AND pull_request_number = 0 + AND check_status != 'success' + AND test_status LIKE 'F%') + AND test_context_raw NOT LIKE '%CannotSendRequest%' and test_context_raw NOT LIKE '%Server does not respond to health check%' +GROUP BY test_name +""" + +# Returns total number of failed checks during the last 24 hours +# and previous value of that metric (check_period hours ago) +COUNT_FAILURES_QUERY = """ +WITH + 1 AS check_period, + '%' AS check_name_pattern, + now() as now +SELECT + countIf((check_start_time + check_duration_ms / 1000) >= now - INTERVAL 24 HOUR) AS new_val, + countIf((check_start_time + check_duration_ms / 1000) <= now - INTERVAL check_period HOUR) AS prev_val +FROM checks +WHERE 1 + AND check_start_time >= now - INTERVAL 1 WEEK + AND (check_start_time + check_duration_ms / 1000) >= now - INTERVAL 24 + check_period HOUR + AND pull_request_number = 0 + AND test_status LIKE 'F%' + AND check_status != 'success' + AND check_name ILIKE check_name_pattern +""" + +SLACK_MESSAGE_JSON = {"type": "mrkdwn", "text": None} + + +def get_play_url(query): + return ( + "https://play.clickhouse.com/play?user=play#" + + base64.b64encode(query.encode()).decode() + ) + + +def run_clickhouse_query(query): + url = "https://play.clickhouse.com/?user=play&query=" + requests.utils.quote(query) + res = requests.get(url) + if res.status_code != 200: + print("Failed to execute query: ", res.status_code, res.content) + raise Exception( + "Failed to execute query: {}: {}".format(res.status_code, res.content) + ) + + lines = res.text.strip().splitlines() + return [x.split("\t") for x in lines] + + +def get_new_broken_tests_message(broken_tests): + if not broken_tests: + return None + msg = "There are {} new broken tests in master:\n".format(len(broken_tests)) + for name, report in broken_tests: + msg += " - *{}* - <{}|Report>\n".format(name, report) + return msg + + +def get_too_many_failures_message(failures_count): + MAX_FAILURES = int(os.environ.get("MAX_FAILURES", MAX_FAILURES_DEFAULT)) + curr_failures = int(failures_count[0][0]) + prev_failures = int(failures_count[0][1]) + if curr_failures == 0: + return ( + "Looks like CI is completely broken: there are *no failures* at all... 0_o" + ) + if curr_failures < MAX_FAILURES: + return None + if prev_failures < MAX_FAILURES: + return "*CI is broken: there are {} failures during the last 24 hours*".format( + curr_failures + ) + if curr_failures < prev_failures: + return None + if (curr_failures - prev_failures) / prev_failures < 0.2: + return None + return "CI is broken and it's getting worse: there are {} failures during the last 24 hours".format( + curr_failures + ) + + +def send_to_slack(message): + SLACK_URL = os.environ.get("SLACK_URL", SLACK_URL_DEFAULT) + if SLACK_URL == DRY_RUN_MARK: + return + + payload = SLACK_MESSAGE_JSON.copy() + payload["text"] = message + res = requests.post(SLACK_URL, json.dumps(payload)) + if res.status_code != 200: + print("Failed to send a message to Slack: ", res.status_code, res.content) + raise Exception( + "Failed to send a message to Slack: {}: {}".format( + res.status_code, res.content + ) + ) + + +def query_and_alert_if_needed(query, get_message_func): + query_res = run_clickhouse_query(query) + print("Got result {} for query {}", query_res, query) + msg = get_message_func(query_res) + if msg is None: + return + + msg += "\nCI DB query: <{}|link>".format(get_play_url(query)) + print("Sending message to slack:", msg) + send_to_slack(msg) + + +def check_and_alert(): + query_and_alert_if_needed(NEW_BROKEN_TESTS_QUERY, get_new_broken_tests_message) + query_and_alert_if_needed(COUNT_FAILURES_QUERY, get_too_many_failures_message) + + +def lambda_handler(event, context): + try: + check_and_alert() + return {"statusCode": 200, "body": "OK"} + except Exception as e: + send_to_slack( + "I failed, please help me (see ClickHouse/utils/ci-slack-bot/ci-slack-bot.py): " + + str(e) + ) + return {"statusCode": 200, "body": "FAIL"} + + +if __name__ == "__main__": + check_and_alert()