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 0001/1548] 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 0002/1548] 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 0003/1548] 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 0004/1548] 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 0005/1548] 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 0006/1548] 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 0007/1548] 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 0008/1548] 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 0009/1548] 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 0010/1548] 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 0011/1548] 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 0012/1548] 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 0013/1548] 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 0014/1548] 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 b4b1158a600a729890dc524034a6d0d294773c94 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 16 Jan 2023 12:20:54 +0100 Subject: [PATCH 0015/1548] Change 02129_add_column_add_ttl to check differences between using materialize_ttl_recalculate_only=1/0 --- .../02129_add_column_add_ttl.reference | 39 ++++++++++--------- .../0_stateless/02129_add_column_add_ttl.sql | 16 +++++++- 2 files changed, 34 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/02129_add_column_add_ttl.reference b/tests/queries/0_stateless/02129_add_column_add_ttl.reference index 8b3280ef095..5bffe7b5903 100644 --- a/tests/queries/0_stateless/02129_add_column_add_ttl.reference +++ b/tests/queries/0_stateless/02129_add_column_add_ttl.reference @@ -1,41 +1,42 @@ 0 2021-01-01 0 -0 2021-01-01 0 1 2021-01-01 0 -1 2021-01-01 0 -2 2021-01-01 0 2 2021-01-01 0 3 2021-01-01 0 -3 2021-01-01 0 -4 2021-01-01 0 4 2021-01-01 0 5 2021-01-01 0 -5 2021-01-01 0 -6 2021-01-01 0 6 2021-01-01 0 7 2021-01-01 0 -7 2021-01-01 0 8 2021-01-01 0 -8 2021-01-01 0 -9 2021-01-01 0 9 2021-01-01 0 ========== 0 2021-01-01 0 +0 2021-01-01 1 +1 2021-01-01 0 +1 2021-01-01 1 +2 2021-01-01 0 +2 2021-01-01 1 +3 2021-01-01 0 +3 2021-01-01 1 +4 2021-01-01 0 +4 2021-01-01 1 +5 2021-01-01 0 +5 2021-01-01 1 +6 2021-01-01 0 +6 2021-01-01 1 +7 2021-01-01 0 +7 2021-01-01 1 +8 2021-01-01 0 +8 2021-01-01 1 +9 2021-01-01 0 +9 2021-01-01 1 +========== 0 2021-01-01 0 1 2021-01-01 0 -1 2021-01-01 0 -2 2021-01-01 0 2 2021-01-01 0 3 2021-01-01 0 -3 2021-01-01 0 -4 2021-01-01 0 4 2021-01-01 0 5 2021-01-01 0 -5 2021-01-01 0 -6 2021-01-01 0 6 2021-01-01 0 7 2021-01-01 0 -7 2021-01-01 0 -8 2021-01-01 0 8 2021-01-01 0 9 2021-01-01 0 -9 2021-01-01 0 diff --git a/tests/queries/0_stateless/02129_add_column_add_ttl.sql b/tests/queries/0_stateless/02129_add_column_add_ttl.sql index 7a6dd928a3f..a68868ed3bc 100644 --- a/tests/queries/0_stateless/02129_add_column_add_ttl.sql +++ b/tests/queries/0_stateless/02129_add_column_add_ttl.sql @@ -4,10 +4,12 @@ create table ttl_test_02129(a Int64, b String, d Date) Engine=MergeTree partition by d order by a settings min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, materialize_ttl_recalculate_only = 0; +system stop ttl merges ttl_test_02129; + insert into ttl_test_02129 select number, '', '2021-01-01' from numbers(10); alter table ttl_test_02129 add column c Int64 settings mutations_sync=2; -insert into ttl_test_02129 select number, '', '2021-01-01', 0 from numbers(10); +insert into ttl_test_02129 select number, '', '2021-01-01', 1 from numbers(10); alter table ttl_test_02129 modify TTL (d + INTERVAL 1 MONTH) DELETE WHERE c=1 settings mutations_sync=2; select * from ttl_test_02129 order by a, b, d, c; @@ -21,11 +23,21 @@ create table ttl_test_02129(a Int64, b String, d Date) Engine=MergeTree partition by d order by a settings min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, materialize_ttl_recalculate_only = 1; +system stop ttl merges ttl_test_02129; + insert into ttl_test_02129 select number, '', '2021-01-01' from numbers(10); alter table ttl_test_02129 add column c Int64 settings mutations_sync=2; -insert into ttl_test_02129 select number, '', '2021-01-01', 0 from numbers(10); +insert into ttl_test_02129 select number, '', '2021-01-01', 1 from numbers(10); alter table ttl_test_02129 modify TTL (d + INTERVAL 1 MONTH) DELETE WHERE c=1 settings mutations_sync=2; +select * from ttl_test_02129 order by a, b, d, c; + +select '=========='; + +system start ttl merges ttl_test_02129; + +optimize table ttl_test_02129 final; + select * from ttl_test_02129 order by a, b, d, c; drop table ttl_test_02129; From 9c6c6d98444d160608690634cf794195fc37eaae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Jan 2023 20:12:34 +0000 Subject: [PATCH 0016/1548] Aggregate Projections analysis using query plan [In progress] --- src/Dictionaries/HashedDictionary.cpp | 24 +- src/Dictionaries/HashedDictionary.h | 4 +- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../QueryPlan/Optimizations/matchTrees.cpp | 215 +++++++++++++ .../QueryPlan/Optimizations/matchTrees.h | 43 +++ .../Optimizations/optimizeReadInOrder.cpp | 241 +------------- .../Optimizations/optimizeUseProjections.cpp | 297 ++++++++++++++++++ 7 files changed, 571 insertions(+), 254 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/matchTrees.cpp create mode 100644 src/Processors/QueryPlan/Optimizations/matchTrees.h create mode 100644 src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 99c3dd05f45..331cd6cfde9 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -189,8 +189,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co extractor.rollbackCurrentKey(); } - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys_size); + found_count.fetch_add(keys_found); return result; } @@ -214,8 +214,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co } }); - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys_size); + found_count.fetch_add(keys_found); return result; } @@ -271,8 +271,8 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, is_key_valid_func, get_parent_func); - query_count.fetch_add(keys.size(), std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys.size()); + found_count.fetch_add(keys_found); return dictionary_hierarchy_array; } @@ -342,8 +342,8 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( auto result = getKeysIsInHierarchyColumn(keys, keys_in, is_key_valid_func, get_parent_key_func); - query_count.fetch_add(keys.size(), std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys.size()); + found_count.fetch_add(keys_found); return result; } @@ -392,8 +392,8 @@ ColumnPtr HashedDictionary::getDescendants( size_t keys_found; auto result = getKeysDescendantsArray(keys, *parent_to_child_index, level, keys_found); - query_count.fetch_add(keys.size(), std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys.size()); + found_count.fetch_add(keys_found); return result; } @@ -635,8 +635,8 @@ void HashedDictionary::getItemsImpl( keys_extractor.rollbackCurrentKey(); } - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys_size); + found_count.fetch_add(keys_found); } template diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 1fb38e8f9a6..26f4a4c7a88 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -62,10 +62,10 @@ public: double getFoundRate() const override { - size_t queries = query_count.load(std::memory_order_relaxed); + size_t queries = query_count.load(); if (!queries) return 0; - return static_cast(found_count.load(std::memory_order_relaxed)) / queries; + return static_cast(found_count.load()) / queries; } double getHitRate() const override { return 1.0; } diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 7f435463d64..6f733c75e88 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -93,6 +93,7 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); +void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp new file mode 100644 index 00000000000..4e11f65f1e2 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp @@ -0,0 +1,215 @@ +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) +{ + using Parents = std::set; + std::unordered_map inner_parents; + std::unordered_map inner_inputs; + + { + std::stack stack; + for (const auto * out : inner_dag.getOutputs()) + { + if (inner_parents.contains(out)) + continue; + + stack.push(out); + inner_parents.emplace(out, Parents()); + while (!stack.empty()) + { + const auto * node = stack.top(); + stack.pop(); + + if (node->type == ActionsDAG::ActionType::INPUT) + inner_inputs.emplace(node->result_name, node); + + for (const auto * child : node->children) + { + auto [it, inserted] = inner_parents.emplace(child, Parents()); + it->second.emplace(node); + + if (inserted) + stack.push(child); + } + } + } + } + + struct Frame + { + const ActionsDAG::Node * node; + ActionsDAG::NodeRawConstPtrs mapped_children; + }; + + MatchedTrees::Matches matches; + std::stack stack; + + for (const auto & node : outer_dag.getNodes()) + { + if (matches.contains(&node)) + continue; + + stack.push(Frame{&node, {}}); + while (!stack.empty()) + { + auto & frame = stack.top(); + frame.mapped_children.reserve(frame.node->children.size()); + + while (frame.mapped_children.size() < frame.node->children.size()) + { + const auto * child = frame.node->children[frame.mapped_children.size()]; + auto it = matches.find(child); + if (it == matches.end()) + { + /// If match map does not contain a child, it was not visited. + stack.push(Frame{child, {}}); + break; + } + /// A node from found match may be nullptr. + /// It means that node is visited, but no match was found. + frame.mapped_children.push_back(it->second.node); + } + + if (frame.mapped_children.size() < frame.node->children.size()) + continue; + + /// Create an empty match for current node. + /// natch.node will be set if match is found. + auto & match = matches[frame.node]; + + if (frame.node->type == ActionsDAG::ActionType::INPUT) + { + const ActionsDAG::Node * mapped = nullptr; + if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) + mapped = it->second; + + match.node = mapped; + } + else if (frame.node->type == ActionsDAG::ActionType::ALIAS) + { + match = matches[frame.node->children.at(0)]; + } + else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) + { + + //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; + + bool found_all_children = true; + for (const auto * child : frame.mapped_children) + if (!child) + found_all_children = false; + + if (found_all_children && !frame.mapped_children.empty()) + { + Parents container; + Parents * intersection = &inner_parents[frame.mapped_children[0]]; + + if (frame.mapped_children.size() > 1) + { + std::vector other_parents; + size_t mapped_children_size = frame.mapped_children.size(); + other_parents.reserve(mapped_children_size); + for (size_t i = 1; i < mapped_children_size; ++i) + other_parents.push_back(&inner_parents[frame.mapped_children[i]]); + + for (const auto * parent : *intersection) + { + bool is_common = true; + for (const auto * set : other_parents) + { + if (!set->contains(parent)) + { + is_common = false; + break; + } + } + + if (is_common) + container.insert(parent); + } + + intersection = &container; + } + + //std::cerr << ".. Candidate parents " << intersection->size() << std::endl; + + if (!intersection->empty()) + { + auto func_name = frame.node->function_base->getName(); + for (const auto * parent : *intersection) + { + //std::cerr << ".. candidate " << parent->result_name << std::endl; + if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) + { + const auto & children = parent->children; + size_t num_children = children.size(); + if (frame.mapped_children.size() == num_children) + { + bool all_children_matched = true; + for (size_t i = 0; all_children_matched && i < num_children; ++i) + all_children_matched = frame.mapped_children[i] == children[i]; + + if (all_children_matched) + { + match.node = parent; + break; + } + } + } + } + } + } + + if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) + { + size_t num_const_args = 0; + const ActionsDAG::Node * monotonic_child = nullptr; + for (const auto * child : frame.node->children) + { + if (child->column) + ++num_const_args; + else + monotonic_child = child; + } + + if (monotonic_child && num_const_args + 1 == frame.node->children.size()) + { + const auto & child_match = matches[monotonic_child]; + if (child_match.node) + { + auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); + if (info.is_monotonic) + { + MatchedTrees::Monotonicity monotonicity; + monotonicity.direction *= info.is_positive ? 1 : -1; + monotonicity.strict = info.is_strict; + + if (child_match.monotonicity) + { + monotonicity.direction *= child_match.monotonicity->direction; + if (!child_match.monotonicity->strict) + monotonicity.strict = false; + } + + match.node = child_match.node; + match.monotonicity = monotonicity; + } + } + } + } + } + + stack.pop(); + } + } + + return matches; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.h b/src/Processors/QueryPlan/Optimizations/matchTrees.h new file mode 100644 index 00000000000..18f0fe8b292 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/matchTrees.h @@ -0,0 +1,43 @@ +#include + +namespace DB::QueryPlanOptimizations +{ + + +/// This structure stores a node mapping from one DAG to another. +/// The rule is following: +/// * Input nodes are mapped by name. +/// * Function is mapped to function if all children are mapped and function names are same. +/// * Alias is mapped to it's children mapping. +/// * Monotonic function can be mapped to it's children mapping if direct mapping does not exist. +/// In this case, information about monotonicity is filled. +/// * Mapped node is nullptr if there is no mapping found. +/// +/// Overall, directly mapped nodes represent equal calculations. +/// Notes: +/// * Mapped DAG can contain many nodes which represent the same calculation. +/// In this case mapping is ambiguous and only one node is mapped. +/// * Aliases for mapped DAG are not supported. +/// DAG for PK does not contain aliases and ambiguous nodes. +struct MatchedTrees +{ + /// Monotonicity is calculated for monotonic functions chain. + /// Chain is not strict if there is any non-strict monotonic function. + struct Monotonicity + { + int direction = 1; + bool strict = true; + }; + + struct Match + { + const ActionsDAG::Node * node = nullptr; + std::optional monotonicity; + }; + + using Matches = std::unordered_map; +}; + +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag); + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index bdf8f24f9d6..1876ebc4014 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -289,246 +290,6 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) } } -/// This structure stores a node mapping from one DAG to another. -/// The rule is following: -/// * Input nodes are mapped by name. -/// * Function is mapped to function if all children are mapped and function names are same. -/// * Alias is mapped to it's children mapping. -/// * Monotonic function can be mapped to it's children mapping if direct mapping does not exist. -/// In this case, information about monotonicity is filled. -/// * Mapped node is nullptr if there is no mapping found. -/// -/// Overall, directly mapped nodes represent equal calculations. -/// Notes: -/// * Mapped DAG can contain many nodes which represent the same calculation. -/// In this case mapping is ambiguous and only one node is mapped. -/// * Aliases for mapped DAG are not supported. -/// DAG for PK does not contain aliases and ambiguous nodes. -struct MatchedTrees -{ - /// Monotonicity is calculated for monotonic functions chain. - /// Chain is not strict if there is any non-strict monotonic function. - struct Monotonicity - { - int direction = 1; - bool strict = true; - }; - - struct Match - { - const ActionsDAG::Node * node = nullptr; - std::optional monotonicity; - }; - - using Matches = std::unordered_map; -}; - -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) -{ - using Parents = std::set; - std::unordered_map inner_parents; - std::unordered_map inner_inputs; - - { - std::stack stack; - for (const auto * out : inner_dag.getOutputs()) - { - if (inner_parents.contains(out)) - continue; - - stack.push(out); - inner_parents.emplace(out, Parents()); - while (!stack.empty()) - { - const auto * node = stack.top(); - stack.pop(); - - if (node->type == ActionsDAG::ActionType::INPUT) - inner_inputs.emplace(node->result_name, node); - - for (const auto * child : node->children) - { - auto [it, inserted] = inner_parents.emplace(child, Parents()); - it->second.emplace(node); - - if (inserted) - stack.push(child); - } - } - } - } - - struct Frame - { - const ActionsDAG::Node * node; - ActionsDAG::NodeRawConstPtrs mapped_children; - }; - - MatchedTrees::Matches matches; - std::stack stack; - - for (const auto & node : outer_dag.getNodes()) - { - if (matches.contains(&node)) - continue; - - stack.push(Frame{&node, {}}); - while (!stack.empty()) - { - auto & frame = stack.top(); - frame.mapped_children.reserve(frame.node->children.size()); - - while (frame.mapped_children.size() < frame.node->children.size()) - { - const auto * child = frame.node->children[frame.mapped_children.size()]; - auto it = matches.find(child); - if (it == matches.end()) - { - /// If match map does not contain a child, it was not visited. - stack.push(Frame{child, {}}); - break; - } - /// A node from found match may be nullptr. - /// It means that node is visited, but no match was found. - frame.mapped_children.push_back(it->second.node); - } - - if (frame.mapped_children.size() < frame.node->children.size()) - continue; - - /// Create an empty match for current node. - /// natch.node will be set if match is found. - auto & match = matches[frame.node]; - - if (frame.node->type == ActionsDAG::ActionType::INPUT) - { - const ActionsDAG::Node * mapped = nullptr; - if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) - mapped = it->second; - - match.node = mapped; - } - else if (frame.node->type == ActionsDAG::ActionType::ALIAS) - { - match = matches[frame.node->children.at(0)]; - } - else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) - { - - //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; - - bool found_all_children = true; - for (const auto * child : frame.mapped_children) - if (!child) - found_all_children = false; - - if (found_all_children && !frame.mapped_children.empty()) - { - Parents container; - Parents * intersection = &inner_parents[frame.mapped_children[0]]; - - if (frame.mapped_children.size() > 1) - { - std::vector other_parents; - size_t mapped_children_size = frame.mapped_children.size(); - other_parents.reserve(mapped_children_size); - for (size_t i = 1; i < mapped_children_size; ++i) - other_parents.push_back(&inner_parents[frame.mapped_children[i]]); - - for (const auto * parent : *intersection) - { - bool is_common = true; - for (const auto * set : other_parents) - { - if (!set->contains(parent)) - { - is_common = false; - break; - } - } - - if (is_common) - container.insert(parent); - } - - intersection = &container; - } - - //std::cerr << ".. Candidate parents " << intersection->size() << std::endl; - - if (!intersection->empty()) - { - auto func_name = frame.node->function_base->getName(); - for (const auto * parent : *intersection) - { - //std::cerr << ".. candidate " << parent->result_name << std::endl; - if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) - { - const auto & children = parent->children; - size_t num_children = children.size(); - if (frame.mapped_children.size() == num_children) - { - bool all_children_matched = true; - for (size_t i = 0; all_children_matched && i < num_children; ++i) - all_children_matched = frame.mapped_children[i] == children[i]; - - if (all_children_matched) - { - match.node = parent; - break; - } - } - } - } - } - } - - if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) - { - size_t num_const_args = 0; - const ActionsDAG::Node * monotonic_child = nullptr; - for (const auto * child : frame.node->children) - { - if (child->column) - ++num_const_args; - else - monotonic_child = child; - } - - if (monotonic_child && num_const_args + 1 == frame.node->children.size()) - { - const auto & child_match = matches[monotonic_child]; - if (child_match.node) - { - auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); - if (info.is_monotonic) - { - MatchedTrees::Monotonicity monotonicity; - monotonicity.direction *= info.is_positive ? 1 : -1; - monotonicity.strict = info.is_strict; - - if (child_match.monotonicity) - { - monotonicity.direction *= child_match.monotonicity->direction; - if (!child_match.monotonicity->strict) - monotonicity.strict = false; - } - - match.node = child_match.node; - match.monotonicity = monotonicity; - } - } - } - } - } - - stack.pop(); - } - } - - return matches; -} - InputOrderInfoPtr buildInputOrderInfo( const FixedColumns & fixed_columns, const ActionsDAGPtr & dag, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp new file mode 100644 index 00000000000..f1e97558dbb --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -0,0 +1,297 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get();\ + if (auto * reading = typeid_cast(step)) + { + /// Already read-in-order, skip. + if (reading->getQueryInfo().input_order_info) + return nullptr; + + const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); + if (sorting_key.column_names.empty()) + return nullptr; + + return &node; + } + + if (node.children.size() != 1) + return nullptr; + + if (typeid_cast(step) || typeid_cast(step)) + return findReadingStep(*node.children.front()); + + return nullptr; +} + +void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +{ + if (dag) + dag->mergeInplace(std::move(*expression->clone())); + else + dag = expression->clone(); +} + + +/// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. +bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + { + if (const auto * prewhere_info = reading->getPrewhereInfo()) + { + if (prewhere_info->row_level_filter) + { + appendExpression(dag, prewhere_info->row_level_filter); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + + if (prewhere_info->prewhere_actions) + { + appendExpression(dag, prewhere_info->prewhere_actions); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + } + return true; + } + + if (node.children.size() != 1) + return false; + + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) + return false; + + if (auto * expression = typeid_cast(step)) + { + const auto & actions = expression->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(dag, actions); + } + + if (auto * filter = typeid_cast(step)) + { + const auto & actions = filter->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(dag, actions); + if (const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName())) + filter_nodes.push_back(filter_expression); + else + return false; + } + + return false; +} + +struct AggregateProjectionInfo +{ + ActionsDAGPtr before_aggregation; + NamesAndTypesList keys; + AggregateDescriptions aggregates; + + /// A context copy from interpreter which was used for analysis. + /// Just in case it is used by some function. + ContextPtr context; +}; + +AggregateProjectionInfo getAggregatingProjectionInfo( + const ProjectionDescription & projection, + const ContextPtr & context, + StoragePtr & storage, + const StorageMetadataPtr & metadata_snapshot) +{ + /// 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. + InterpreterSelectQuery interpreter( + projection.query_ast, + context, + storage, + metadata_snapshot, + SelectQueryOptions{QueryProcessingStage::WithMergeableState}); + + const auto & analysis_result = interpreter.getAnalysisResult(); + const auto & query_analyzer = interpreter.getQueryAnalyzer(); + + AggregateProjectionInfo info; + info.context = interpreter.getContext(); + info.before_aggregation = analysis_result.before_aggregation; + info.keys = query_analyzer->aggregationKeys(); + info.aggregates = query_analyzer->aggregates(); + + return info; +} + +struct AggregateProjectionCandidate +{ + AggregateProjectionInfo info; + ProjectionDescription * projection; +}; + +std::optional analyzeAggregateProjection( + ProjectionDescription & projection, + AggregateProjectionInfo info, + ActionsDAG & query_dag, + const Names & keys, + const AggregateDescriptions & aggregates) +{ + + ActionsDAG::NodeRawConstPtrs key_nodes; + std::unordered_set aggregate_args; + + std::unordered_map index; + for (const auto * output : query_dag.getOutputs()) + index.emplace(output->result_name, output); + + key_nodes.reserve(keys.size()); + for (const auto & key : keys) + { + auto it = index.find(key); + /// This should not happen ideally. + if (it == index.end()) + return {}; + + key_nodes.push_back(it->second); + } + + for (const auto & aggregate : aggregates) + { + for (const auto & argument : aggregate.argument_names) + { + auto it = index.find(argument); + /// This should not happen ideally. + if (it == index.end()) + return {}; + + aggregate_args.insert(it->second); + } + } + + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); + + std::unordered_map> projection_aggregate_functions; + for (size_t i = 0; i < info.aggregates.size(); ++i) + projection_aggregate_functions[info.aggregates[i].function->getName()].push_back(i); + + struct AggFuncMatch + { + /// idx in projection + size_t idx; + /// nodes in query DAG + ActionsDAG::NodeRawConstPtrs args; + }; + + std::vector aggregate_function_matches; + aggregate_function_matches.reserve(aggregates.size()); + + for (const auto & aggregate : aggregates) + { + auto it = projection_aggregate_functions.find(aggregate.function->getName()); + if (it == projection_aggregate_functions.end()) + return {}; + auto & candidates = it->second; + + std::optional match; + + for (size_t idx : candidates) + { + const auto & candidate = info.aggregates[idx]; + + /// Note: this check is a bit strict. + /// We check that aggregate function names, arguemnt types and parameters are equal. + /// In some cases it's possilbe only to check that states are equal, + /// e.g. for quantile(0.3)(...) and quantile(0.5)(...). + /// But also functions sum(...) and sumIf(...) will have equal states, + /// and we can't replace one to another from projection. + if (!candidate.function->getStateType()->equals(*aggregate.function->getStateType())) + continue; + + ActionsDAG::NodeRawConstPtrs args; + args.reserve(aggregate.argument_names.size()); + for (const auto & name : aggregate.argument_names) + { + auto jt = index.find(name); + /// This should not happen ideally. + if (jt == index.end()) + break; + + const auto * outer_node = jt->second; + auto kt = matches.find(outer_node); + if (kt == matches.end()) + break; + + const auto & node_match = kt->second; + if (!node_match.node || node_match.monotonicity) + break; + + args.push_back(node_match.node); + } + + if (args.size() < aggregate.argument_names.size()) + continue; + + match = AggFuncMatch{idx, std::move(args)}; + } + + if (!match) + return {}; + + aggregate_function_matches.emplace_back(std::move(*match)); + } + + +} + +void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) +{ + if (node.children.size() != 1) + return; + + auto * aggregating = typeid_cast(node.step.get()); + if (!aggregating) + return; + + QueryPlan::Node * reading_node = findReadingStep(node); + if (!reading_node) + return; + + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildAggregatingDAG(node, dag, filter_nodes)) + return; + + const auto & keys = aggregating->getParams().keys; + const auto & aggregates = aggregating->getParams().aggregates; + + auto * reading = typeid_cast(reading_node->step.get()); + if (!reading) + return; + + // const auto metadata = reading->getStorageMetadata(); + // const auto & projections = metadata->projections; + + +} + +} From 0e31bd56d88977a91accb1e7f85501bacc4727f3 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 18 Jan 2023 06:47:47 +0000 Subject: [PATCH 0017/1548] Call constructor of TemporaryTableHolder with creator --- src/Interpreters/InterpreterCreateQuery.cpp | 41 +++++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ed4fd5699da..cdb3ce58e4c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -912,9 +912,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.temporary) { - if (create.storage && create.storage->engine && create.storage->engine->name != "Memory") - throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables can only be created with ENGINE = Memory, not {}", - create.storage->engine->name); + // if (create.storage && create.storage->engine && create.storage->engine->name != "Memory") + // throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables can only be created with ENGINE = Memory, not {}", + // create.storage->engine->name); /// It's possible if some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not. /// It makes sense when default_table_engine setting is used, but not for temporary tables. @@ -924,7 +924,8 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table: must be either ENGINE = Memory or empty"); auto engine_ast = std::make_shared(); - engine_ast->name = "Memory"; + // engine_ast->name = "Memory"; + engine_ast->name = create.storage->engine->name; engine_ast->no_empty_args = true; auto storage_ast = std::make_shared(); storage_ast->set(storage_ast->engine, engine_ast); @@ -1240,6 +1241,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties, DDLGuardPtr & ddl_guard) { + /* if (create.temporary) { if (create.if_not_exists && getContext()->tryResolveStorageID({"", create.getTable()}, Context::ResolveExternal)) @@ -1250,6 +1252,21 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, getContext()->getSessionContext()->addExternalTable(temporary_table_name, std::move(temporary_table)); return true; } + */ + + if (create.temporary) + { + // to-do: set correct name + /* + UUID id; + if (create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); + id = create.uuid; + create.setTable("_tmp_" + toString(id)); + */ + create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); + + } if (!ddl_guard) ddl_guard = DatabaseCatalog::instance().getDDLGuard(create.getDatabase(), create.getTable()); @@ -1382,6 +1399,22 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, "ATTACH ... FROM ... query is not supported for {} table engine, " "because such tables do not store any data on disk. Use CREATE instead.", res->getName()); + if (create.temporary) + { + // if (create.if_not_exists && getContext()->tryResolveStorageID({"", create.getTable()}, Context::ResolveExternal)) + // return false; + + String temporary_table_name = create.getTable(); + auto creator = [&](const StorageID &) + { + return std::move(res); + }; + auto temporary_table = TemporaryTableHolder(getContext(), creator, query_ptr); + // auto temporary_table = TemporaryTableHolder(getContext(), properties.columns, properties.constraints, query_ptr); + getContext()->getSessionContext()->addExternalTable(temporary_table_name, std::move(temporary_table)); + return true; + } + database->createTable(getContext(), create.getTable(), res, query_ptr); /// Move table data to the proper place. Wo do not move data earlier to avoid situations From 9fa4502b7836db99281515aa518c7a49835eaa19 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 19 Jan 2023 07:12:29 +0000 Subject: [PATCH 0018/1548] Add new constructor for TemporaryTableHolder --- src/Interpreters/DatabaseCatalog.cpp | 5 +++++ src/Interpreters/DatabaseCatalog.h | 2 ++ src/Interpreters/InterpreterCreateQuery.cpp | 4 +--- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6ac01a9473f..2daca6a55a8 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -53,6 +53,11 @@ namespace ErrorCodes extern const int HAVE_DEPENDENT_OBJECTS; } +TemporaryTableHolder::TemporaryTableHolder(ContextPtr context, const String& table_name, const ASTPtr & query = {}) +{ + +} + TemporaryTableHolder::TemporaryTableHolder(ContextPtr context_, const TemporaryTableHolder::Creator & creator, const ASTPtr & query) : WithContext(context_->getGlobalContext()) , temporary_tables(DatabaseCatalog::instance().getDatabaseForTemporaryTables().get()) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index a3fa4515a69..3028c900599 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -88,6 +88,8 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext { using Creator = std::function; + TemporaryTableHolder(ContextPtr context, const String & table_name, const ASTPtr & query = {}); + TemporaryTableHolder(ContextPtr context, const Creator & creator, const ASTPtr & query = {}); /// Creates temporary table with Engine=Memory diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cdb3ce58e4c..167134bd9ba 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1256,7 +1256,6 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.temporary) { - // to-do: set correct name /* UUID id; if (create.uuid == UUIDHelpers::Nil) @@ -1264,8 +1263,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, id = create.uuid; create.setTable("_tmp_" + toString(id)); */ - create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); - + // create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); } if (!ddl_guard) From 134ac9b2dda076e63468ba84a406afd309025023 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Jan 2023 19:53:22 +0000 Subject: [PATCH 0019/1548] Aggregate Projections analysis using query plan [In progress] --- .../Optimizations/optimizeUseProjections.cpp | 118 ++++++++++++++++-- 1 file changed, 106 insertions(+), 12 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index f1e97558dbb..18dd5c8adcb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB::QueryPlanOptimizations { @@ -150,7 +151,7 @@ struct AggregateProjectionCandidate }; std::optional analyzeAggregateProjection( - ProjectionDescription & projection, + //ProjectionDescription & projection, AggregateProjectionInfo info, ActionsDAG & query_dag, const Names & keys, @@ -164,6 +165,10 @@ std::optional analyzeAggregateProjection( for (const auto * output : query_dag.getOutputs()) index.emplace(output->result_name, output); + std::unordered_map proj_index; + for (const auto * output : info.before_aggregation->getOutputs()) + proj_index.emplace(output->result_name, output); + key_nodes.reserve(keys.size()); for (const auto & key : keys) { @@ -194,6 +199,8 @@ std::optional analyzeAggregateProjection( for (size_t i = 0; i < info.aggregates.size(); ++i) projection_aggregate_functions[info.aggregates[i].function->getName()].push_back(i); + std::unordered_set split_nodes; + struct AggFuncMatch { /// idx in projection @@ -228,29 +235,44 @@ std::optional analyzeAggregateProjection( continue; ActionsDAG::NodeRawConstPtrs args; - args.reserve(aggregate.argument_names.size()); - for (const auto & name : aggregate.argument_names) + size_t num_args = aggregate.argument_names.size(); + args.reserve(num_args); + for (size_t arg = 0; arg < num_args; ++arg) { - auto jt = index.find(name); + const auto & query_name = aggregate.argument_names[arg]; + const auto & proj_name = candidate.argument_names[arg]; + + auto jt = index.find(query_name); /// This should not happen ideally. if (jt == index.end()) break; - const auto * outer_node = jt->second; - auto kt = matches.find(outer_node); - if (kt == matches.end()) + const auto * query_node = jt->second; + + auto kt = proj_index.find(proj_name); + /// This should not happen ideally. + if (kt == proj_index.end()) break; - const auto & node_match = kt->second; - if (!node_match.node || node_match.monotonicity) + const auto * proj_node = kt->second; + + auto mt = matches.find(query_node); + if (mt == matches.end()) break; - args.push_back(node_match.node); + const auto & node_match = mt->second; + if (node_match.node != proj_node || node_match.monotonicity) + break; + + args.push_back(query_node); } if (args.size() < aggregate.argument_names.size()) continue; + for (const auto * node : args) + split_nodes.insert(node); + match = AggFuncMatch{idx, std::move(args)}; } @@ -260,7 +282,79 @@ std::optional analyzeAggregateProjection( aggregate_function_matches.emplace_back(std::move(*match)); } + std::unordered_set proj_key_nodes; + for (const auto & key : info.keys) + { + auto it = proj_index.find(key.name); + /// This should not happen ideally. + if (it == proj_index.end()) + break; + proj_key_nodes.insert(it->second); + } + + std::unordered_set visited; + + struct Frame + { + const ActionsDAG::Node * node; + size_t next_child_to_visit = 0; + }; + + std::stack stack; + for (const auto & key : keys) + { + auto it = index.find(key); + /// This should not happen ideally. + if (it == index.end()) + break; + + const auto * key_node = it->second; + if (visited.contains(key_node)) + continue; + + stack.push({.node = key_node}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (frame.next_child_to_visit == 0) + { + auto jt = matches.find(frame.node); + if (jt != matches.end()) + { + auto & match = jt->second; + if (match.node && !match.monotonicity && proj_key_nodes.contains(match.node)) + { + visited.insert(frame.node); + split_nodes.insert(frame.node); + stack.pop(); + continue; + } + } + } + + if (frame.next_child_to_visit < frame.node->children.size()) + { + stack.push({.node = frame.node->children[frame.next_child_to_visit]}); + ++frame.next_child_to_visit; + continue; + } + + /// Not a match and there is no matched child. + if (frame.node->children.empty()) + return {}; + + /// Not a match, but all children matched. + visited.insert(frame.node); + stack.pop(); + } + } + + + + return {}; } void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) @@ -281,8 +375,8 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) if (!buildAggregatingDAG(node, dag, filter_nodes)) return; - const auto & keys = aggregating->getParams().keys; - const auto & aggregates = aggregating->getParams().aggregates; + // const auto & keys = aggregating->getParams().keys; + // const auto & aggregates = aggregating->getParams().aggregates; auto * reading = typeid_cast(reading_node->step.get()); if (!reading) From 8453c9d27d92dec1480e77944cb9d5b00a1fb4a2 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 20 Jan 2023 07:47:08 +0000 Subject: [PATCH 0020/1548] Add construction of temporary tables by StorageFactory --- src/Interpreters/DatabaseCatalog.cpp | 5 -- src/Interpreters/DatabaseCatalog.h | 2 - src/Interpreters/InterpreterCreateQuery.cpp | 60 ++++++++++----------- 3 files changed, 29 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2daca6a55a8..6ac01a9473f 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -53,11 +53,6 @@ namespace ErrorCodes extern const int HAVE_DEPENDENT_OBJECTS; } -TemporaryTableHolder::TemporaryTableHolder(ContextPtr context, const String& table_name, const ASTPtr & query = {}) -{ - -} - TemporaryTableHolder::TemporaryTableHolder(ContextPtr context_, const TemporaryTableHolder::Creator & creator, const ASTPtr & query) : WithContext(context_->getGlobalContext()) , temporary_tables(DatabaseCatalog::instance().getDatabaseForTemporaryTables().get()) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 3028c900599..a3fa4515a69 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -88,8 +88,6 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext { using Creator = std::function; - TemporaryTableHolder(ContextPtr context, const String & table_name, const ASTPtr & query = {}); - TemporaryTableHolder(ContextPtr context, const Creator & creator, const ASTPtr & query = {}); /// Creates temporary table with Engine=Memory diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 167134bd9ba..d89fcfbc7cf 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1256,14 +1256,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.temporary) { - /* - UUID id; - if (create.uuid == UUIDHelpers::Nil) - create.uuid = UUIDHelpers::generateV4(); - id = create.uuid; - create.setTable("_tmp_" + toString(id)); - */ - // create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); + create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); } if (!ddl_guard) @@ -1372,13 +1365,34 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } else { - res = StorageFactory::instance().get(create, - data_path, - getContext(), - getContext()->getGlobalContext(), - properties.columns, - properties.constraints, - false); + if (create.temporary) + { + String temporary_table_name = create.getTable(); + auto creator = [&](const StorageID &) + { + return StorageFactory::instance().get(create, + data_path, + getContext(), + getContext()->getGlobalContext(), + properties.columns, + properties.constraints, + false); + }; + auto temporary_table = TemporaryTableHolder(getContext(), creator, query_ptr); + + getContext()->getSessionContext()->addExternalTable(temporary_table_name, std::move(temporary_table)); + return true; + } + else + { + res = StorageFactory::instance().get(create, + data_path, + getContext(), + getContext()->getGlobalContext(), + properties.columns, + properties.constraints, + false); + } /// If schema wes inferred while storage creation, add columns description to create query. addColumnsDescriptionToCreateQueryIfNecessary(query_ptr->as(), res); @@ -1397,22 +1411,6 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, "ATTACH ... FROM ... query is not supported for {} table engine, " "because such tables do not store any data on disk. Use CREATE instead.", res->getName()); - if (create.temporary) - { - // if (create.if_not_exists && getContext()->tryResolveStorageID({"", create.getTable()}, Context::ResolveExternal)) - // return false; - - String temporary_table_name = create.getTable(); - auto creator = [&](const StorageID &) - { - return std::move(res); - }; - auto temporary_table = TemporaryTableHolder(getContext(), creator, query_ptr); - // auto temporary_table = TemporaryTableHolder(getContext(), properties.columns, properties.constraints, query_ptr); - getContext()->getSessionContext()->addExternalTable(temporary_table_name, std::move(temporary_table)); - return true; - } - database->createTable(getContext(), create.getTable(), res, query_ptr); /// Move table data to the proper place. Wo do not move data earlier to avoid situations From fa642032258b1d2c3175124b2847bcfde281271d Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 20 Jan 2023 16:11:37 +0000 Subject: [PATCH 0021/1548] Add correct data path for temporary tables; Clean temp DB dir on server startup --- src/Databases/DatabaseMemory.cpp | 4 ++-- src/Interpreters/DatabaseCatalog.cpp | 5 +++++ src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 39295bf499a..ac9a56567d1 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -70,7 +70,7 @@ void DatabaseMemory::dropTable( if (table->storesDataOnDisk()) { - assert(getDatabaseName() != DatabaseCatalog::TEMPORARY_DATABASE); + // assert(getDatabaseName() != DatabaseCatalog::TEMPORARY_DATABASE); fs::path table_data_dir{getTableDataPath(table_name)}; if (fs::exists(table_data_dir)) fs::remove_all(table_data_dir); @@ -79,7 +79,7 @@ void DatabaseMemory::dropTable( catch (...) { std::lock_guard lock{mutex}; - assert(database_name != DatabaseCatalog::TEMPORARY_DATABASE); + // assert(database_name != DatabaseCatalog::TEMPORARY_DATABASE); attachTableUnlocked(table_name, table); throw; } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6ac01a9473f..8bfda0b210d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -149,6 +149,11 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase() unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, getContext()); + + /// Temporary database should not have any data on the moment of its creation + /// In case of sudden server shutdown remove database folder of temporary database + db_for_temporary_and_external_tables->drop(getContext()); + attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d89fcfbc7cf..4a3ccfe18a2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1368,10 +1368,10 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.temporary) { String temporary_table_name = create.getTable(); - auto creator = [&](const StorageID &) + auto creator = [&](const StorageID & table_id) { return StorageFactory::instance().get(create, - data_path, + database->getTableDataPath(table_id.getTableName()), getContext(), getContext()->getGlobalContext(), properties.columns, From d898a4279ad42170b1f17e8a28938b3fae1441be Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Mon, 23 Jan 2023 15:56:45 +0000 Subject: [PATCH 0022/1548] Add possibility to create MergeTree tables --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 4a3ccfe18a2..d17a59b74f4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -920,6 +920,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// It makes sense when default_table_engine setting is used, but not for temporary tables. /// For temporary tables we ignore this setting to allow CREATE TEMPORARY TABLE query without specifying ENGINE /// even if setting is set to MergeTree or something like that (otherwise MergeTree will be substituted and query will fail). + /* if (create.storage && !create.storage->engine) throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table: must be either ENGINE = Memory or empty"); @@ -930,6 +931,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const auto storage_ast = std::make_shared(); storage_ast->set(storage_ast->engine, engine_ast); create.set(create.storage, storage_ast); + */ return; } @@ -1368,6 +1370,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.temporary) { String temporary_table_name = create.getTable(); + // ASTPtr original_create = create.clone(); auto creator = [&](const StorageID & table_id) { return StorageFactory::instance().get(create, From aa0f01a409f164faa11b869531e8ae6c58b06c0c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 24 Jan 2023 06:54:19 +0000 Subject: [PATCH 0023/1548] Fix bug with DROP TEMPORARY TABLE for MergeTree --- src/Interpreters/DatabaseCatalog.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8bfda0b210d..1a117705063 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -121,9 +121,15 @@ TemporaryTableHolder::~TemporaryTableHolder() { if (id != UUIDHelpers::Nil) { - auto table = getTable(); - table->flushAndShutdown(); - temporary_tables->dropTable(getContext(), "_tmp_" + toString(id)); + // auto table = getTable(); + /// Table maybe dropped before by DROP TEMPORARY TABLE + /// Thus we need to verify the existence of the table before calling dropTable + auto table = temporary_tables->tryGetTable("_tmp_" + toString(id), getContext()); + if (!table) + { + table->flushAndShutdown(); + temporary_tables->dropTable(getContext(), "_tmp_" + toString(id)); + } } } From 78c0a6ab128a2642346c5c2dd2dc1c6372a14b8d Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 24 Jan 2023 07:23:47 +0000 Subject: [PATCH 0024/1548] Set Memory engine when there is no storage --- src/Interpreters/InterpreterCreateQuery.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d17a59b74f4..5f1bca6c476 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -923,15 +923,16 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /* if (create.storage && !create.storage->engine) throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table: must be either ENGINE = Memory or empty"); - - auto engine_ast = std::make_shared(); - // engine_ast->name = "Memory"; - engine_ast->name = create.storage->engine->name; - engine_ast->no_empty_args = true; - auto storage_ast = std::make_shared(); - storage_ast->set(storage_ast->engine, engine_ast); - create.set(create.storage, storage_ast); */ + if (!create.storage) + { + auto engine_ast = std::make_shared(); + engine_ast->name = "Memory"; + engine_ast->no_empty_args = true; + auto storage_ast = std::make_shared(); + storage_ast->set(storage_ast->engine, engine_ast); + create.set(create.storage, storage_ast); + } return; } From 640bb66065c18d41d81e6f3a027345e46f1f5f0a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 24 Jan 2023 07:39:29 +0000 Subject: [PATCH 0025/1548] Move StorageFactory to top of doCreateTable for temp tables --- src/Interpreters/InterpreterCreateQuery.cpp | 59 ++++++++------------- 1 file changed, 22 insertions(+), 37 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5f1bca6c476..cae18175757 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1244,23 +1244,30 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties, DDLGuardPtr & ddl_guard) { - /* if (create.temporary) { if (create.if_not_exists && getContext()->tryResolveStorageID({"", create.getTable()}, Context::ResolveExternal)) return false; + create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); + DatabasePtr database = DatabaseCatalog::instance().getDatabase(create.getDatabase()); + String temporary_table_name = create.getTable(); - auto temporary_table = TemporaryTableHolder(getContext(), properties.columns, properties.constraints, query_ptr); + auto creator = [&](const StorageID & table_id) + { + return StorageFactory::instance().get(create, + database->getTableDataPath(table_id.getTableName()), + getContext(), + getContext()->getGlobalContext(), + properties.columns, + properties.constraints, + false); + }; + auto temporary_table = TemporaryTableHolder(getContext(), creator, query_ptr); + getContext()->getSessionContext()->addExternalTable(temporary_table_name, std::move(temporary_table)); return true; } - */ - - if (create.temporary) - { - create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); - } if (!ddl_guard) ddl_guard = DatabaseCatalog::instance().getDDLGuard(create.getDatabase(), create.getTable()); @@ -1368,35 +1375,13 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } else { - if (create.temporary) - { - String temporary_table_name = create.getTable(); - // ASTPtr original_create = create.clone(); - auto creator = [&](const StorageID & table_id) - { - return StorageFactory::instance().get(create, - database->getTableDataPath(table_id.getTableName()), - getContext(), - getContext()->getGlobalContext(), - properties.columns, - properties.constraints, - false); - }; - auto temporary_table = TemporaryTableHolder(getContext(), creator, query_ptr); - - getContext()->getSessionContext()->addExternalTable(temporary_table_name, std::move(temporary_table)); - return true; - } - else - { - res = StorageFactory::instance().get(create, - data_path, - getContext(), - getContext()->getGlobalContext(), - properties.columns, - properties.constraints, - false); - } + res = StorageFactory::instance().get(create, + data_path, + getContext(), + getContext()->getGlobalContext(), + properties.columns, + properties.constraints, + false); /// If schema wes inferred while storage creation, add columns description to create query. addColumnsDescriptionToCreateQueryIfNecessary(query_ptr->as(), res); From 16428b91ec696b8ff9e90536e0285f42f602214c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 26 Jan 2023 09:47:36 +0000 Subject: [PATCH 0026/1548] Fix removing data on disk for table engines like Log for DatabaseMemory --- src/Databases/DatabaseMemory.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index ac9a56567d1..b46946972ed 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -70,8 +70,7 @@ void DatabaseMemory::dropTable( if (table->storesDataOnDisk()) { - // assert(getDatabaseName() != DatabaseCatalog::TEMPORARY_DATABASE); - fs::path table_data_dir{getTableDataPath(table_name)}; + fs::path table_data_dir{fs::path{getContext()->getPath()} / getTableDataPath(table_name)}; if (fs::exists(table_data_dir)) fs::remove_all(table_data_dir); } @@ -79,7 +78,6 @@ void DatabaseMemory::dropTable( catch (...) { std::lock_guard lock{mutex}; - // assert(database_name != DatabaseCatalog::TEMPORARY_DATABASE); attachTableUnlocked(table_name, table); throw; } From c09d8e744fa158b6c76c6809d0d1b087a98d589e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 26 Jan 2023 14:15:09 +0000 Subject: [PATCH 0027/1548] Make MergeTree and Log table engines DROP and clean data --- src/Interpreters/DatabaseCatalog.cpp | 3 +-- src/Interpreters/InterpreterDropQuery.cpp | 5 +++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 1a117705063..f15d6d3812e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -121,11 +121,10 @@ TemporaryTableHolder::~TemporaryTableHolder() { if (id != UUIDHelpers::Nil) { - // auto table = getTable(); /// Table maybe dropped before by DROP TEMPORARY TABLE /// Thus we need to verify the existence of the table before calling dropTable auto table = temporary_tables->tryGetTable("_tmp_" + toString(id), getContext()); - if (!table) + if (table && !table->is_dropped) { table->flushAndShutdown(); temporary_tables->dropTable(getContext(), "_tmp_" + toString(id)); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index f237814f879..c54d35f5e46 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -285,8 +285,9 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, table->flushAndShutdown(); auto table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); /// Delete table data - table->drop(); - table->is_dropped = true; + DatabasePtr database = DatabaseCatalog::instance().getDatabase(DatabaseCatalog::TEMPORARY_DATABASE); + UUID table_uuid = table->getStorageID().uuid; + database->dropTable(getContext(), "_tmp_" + toString(table_uuid)); } } } From 1793730df85ba187e4943733bc208aa227aacaf0 Mon Sep 17 00:00:00 2001 From: exX512 Date: Thu, 26 Jan 2023 21:30:36 +0000 Subject: [PATCH 0028/1548] Change behaviour of cancel in PipelineExecutor --- src/Processors/Executors/ExecutingGraph.cpp | 4 +++- src/Processors/Executors/PipelineExecutor.cpp | 7 ++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index f84efabdee1..f43f0ce8cff 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -400,7 +400,9 @@ void ExecutingGraph::cancel() { try { - processor->cancel(); + bool is_source = processor->getInputs().empty(); + if (is_source) + processor->cancel(); } catch (...) { diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index f1e044e470b..736098b267e 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -70,7 +70,6 @@ const Processors & PipelineExecutor::getProcessors() const void PipelineExecutor::cancel() { cancelled = true; - finish(); graph->cancel(); } @@ -147,7 +146,10 @@ bool PipelineExecutor::checkTimeLimitSoft() // We call cancel here so that all processors are notified and tasks waken up // so that the "break" is faster and doesn't wait for long events if (!continuing) + { cancel(); + finish(); + } return continuing; } @@ -227,7 +229,10 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie break; if (!context.executeTask()) + { cancel(); + finish(); + } if (tasks.isFinished()) break; From 286a58801e0e2ec770b8169e9a85b65233a30fc4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Jan 2023 18:38:14 +0000 Subject: [PATCH 0029/1548] Aggregate Projections analysis using query plan [In progress] --- src/Interpreters/ActionsDAG.cpp | 73 +++++++++++++++++++ src/Interpreters/ActionsDAG.h | 2 + .../Optimizations/optimizeUseProjections.cpp | 9 ++- 3 files changed, 81 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 9968c43cc9d..d2a1d7becfe 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -723,6 +723,79 @@ NameSet ActionsDAG::foldActionsByProjection( return next_required_columns; } + +ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs) +{ + auto dag = std::make_unique(); + std::unordered_map new_input_to_pos; + + std::unordered_map mapping; + struct Frame + { + const Node * node; + size_t next_child = 0; + }; + + std::vector stack; + for (const auto * output : outputs) + { + if (mapping.contains(output)) + continue; + + stack.push_back({.node = output}); + while (!stack.empty()) + { + auto & frame = stack.back(); + + if (frame.next_child == 0) + { + auto it = new_inputs.find(frame.node); + if (it != new_inputs.end()) + { + const auto & [new_input, rename] = *it; + + const auto * node = &dag->addInput(new_input->result_name, new_input->result_type); + if (!rename.empty() && new_input->result_name != rename) + node = &dag->addAlias(*node, rename); + + mapping.emplace(frame.node, node); + stack.pop_back(); + continue; + } + } + + const auto & children = frame.node->children; + + while (frame.next_child < children.size() && !mapping.emplace(children[frame.next_child], nullptr).second) + ++frame.next_child; + + if (frame.next_child < children.size()) + { + const auto * child = children[frame.next_child]; + ++frame.next_child; + stack.push_back({.node = child}); + continue; + } + + if (frame.node->type == ActionType::INPUT) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot fold actions for projection. Node {} requires input {} which does not belong to projection", + stack.front().node->result_name, frame.node->result_name); + + auto & node = dag->nodes.emplace_back(*frame.node); + for (auto & child : node.children) + child = mapping[child]; + + stack.pop_back(); + } + } + + for (const auto * output : outputs) + dag->outputs.push_back(mapping[output]); + + return dag; +} + void ActionsDAG::reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map) { ::sort(outputs.begin(), outputs.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index a26694e00f5..b23c87b4903 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -214,6 +214,8 @@ public: const String & predicate_column_name = {}, bool add_missing_keys = true); + ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs); + /// Reorder the output nodes using given position mapping. void reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 18dd5c8adcb..3ec632843ff 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -148,9 +148,10 @@ struct AggregateProjectionCandidate { AggregateProjectionInfo info; ProjectionDescription * projection; + ActionsDAGPtr dag; }; -std::optional analyzeAggregateProjection( +ActionsDAGPtr analyzeAggregateProjection( //ProjectionDescription & projection, AggregateProjectionInfo info, ActionsDAG & query_dag, @@ -352,9 +353,11 @@ std::optional analyzeAggregateProjection( } } + std::unordered_map new_inputs; + for (const auto * node : split_nodes) + new_inputs[node] = matches[node].node->result_name; - - return {}; + return query_dag.foldActionsByProjection(new_inputs); } void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) From df4a9f111c56c183465c2c205be31fc0041f33f7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Mon, 30 Jan 2023 08:08:20 +0000 Subject: [PATCH 0030/1548] Disallow creation of temporary tables with ON CLUSTER and with Repliacted table engines --- src/Interpreters/InterpreterCreateQuery.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cae18175757..01a6bb76f60 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -924,6 +924,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.storage && !create.storage->engine) throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table: must be either ENGINE = Memory or empty"); */ + + if (!create.cluster.empty()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with ON CLUSTER clause"); + + if (create.storage && create.storage->engine && create.storage->engine->name.starts_with("Replicated")) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated table engines"); + if (!create.storage) { auto engine_ast = std::make_shared(); From f09f8f80af5a62c1550928fa19dd8ef403aad5b2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Jan 2023 20:09:58 +0000 Subject: [PATCH 0031/1548] Aggregate Projections analysis using query plan [In progress] --- .../Optimizations/optimizeUseProjections.cpp | 76 +++++++++++++++---- 1 file changed, 60 insertions(+), 16 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 3ec632843ff..5795687fbb3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -6,6 +6,9 @@ #include #include #include +#include +#include +#include #include namespace DB::QueryPlanOptimizations @@ -119,7 +122,6 @@ struct AggregateProjectionInfo AggregateProjectionInfo getAggregatingProjectionInfo( const ProjectionDescription & projection, const ContextPtr & context, - StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) { /// This is a bad approach. @@ -128,8 +130,7 @@ AggregateProjectionInfo getAggregatingProjectionInfo( InterpreterSelectQuery interpreter( projection.query_ast, context, - storage, - metadata_snapshot, + Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), SelectQueryOptions{QueryProcessingStage::WithMergeableState}); const auto & analysis_result = interpreter.getAnalysisResult(); @@ -147,13 +148,12 @@ AggregateProjectionInfo getAggregatingProjectionInfo( struct AggregateProjectionCandidate { AggregateProjectionInfo info; - ProjectionDescription * projection; + const ProjectionDescription * projection; ActionsDAGPtr dag; }; ActionsDAGPtr analyzeAggregateProjection( - //ProjectionDescription & projection, - AggregateProjectionInfo info, + const AggregateProjectionInfo & info, ActionsDAG & query_dag, const Names & keys, const AggregateDescriptions & aggregates) @@ -373,22 +373,66 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) if (!reading_node) return; - ActionsDAGPtr dag; - ActionsDAG::NodeRawConstPtrs filter_nodes; - if (!buildAggregatingDAG(node, dag, filter_nodes)) - return; - - // const auto & keys = aggregating->getParams().keys; - // const auto & aggregates = aggregating->getParams().aggregates; - auto * reading = typeid_cast(reading_node->step.get()); if (!reading) return; - // const auto metadata = reading->getStorageMetadata(); - // const auto & projections = metadata->projections; + const auto metadata = reading->getStorageMetadata(); + const auto & projections = metadata->projections; + std::vector agg_projections; + for (const auto & projection : projections) + if (projection.type == ProjectionDescription::Type::Aggregate) + agg_projections.push_back(&projection); + if (agg_projections.empty()) + return; + + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) + return; + + const ActionsDAG::Node * filter_node = nullptr; + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().insert(dag->getOutputs().begin(), filter_node); + } + + ContextPtr context = reading->getContext(); + + const auto & keys = aggregating->getParams().keys; + const auto & aggregates = aggregating->getParams().aggregates; + + std::vector candidates; + candidates.reserve(agg_projections.size()); + for (const auto * projection : agg_projections) + { + auto info = getAggregatingProjectionInfo(*projection, context, metadata); + if (auto proj_dag = analyzeAggregateProjection(info, *dag, keys, aggregates)) + { + candidates.emplace_back(AggregateProjectionCandidate{ + .info = std::move(info), + .projection = projection, + .dag = std::move(proj_dag), + }); + } + } + + if (candidates.empty()) + return; + + } } From c65e90da2c1b9b8427fb5fc3d892ba18137ec3bd Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 31 Jan 2023 10:45:42 +0000 Subject: [PATCH 0032/1548] Fix SHOW CREATE TEMPORARY TABLE --- src/Parsers/ASTCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index d7dc4e217b7..27b65030966 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -288,7 +288,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat << what << " " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") - << (database ? backQuoteIfNeed(getDatabase()) + "." : "") << backQuoteIfNeed(getTable()); + << (database && !temporary ? backQuoteIfNeed(getDatabase()) + "." : "") << backQuoteIfNeed(getTable()); if (uuid != UUIDHelpers::Nil) settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") From e2c32ccbca9d1ee13af2f3febee9891ce9692fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Jan 2023 20:33:01 +0000 Subject: [PATCH 0033/1548] Aggregate Projections analysis using query plan [In progress] --- src/Processors/QueryPlan/AggregatingStep.cpp | 23 +++ src/Processors/QueryPlan/AggregatingStep.h | 3 + .../Optimizations/optimizeUseProjections.cpp | 131 +++++++++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 11 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 ++- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- src/Storages/StorageReplicatedMergeTree.h | 5 +- 7 files changed, 179 insertions(+), 18 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 4fd6e7c11dd..b52982b0425 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -406,6 +406,11 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return; } + if (input_streams.size() > 1) + { + + } + /// If there are several sources, then we perform parallel aggregation if (pipeline.getNumStreams() > 1) { @@ -465,6 +470,24 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const } } +void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) +{ + auto cur_header = getOutputStream().header; + input_streams.front() = input_stream; + params.only_merge = true; + updateOutputStream(); + assertBlocksHaveEqualStructure(cur_header, getOutputStream().header, "AggregatingStep"); +} + +void AggregatingStep::appendAggregateProjection(const DataStream & input_stream) +{ + input_streams.emplace_back(input_stream); + params.only_merge = true; + auto added_header = appendGroupingColumn(params.getHeader(input_streams.front().header, final), params.keys, !grouping_sets_params.empty(), group_by_use_nulls); + assertBlocksHaveEqualStructure(getOutputStream().header, added_header, "AggregatingStep"); + params.only_merge = false; +} + void AggregatingStep::updateOutputStream() { output_stream = createOutputStream( diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 0dc06649d2d..2ceca926684 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -60,6 +60,9 @@ public: void applyOrder(SortDescription sort_description_for_merging_, SortDescription group_by_sort_description_); bool memoryBoundMergingWillBeUsed() const; + void requestOnlyMergeForAggregateProjection(const DataStream & input_stream); + void appendAggregateProjection(const DataStream & input_stream); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 5795687fbb3..2c9ad818ed4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include #include namespace DB::QueryPlanOptimizations @@ -16,7 +18,7 @@ namespace DB::QueryPlanOptimizations QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { - IQueryPlanStep * step = node.step.get();\ + IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) { /// Already read-in-order, skip. @@ -150,6 +152,9 @@ struct AggregateProjectionCandidate AggregateProjectionInfo info; const ProjectionDescription * projection; ActionsDAGPtr dag; + + MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; + MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; }; ActionsDAGPtr analyzeAggregateProjection( @@ -360,7 +365,7 @@ ActionsDAGPtr analyzeAggregateProjection( return query_dag.foldActionsByProjection(new_inputs); } -void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) +void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) return; @@ -432,6 +437,128 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) if (candidates.empty()) return; + AggregateProjectionCandidate * best_candidate = nullptr; + size_t best_candidate_marks = 0; + + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + + std::shared_ptr max_added_blocks; + if (context->getSettingsRef().select_sequential_consistency) + { + if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); + } + + for (auto & candidate : candidates) + { + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & created_projections = part->getProjectionParts(); + auto it = created_projections.find(candidate.projection->name); + if (it != created_projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } + + if (projection_parts.empty()) + continue; + + ActionDAGNodes added_filter_nodes; + if (filter_node) + added_filter_nodes.nodes.push_back(candidate.dag->getOutputs().front()); + + auto projection_result_ptr = reader.estimateNumMarksToRead( + projection_parts, + nullptr, + candidate.dag->getRequiredColumnsNames(), + metadata, + candidate.projection->metadata, + query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes + added_filter_nodes, + context, + context->getSettingsRef().max_threads, + max_added_blocks); + + if (projection_result_ptr->error()) + continue; + + size_t sum_marks = projection_result_ptr->marks(); + + if (!normal_parts.empty()) + { + auto normal_result_ptr = reading->selectRangesToRead(std::move(normal_parts)); + + if (normal_result_ptr->error()) + continue; + + if (normal_result_ptr->marks() != 0) + { + sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); + } + } + + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + + if (best_candidate == nullptr || best_candidate_marks > sum_marks) + { + best_candidate = &candidate; + best_candidate_marks = sum_marks; + } + } + + if (!best_candidate) + return; + + auto projection_reading = reader.readFromParts( + {}, + best_candidate->dag->getRequiredColumnsNames(), + reading->getStorageSnapshot(), + query_info, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + projection_reading->setStepDescription(best_candidate->projection->name); + + if (!best_candidate->merge_tree_normal_select_result_ptr) + { + /// All parts are taken from projection + + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto & expr_or_filter_node = nodes.emplace_back(); + + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag, + best_candidate->dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); + node.children.front() = &expr_or_filter_node; + + optimizeAggregationInOrder(node, nodes); + + return; + } + } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index a3cea2a8afe..1c614ed09f1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -147,13 +147,23 @@ public: bool sample_factor_column_queried, Poco::Logger * log); + MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; + ContextPtr getContext() const { return context; } const SelectQueryInfo & getQueryInfo() const { return query_info; } StorageMetadataPtr getStorageMetadata() const { return metadata_for_reading; } + StorageSnapshotPtr getStorageSnapshot() const { return storage_snapshot; } const PrewhereInfo * getPrewhereInfo() const { return prewhere_info.get(); } void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } + const MergeTreeData & getMergeTreeData() const { return data; } + const Names & getRealColumnNames() const { return real_column_names; } + size_t getMaxBlockSize() const { return max_block_size; } + size_t getNumStreams() const { return requested_num_streams; } + bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; } + private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, @@ -232,7 +242,6 @@ private: const Names & column_names, ActionsDAGPtr & out_projection); - MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; ReadFromMergeTree::AnalysisResult getAnalysisResult() const; MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1ca1779e4b0..242f86e171c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -156,7 +156,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( if (!query_info.projection) { - auto plan = readFromParts( + auto step = readFromParts( query_info.merge_tree_select_result_ptr ? MergeTreeData::DataPartsVector{} : parts, column_names_to_return, storage_snapshot, @@ -168,12 +168,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( query_info.merge_tree_select_result_ptr, enable_parallel_reading); - if (plan->isInitialized() && settings.allow_experimental_projection_optimization && settings.force_optimize_projection + if (!step && settings.allow_experimental_projection_optimization && settings.force_optimize_projection && !metadata_for_reading->projections.empty()) throw Exception( "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", ErrorCodes::PROJECTION_NOT_USED); + auto plan = std::make_unique(); + plan->addStep(std::move(step)); return plan; } @@ -197,7 +199,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( else if (query_info.projection->merge_tree_projection_select_result_ptr) { LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", ")); - projection_plan = readFromParts( + projection_plan->addStep(readFromParts( {}, query_info.projection->required_columns, storage_snapshot, @@ -207,7 +209,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( num_streams, max_block_numbers_to_read, query_info.projection->merge_tree_projection_select_result_ptr, - enable_parallel_reading); + enable_parallel_reading)); } if (projection_plan->isInitialized()) @@ -1337,7 +1339,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar log); } -QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( +QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, @@ -1353,10 +1355,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (merge_tree_select_result_ptr) { if (merge_tree_select_result_ptr->marks() == 0) - return std::make_unique(); + return {}; } else if (parts.empty()) - return std::make_unique(); + return {}; Names real_column_names; Names virt_column_names; @@ -1366,7 +1368,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - auto read_from_merge_tree = std::make_unique( + return std::make_unique( std::move(parts), real_column_names, virt_column_names, @@ -1382,10 +1384,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( merge_tree_select_result_ptr, enable_parallel_reading ); - - QueryPlanPtr plan = std::make_unique(); - plan->addStep(std::move(read_from_merge_tree)); - return plan; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 30d09312245..a337574bb64 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -39,7 +39,7 @@ public: bool enable_parallel_reading = false) const; /// The same as read, but with specified set of parts. - QueryPlanPtr readFromParts( + QueryPlanStepPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c3bd682a29c..44f34adf3c0 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -322,12 +322,13 @@ public: const String & replica_name, const String & zookeeper_path, const ContextPtr & local_context, const zkutil::ZooKeeperPtr & zookeeper); bool canUseZeroCopyReplication() const; -private: - std::atomic_bool are_restoring_replica {false}; /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; +private: + std::atomic_bool are_restoring_replica {false}; + /// Delete old parts from disk and from ZooKeeper. void clearOldPartsAndRemoveFromZK(); From ae420b30e50196876a87a57878e0c5baa80a9e2a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Feb 2023 17:55:06 +0000 Subject: [PATCH 0034/1548] Aggregate Projections analysis using query plan [In progress] --- src/Processors/QueryPlan/AggregatingStep.cpp | 133 ++++++++++++++++-- src/Processors/QueryPlan/AggregatingStep.h | 40 +++++- .../Optimizations/optimizeUseProjections.cpp | 46 +++--- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- 4 files changed, 188 insertions(+), 33 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index b52982b0425..db85764ef33 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -406,11 +406,6 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return; } - if (input_streams.size() > 1) - { - - } - /// If there are several sources, then we perform parallel aggregation if (pipeline.getNumStreams() > 1) { @@ -470,8 +465,16 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const } } +bool AggregatingStep::canUseProjection() const +{ + return grouping_sets_params.empty() && sort_description_for_merging.empty() && !memory_bound_merging_of_aggregation_results_enabled; +} + void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) { + if (!canUseProjection()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot aggregate from projection"); + auto cur_header = getOutputStream().header; input_streams.front() = input_stream; params.only_merge = true; @@ -479,13 +482,26 @@ void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & assertBlocksHaveEqualStructure(cur_header, getOutputStream().header, "AggregatingStep"); } -void AggregatingStep::appendAggregateProjection(const DataStream & input_stream) +std::unique_ptr AggregatingStep::convertToAggregatingProjection(const DataStream & input_stream) const { - input_streams.emplace_back(input_stream); - params.only_merge = true; - auto added_header = appendGroupingColumn(params.getHeader(input_streams.front().header, final), params.keys, !grouping_sets_params.empty(), group_by_use_nulls); - assertBlocksHaveEqualStructure(getOutputStream().header, added_header, "AggregatingStep"); - params.only_merge = false; + if (!canUseProjection()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot aggregate from projection"); + + auto aggregating_projection = std::make_unique( + DataStreams{input_streams.front(), input_stream}, + params, + final, + //max_block_size, + merge_threads, + temporary_data_merge_threads + //group_by_use_nulls, + //group_by_sort_description, + //should_produce_results_in_order_of_bucket_number + //memory_bound_merging_of_aggregation_results_enabled, + ); + + assertBlocksHaveEqualStructure(getOutputStream().header, aggregating_projection->getOutputStream().header, "AggregatingStep"); + return aggregating_projection; } void AggregatingStep::updateOutputStream() @@ -502,4 +518,99 @@ bool AggregatingStep::memoryBoundMergingWillBeUsed() const should_produce_results_in_order_of_bucket_number, memory_bound_merging_of_aggregation_results_enabled, sort_description_for_merging); } +AggregatingProjectionStep::AggregatingProjectionStep( + DataStreams input_streams_, + Aggregator::Params params_, + bool final_, + //size_t max_block_size_, + size_t merge_threads_, + size_t temporary_data_merge_threads_ + //bool group_by_use_nulls_, + //SortDescription group_by_sort_description_, + //bool should_produce_results_in_order_of_bucket_number_ + //bool memory_bound_merging_of_aggregation_results_enabled_ + ) + : params(std::move(params_)) + , final(final_) + //, max_block_size(max_block_size_) + , merge_threads(merge_threads_) + , temporary_data_merge_threads(temporary_data_merge_threads_) + //, group_by_use_nulls(group_by_use_nulls_) + //, group_by_sort_description(std::move(group_by_sort_description_)) + //, should_produce_results_in_order_of_bucket_number(should_produce_results_in_order_of_bucket_number_) + //, memory_bound_merging_of_aggregation_results_enabled(memory_bound_merging_of_aggregation_results_enabled_) +{ + input_streams = std::move(input_streams_); + + if (input_streams.size() != 2) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "AggregatingProjectionStep is expected to have two input streams, got {}", + input_streams.size()); + + auto normal_parts_header = params.getHeader(input_streams.front().header, final); + params.only_merge = true; + auto projection_parts_header = params.getHeader(input_streams.back().header, final); + params.only_merge = false; + + assertBlocksHaveEqualStructure(normal_parts_header, projection_parts_header, "AggregatingProjectionStep"); + output_stream.emplace(); + output_stream->header = std::move(normal_parts_header); +} + +QueryPipelineBuilderPtr AggregatingProjectionStep::updatePipeline( + QueryPipelineBuilders pipelines, + const BuildQueryPipelineSettings &) +{ + auto & normal_parts_pipeline = pipelines.front(); + auto & projection_parts_pipeline = pipelines.back(); + + /// Here we create shared ManyAggregatedData for both projection and ordinary data. + /// For ordinary data, AggregatedData is filled in a usual way. + /// For projection data, AggregatedData is filled by merging aggregation states. + /// When all AggregatedData is filled, we merge aggregation states together in a usual way. + /// Pipeline will look like: + /// ReadFromProjection -> Aggregating (only merge states) -> + /// ReadFromProjection -> Aggregating (only merge states) -> + /// ... -> Resize -> ConvertingAggregatedToChunks + /// ReadFromOrdinaryPart -> Aggregating (usual) -> (added by last Aggregating) + /// ReadFromOrdinaryPart -> Aggregating (usual) -> + /// ... + auto many_data = std::make_shared(normal_parts_pipeline->getNumStreams() + projection_parts_pipeline->getNumStreams()); + size_t counter = 0; + + AggregatorListPtr aggregator_list_ptr = std::make_shared(); + + /// TODO apply optimize_aggregation_in_order here too (like below) + auto build_aggregate_pipeline = [&](QueryPipelineBuilder & pipeline, bool projection) + { + auto params_copy = params; + if (projection) + params_copy.only_merge = true; + + AggregatingTransformParamsPtr transform_params = std::make_shared( + pipeline.getHeader(), std::move(params_copy), aggregator_list_ptr, final); + + pipeline.resize(pipeline.getNumStreams(), true, true); + + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, transform_params, many_data, counter++, merge_threads, temporary_data_merge_threads); + }); + }; + + build_aggregate_pipeline(*normal_parts_pipeline, false); + build_aggregate_pipeline(*projection_parts_pipeline, true); + + auto pipeline = std::make_unique(); + + for (auto & cur_pipeline : pipelines) + assertBlocksHaveEqualStructure(cur_pipeline->getHeader(), getOutputStream().header, "AggregatingProjectionStep"); + + *pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), 0, &processors); + pipeline->resize(1); + return pipeline; +} + } diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 2ceca926684..19c6a3a9d45 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -22,6 +22,8 @@ using GroupingSetsParamsList = std::vector; Block appendGroupingSetColumn(Block header); Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls); +class AggregatingProjectionStep; + /// Aggregation. See AggregatingTransform. class AggregatingStep : public ITransformingStep { @@ -60,8 +62,9 @@ public: void applyOrder(SortDescription sort_description_for_merging_, SortDescription group_by_sort_description_); bool memoryBoundMergingWillBeUsed() const; + bool canUseProjection() const; void requestOnlyMergeForAggregateProjection(const DataStream & input_stream); - void appendAggregateProjection(const DataStream & input_stream); + std::unique_ptr convertToAggregatingProjection(const DataStream & input_stream) const; private: void updateOutputStream() override; @@ -95,4 +98,39 @@ private: Processors aggregating; }; +class AggregatingProjectionStep : public IQueryPlanStep +{ +public: + AggregatingProjectionStep( + DataStreams input_streams_, + Aggregator::Params params_, + bool final_, + //size_t max_block_size_, + size_t merge_threads_, + size_t temporary_data_merge_threads_ + //bool group_by_use_nulls_, + //SortDescription group_by_sort_description_, + //bool should_produce_results_in_order_of_bucket_number_ + //bool memory_bound_merging_of_aggregation_results_enabled_ + ); + + String getName() const override { return "AggregatingProjection"; } + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; + +private: + Aggregator::Params params; + bool final; + //size_t max_block_size; + size_t merge_threads; + size_t temporary_data_merge_threads; + + //bool storage_has_evenly_distributed_read; + //bool group_by_use_nulls; + //SortDescription group_by_sort_description; + //bool should_produce_results_in_order_of_bucket_number; + //bool memory_bound_merging_of_aggregation_results_enabled; + + Processors aggregating; +}; + } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 2c9ad818ed4..e15a865a809 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -374,6 +374,9 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!aggregating) return; + if (!aggregating->canUseProjection()) + return; + QueryPlan::Node * reading_node = findReadingStep(node); if (!reading_node) return; @@ -530,36 +533,39 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) projection_reading->setStepDescription(best_candidate->projection->name); + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto & expr_or_filter_node = nodes.emplace_back(); + + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag, + best_candidate->dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + if (!best_candidate->merge_tree_normal_select_result_ptr) { /// All parts are taken from projection - auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); - auto & expr_or_filter_node = nodes.emplace_back(); - if (filter_node) - { - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - best_candidate->dag, - best_candidate->dag->getOutputs().front()->result_name, - true); - } - else - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - best_candidate->dag); - - expr_or_filter_node.children.push_back(&projection_reading_node); aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; optimizeAggregationInOrder(node, nodes); - - return; } - - + else + { + node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); + node.children.push_back(&expr_or_filter_node); + } } } diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 7bad9947a88..2e0ad116d70 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -67,7 +67,7 @@ public: size_t max_block_size, size_t num_streams) override { - query_plan = std::move(*MergeTreeDataSelectExecutor(storage) + query_plan.addStep(MergeTreeDataSelectExecutor(storage) .readFromParts( parts, column_names, From bfec300aa9e46eeec49c5deac2f2eb8f08191ea1 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 2 Feb 2023 12:33:10 +0000 Subject: [PATCH 0035/1548] Add 02525_different_engines_in_temporary_tables test --- ...rent_engines_in_temporary_tables.reference | 6 +++++ ..._different_engines_in_temporary_tables.sql | 23 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference create mode 100644 tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference new file mode 100644 index 00000000000..7ca2e136b02 --- /dev/null +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference @@ -0,0 +1,6 @@ +1 a +2 b +3 c +1 a +2 b +3 c diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql new file mode 100644 index 00000000000..1296c93ed28 --- /dev/null +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql @@ -0,0 +1,23 @@ +DROP TEMPORARY TABLE IF EXISTS table_merge_tree_02525; +CREATE TEMPORARY TABLE table_merge_tree_02525 +( + id UInt64, + info String +) +ENGINE = MergeTree +ORDER BY id +PRIMARY KEY id; +INSERT INTO table_merge_tree_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM table_merge_tree_02525; +DROP TEMPORARY TABLE table_merge_tree_02525; + +DROP TEMPORARY TABLE IF EXISTS table_log_02525; +CREATE TEMPORARY TABLE table_log_02525 +( + id UInt64, + info String +) +ENGINE = Log; +INSERT INTO table_log_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM table_log_02525; +DROP TEMPORARY TABLE table_log_02525; From fa78c5b45982a96950cf30a59fd15441f8a0c0ad Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 3 Feb 2023 08:26:04 +0000 Subject: [PATCH 0036/1548] Add TinyLog and StripeLog table engines to test --- ...rent_engines_in_temporary_tables.reference | 6 +++++ ..._different_engines_in_temporary_tables.sql | 22 +++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference index 7ca2e136b02..3ea857b1652 100644 --- a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.reference @@ -4,3 +4,9 @@ 1 a 2 b 3 c +1 a +2 b +3 c +1 a +2 b +3 c diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql index 1296c93ed28..2913a38865e 100644 --- a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql @@ -21,3 +21,25 @@ ENGINE = Log; INSERT INTO table_log_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); SELECT * FROM table_log_02525; DROP TEMPORARY TABLE table_log_02525; + +DROP TEMPORARY TABLE IF EXISTS table_stripe_log_02525; +CREATE TEMPORARY TABLE table_stripe_log_02525 +( + id UInt64, + info String +) +ENGINE = StripeLog; +INSERT INTO table_stripe_log_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM table_stripe_log_02525; +DROP TEMPORARY TABLE table_stripe_log_02525; + +DROP TEMPORARY TABLE IF EXISTS table_tiny_log_02525; +CREATE TEMPORARY TABLE table_tiny_log_02525 +( + id UInt64, + info String +) +ENGINE = TinyLog; +INSERT INTO table_tiny_log_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM table_tiny_log_02525; +DROP TEMPORARY TABLE table_tiny_log_02525; From 310a4c69af253d1013666108dac41fa080f6431e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Feb 2023 18:30:06 +0000 Subject: [PATCH 0037/1548] Aggregate Projections analysis using query plan [In progress] --- src/Core/Settings.h | 1 + .../QueryPlanOptimizationSettings.cpp | 1 + .../QueryPlanOptimizationSettings.h | 3 ++ .../Optimizations/optimizeReadInOrder.cpp | 8 +-- .../QueryPlan/Optimizations/optimizeTree.cpp | 3 ++ .../Optimizations/optimizeUseProjections.cpp | 51 ++++++++++++------- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++ 7 files changed, 47 insertions(+), 23 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b8d46244b6c..9ab1e274dff 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -587,6 +587,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimisation", 0) \ M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimisation", 0) \ + M(Bool, query_plan_optimize_projection, true, "Use query plan for aggregation-in-order optimisation", 0) \ M(UInt64, regexp_max_matches_per_row, 1000, "Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.", 0) \ \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 00abd803d2a..dcc37d197cb 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -14,6 +14,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.distinct_in_order = from.optimize_distinct_in_order; settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; settings.aggregation_in_order = from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; + settings.optimize_projection = from.allow_experimental_projection_optimization && from.query_plan_optimize_projection; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index d4989b86b68..835a8307188 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -30,6 +30,9 @@ struct QueryPlanOptimizationSettings /// If aggregation-in-order optimisation is enabled bool aggregation_in_order = false; + /// If reading from projection can be applied + bool optimize_projection = false; + static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 1876ebc4014..80dca5d3a27 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -29,7 +29,7 @@ namespace DB::QueryPlanOptimizations { -ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) +static ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) { if (auto * reading = typeid_cast(step)) { @@ -64,7 +64,7 @@ ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) return nullptr; } -QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { IQueryPlanStep * step = node.step.get(); if (auto * reading = checkSupportedReadingStep(step)) @@ -89,7 +89,7 @@ using FixedColumns = std::unordered_set; /// Right now we find only simple cases like 'and(..., and(..., and(column = value, ...), ...' /// Injective functions are supported here. For a condition 'injectiveFunction(x) = 5' column 'x' is fixed. -void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fixed_columns) +static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fixed_columns) { std::stack stack; stack.push(&filter_expression); @@ -138,7 +138,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr } } -void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) { if (dag) dag->mergeInplace(std::move(*expression->clone())); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 13095dfad47..e6568db1263 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -114,6 +114,9 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); + if (optimization_settings.optimize_projection) + optimizeUseProjections(*frame.node, nodes); + if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index e15a865a809..e2a942ed2dc 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -11,26 +11,17 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations { -QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) - { - /// Already read-in-order, skip. - if (reading->getQueryInfo().input_order_info) - return nullptr; - - const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); - if (sorting_key.column_names.empty()) - return nullptr; - return &node; - } if (node.children.size() != 1) return nullptr; @@ -41,7 +32,7 @@ QueryPlan::Node * findReadingStep(QueryPlan::Node & node) return nullptr; } -void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) { if (dag) dag->mergeInplace(std::move(*expression->clone())); @@ -51,7 +42,7 @@ void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) /// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. -bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) +static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) @@ -92,6 +83,7 @@ bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG return false; appendExpression(dag, actions); + return true; } if (auto * filter = typeid_cast(step)) @@ -101,10 +93,12 @@ bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG return false; appendExpression(dag, actions); - if (const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName())) - filter_nodes.push_back(filter_expression); - else + const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); + if (!filter_expression) return false; + + filter_nodes.push_back(filter_expression); + return true; } return false; @@ -374,17 +368,23 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!aggregating) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 2"); if (!aggregating->canUseProjection()) return; - QueryPlan::Node * reading_node = findReadingStep(node); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 3"); + QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); if (!reading_node) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 4"); + auto * reading = typeid_cast(reading_node->step.get()); if (!reading) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); + const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; @@ -396,11 +396,15 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (agg_projections.empty()) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); + ActionsDAGPtr dag; ActionsDAG::NodeRawConstPtrs filter_nodes; if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + const ActionsDAG::Node * filter_node = nullptr; if (!filter_nodes.empty()) { @@ -426,9 +430,13 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) candidates.reserve(agg_projections.size()); for (const auto * projection : agg_projections) { + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, keys, aggregates)) { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); candidates.emplace_back(AggregateProjectionCandidate{ .info = std::move(info), .projection = projection, @@ -519,10 +527,15 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!best_candidate) return; + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + auto projection_reading = reader.readFromParts( {}, best_candidate->dag->getRequiredColumnsNames(), - reading->getStorageSnapshot(), + proj_snapshot, query_info, context, reading->getMaxBlockSize(), @@ -559,7 +572,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; - optimizeAggregationInOrder(node, nodes); + //optimizeAggregationInOrder(node, nodes); } else { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2a341b6f1de..dc6c70f66cd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6159,6 +6159,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (analysis_result.join != nullptr || analysis_result.array_join != nullptr) can_use_aggregate_projection = false; + if (settings.query_plan_optimize_projection) + can_use_aggregate_projection = false; + /// Check if all needed columns can be provided by some aggregate projection. Here we also try /// to find expression matches. For example, suppose an aggregate projection contains a column /// named sum(x) and the given query also has an expression called sum(x), it's a match. This is From b31decc15a76474bebddd357d4009a7254331f70 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Sun, 5 Feb 2023 12:54:50 +0000 Subject: [PATCH 0038/1548] Fix bug with wrong checking of execution finish in PullingAsyncPipeline --- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index fbbf8c119ce..0a7a9025b30 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -117,7 +117,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) data->rethrowExceptionIfHas(); bool is_execution_finished - = !data->executor->checkTimeLimitSoft() || lazy_format ? lazy_format->isFinished() : data->is_finished.load(); + = !data->executor->checkTimeLimitSoft() || (lazy_format ? lazy_format->isFinished() : data->is_finished.load()); if (is_execution_finished) { From a4079bc360c4872eff00f43e63b2fd43e9af56cb Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Mon, 6 Feb 2023 09:52:52 +0000 Subject: [PATCH 0039/1548] Update docs for temporary tables --- .../sql-reference/statements/create/table.md | 4 ++-- .../sql-reference/statements/create/table.md | 18 +++++++++--------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 68fb968c609..17580f806e6 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -369,7 +369,7 @@ ENGINE = MergeTree ORDER BY x; ClickHouse supports temporary tables which have the following characteristics: - Temporary tables disappear when the session ends, including if the connection is lost. -- A temporary table uses the Memory engine only. +- A temporary table uses the Memory table engine when engine is not specified and it may use any table engine except for Replicated engines. - The DB can’t be specified for a temporary table. It is created outside of databases. - Impossible to create a temporary table with distributed DDL query on all cluster servers (by using `ON CLUSTER`): this table exists only in the current session. - If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used. @@ -383,7 +383,7 @@ CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) +) [ENGINE = engine] ``` In most cases, temporary tables are not created manually, but when using external data for a query, or for distributed `(GLOBAL) IN`. For more information, see the appropriate sections diff --git a/docs/ru/sql-reference/statements/create/table.md b/docs/ru/sql-reference/statements/create/table.md index c7ee2a62d98..7152f01892e 100644 --- a/docs/ru/sql-reference/statements/create/table.md +++ b/docs/ru/sql-reference/statements/create/table.md @@ -260,8 +260,8 @@ ENGINE = MergeTree() Кодеки шифрованиÑ: -- `CODEC('AES-128-GCM-SIV')` — Зашифровывает данные Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ AES-128 в режиме [RFC 8452](https://tools.ietf.org/html/rfc8452) GCM-SIV. -- `CODEC('AES-256-GCM-SIV')` — Зашифровывает данные Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ AES-256 в режиме GCM-SIV. +- `CODEC('AES-128-GCM-SIV')` — Зашифровывает данные Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ AES-128 в режиме [RFC 8452](https://tools.ietf.org/html/rfc8452) GCM-SIV. +- `CODEC('AES-256-GCM-SIV')` — Зашифровывает данные Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ AES-256 в режиме GCM-SIV. Эти кодеки иÑпользуют фикÑированный одноразовый ключ шифрованиÑ. Таким образом, Ñто детерминированное шифрование. Оно ÑовмеÑтимо Ñ Ð¿Ð¾Ð´Ð´ÐµÑ€Ð¶Ð¸Ð²Ð°ÑŽÑ‰Ð¸Ð¼Ð¸ дедупликацию движками, в чаÑтноÑти, [ReplicatedMergeTree](../../../engines/table-engines/mergetree-family/replication.md). Однако у ÑˆÐ¸Ñ„Ñ€Ð¾Ð²Ð°Ð½Ð¸Ñ Ð¸Ð¼ÐµÐµÑ‚ÑÑ Ð½ÐµÐ´Ð¾Ñтаток: еÑли дважды зашифровать один и тот же блок данных, текÑÑ‚ на выходе получитÑÑ Ð¾Ð´Ð¸Ð½Ð°ÐºÐ¾Ð²Ñ‹Ð¼, и злоумышленник, у которого еÑÑ‚ÑŒ доÑтуп к диÑку, заметит Ñту ÑквивалентноÑÑ‚ÑŒ (при Ñтом доÑтупа к Ñодержимому он не получит). @@ -274,10 +274,10 @@ ENGINE = MergeTree() **Пример** ```sql -CREATE TABLE mytable +CREATE TABLE mytable ( x String Codec(AES_128_GCM_SIV) -) +) ENGINE = MergeTree ORDER BY x; ``` @@ -287,10 +287,10 @@ ENGINE = MergeTree ORDER BY x; **Пример** ```sql -CREATE TABLE mytable +CREATE TABLE mytable ( x String Codec(Delta, LZ4, AES_128_GCM_SIV) -) +) ENGINE = MergeTree ORDER BY x; ``` @@ -299,9 +299,9 @@ ENGINE = MergeTree ORDER BY x; ClickHouse поддерживает временные таблицы Ñо Ñледующими характериÑтиками: - Временные таблицы иÑчезают поÑле Ð·Ð°Ð²ÐµÑ€ÑˆÐµÐ½Ð¸Ñ ÑеÑÑии, в том чиÑле при обрыве ÑоединениÑ. -- Ð’Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° иÑпользует только модуль памÑти. +- Ð’Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° иÑпользует движок таблиц Memory когда движок не указан и она может иÑпользовать любой движок таблиц за иÑключением движков Replicated. - Ðевозможно указать базу данных Ð´Ð»Ñ Ð²Ñ€ÐµÐ¼ÐµÐ½Ð½Ð¾Ð¹ таблицы. Она ÑоздаетÑÑ Ð²Ð½Ðµ баз данных. -- Ðевозможно Ñоздать временную таблицу раÑпределнным DDL запроÑом на вÑех Ñерверах клаÑтера (Ñ Ð¾Ð¿Ñ†Ð¸ÐµÐ¹ `ON CLUSTER`): Ñ‚Ð°ÐºÐ°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° ÑущеÑтвует только в рамках ÑущеÑтвующей ÑеÑÑии. +- Ðевозможно Ñоздать временную таблицу раÑпределённым DDL запроÑом на вÑех Ñерверах клаÑтера (Ñ Ð¾Ð¿Ñ†Ð¸ÐµÐ¹ `ON CLUSTER`): Ñ‚Ð°ÐºÐ°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° ÑущеÑтвует только в рамках ÑущеÑтвующей ÑеÑÑии. - ЕÑли Ð²Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° имеет то же имÑ, что и Ð½ÐµÐºÐ¾Ñ‚Ð¾Ñ€Ð°Ñ Ð´Ñ€ÑƒÐ³Ð°Ñ, то, при упоминании в запроÑе без ÑƒÐºÐ°Ð·Ð°Ð½Ð¸Ñ Ð‘Ð”, будет иÑпользована Ð²Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð°. - При раÑпределённой обработке запроÑа, иÑпользуемые в запроÑе временные таблицы, передаютÑÑ Ð½Ð° удалённые Ñерверы. @@ -313,7 +313,7 @@ CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) +) [ENGINE = engine] ``` Ð’ большинÑтве Ñлучаев, временные таблицы ÑоздаютÑÑ Ð½Ðµ вручную, а при иÑпользовании внешних данных Ð´Ð»Ñ Ð·Ð°Ð¿Ñ€Ð¾Ñа, или при раÑпределённом `(GLOBAL) IN`. Подробнее Ñм. ÑоответÑтвующие разделы From a542626fa35a7f5a37eb98358738554188f74a3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Feb 2023 18:42:58 +0000 Subject: [PATCH 0040/1548] Make 01710_aggregate_projections work. --- src/Interpreters/ActionsDAG.cpp | 19 +++++--- src/Interpreters/ActionsDAG.h | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 3 +- .../Optimizations/optimizeUseProjections.cpp | 47 ++++++++++++------- 4 files changed, 46 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index d2a1d7becfe..0df7b71f719 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -724,7 +724,7 @@ NameSet ActionsDAG::foldActionsByProjection( } -ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs) +ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs) { auto dag = std::make_unique(); std::unordered_map new_input_to_pos; @@ -737,7 +737,7 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map stack; - for (const auto * output : outputs) + for (const auto * output : required_outputs) { if (mapping.contains(output)) continue; @@ -754,11 +754,15 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapaddInput(new_input->result_name, new_input->result_type); - if (!rename.empty() && new_input->result_name != rename) - node = &dag->addAlias(*node, rename); + auto & node = mapping[frame.node]; + + if (!node) + { + node = &dag->addInput(new_input->result_name, new_input->result_type); + if (!rename.empty() && new_input->result_name != rename) + node = &dag->addAlias(*node, rename); + } - mapping.emplace(frame.node, node); stack.pop_back(); continue; } @@ -786,11 +790,12 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapoutputs.push_back(mapping[output]); return dag; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index b23c87b4903..93280c66668 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -214,7 +214,7 @@ public: const String & predicate_column_name = {}, bool add_missing_keys = true); - ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs); + static ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs); /// Reorder the output nodes using given position mapping. void reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index db85764ef33..6aacc2f8fae 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -467,7 +467,8 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const bool AggregatingStep::canUseProjection() const { - return grouping_sets_params.empty() && sort_description_for_merging.empty() && !memory_bound_merging_of_aggregation_results_enabled; + //LOG_TRACE(&Poco::Logger::get("AggregatingStep"), "canUseProjection {} {} {}", grouping_sets_params.size(), sort_description_for_merging.size(), memory_bound_merging_of_aggregation_results_enabled); + return grouping_sets_params.empty() && sort_description_for_merging.empty(); // && !memory_bound_merging_of_aggregation_results_enabled; } void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index e2a942ed2dc..185cc10e5ff 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -154,6 +154,7 @@ struct AggregateProjectionCandidate ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, ActionsDAG & query_dag, + const ActionsDAG::Node * filter_node, const Names & keys, const AggregateDescriptions & aggregates) { @@ -169,7 +170,11 @@ ActionsDAGPtr analyzeAggregateProjection( for (const auto * output : info.before_aggregation->getOutputs()) proj_index.emplace(output->result_name, output); - key_nodes.reserve(keys.size()); + key_nodes.reserve(keys.size() + 1); + + if (filter_node) + key_nodes.push_back(filter_node); + for (const auto & key : keys) { auto it = index.find(key); @@ -270,8 +275,8 @@ ActionsDAGPtr analyzeAggregateProjection( if (args.size() < aggregate.argument_names.size()) continue; - for (const auto * node : args) - split_nodes.insert(node); + // for (const auto * node : args) + // split_nodes.insert(node); match = AggFuncMatch{idx, std::move(args)}; } @@ -302,14 +307,8 @@ ActionsDAGPtr analyzeAggregateProjection( }; std::stack stack; - for (const auto & key : keys) + for (const auto * key_node : key_nodes) { - auto it = index.find(key); - /// This should not happen ideally. - if (it == index.end()) - break; - - const auto * key_node = it->second; if (visited.contains(key_node)) continue; @@ -343,8 +342,11 @@ ActionsDAGPtr analyzeAggregateProjection( } /// Not a match and there is no matched child. - if (frame.node->children.empty()) + if (frame.node->type == ActionsDAG::ActionType::INPUT) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find match for {}", frame.node->result_name); return {}; + } /// Not a match, but all children matched. visited.insert(frame.node); @@ -356,7 +358,13 @@ ActionsDAGPtr analyzeAggregateProjection( for (const auto * node : split_nodes) new_inputs[node] = matches[node].node->result_name; - return query_dag.foldActionsByProjection(new_inputs); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection"); + auto proj_dag = query_dag.foldActionsByProjection(new_inputs, key_nodes); + auto & proj_dag_outputs = proj_dag->getOutputs(); + for (const auto & aggregate : aggregates) + proj_dag_outputs.push_back(&proj_dag->addInput(aggregate.column_name, aggregate.function->getResultType())); + + return proj_dag; } void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) @@ -418,7 +426,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); } - dag->getOutputs().insert(dag->getOutputs().begin(), filter_node); + dag->getOutputs().push_back(filter_node); } ContextPtr context = reading->getContext(); @@ -434,7 +442,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); - if (auto proj_dag = analyzeAggregateProjection(info, *dag, keys, aggregates)) + if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); candidates.emplace_back(AggregateProjectionCandidate{ @@ -532,11 +540,16 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); proj_snapshot->addProjection(best_candidate->projection); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + auto projection_reading = reader.readFromParts( {}, best_candidate->dag->getRequiredColumnsNames(), proj_snapshot, - query_info, + query_info_copy, context, reading->getMaxBlockSize(), reading->getNumStreams(), @@ -544,6 +557,8 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) best_candidate->merge_tree_projection_select_result_ptr, reading->isParallelReadingEnabled()); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); + projection_reading->setStepDescription(best_candidate->projection->name); auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); @@ -568,7 +583,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { /// All parts are taken from projection - + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Expr stream {}", expr_or_filter_node.step->getOutputStream().header.dumpStructure()); aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; From b2cc2332f5883b9bdca2587aa9e4a33ee916c355 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Feb 2023 19:20:03 +0000 Subject: [PATCH 0041/1548] Fix 01710_force_use_projection. --- .../QueryPlan/Optimizations/Optimizations.h | 2 +- .../QueryPlanOptimizationSettings.cpp | 1 + .../QueryPlanOptimizationSettings.h | 1 + .../QueryPlan/Optimizations/optimizeTree.cpp | 14 +++++++++++- .../Optimizations/optimizeUseProjections.cpp | 22 ++++++++++--------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- 6 files changed, 31 insertions(+), 14 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 6f733c75e88..0ee2cecb4df 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -93,7 +93,7 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); +bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index dcc37d197cb..68b078ecd0a 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -15,6 +15,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; settings.aggregation_in_order = from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; settings.optimize_projection = from.allow_experimental_projection_optimization && from.query_plan_optimize_projection; + settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 835a8307188..7f9cc43eacd 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -32,6 +32,7 @@ struct QueryPlanOptimizationSettings /// If reading from projection can be applied bool optimize_projection = false; + bool force_use_projection = false; static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index e6568db1263..d0d634d931d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -11,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_MANY_QUERY_PLAN_OPTIMIZATIONS; + extern const int PROJECTION_NOT_USED; } namespace QueryPlanOptimizations @@ -102,6 +104,9 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { + bool applied_projection = false; + bool has_reading_from_mt = false; + Stack stack; stack.push_back({.node = &root}); @@ -111,11 +116,13 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (frame.next_child == 0) { + has_reading_from_mt |= typeid_cast(frame.node->step.get()) != nullptr; + if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - optimizeUseProjections(*frame.node, nodes); + applied_projection |= optimizeUseProjections(*frame.node, nodes); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -138,6 +145,11 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s stack.pop_back(); } + + if (optimization_settings.force_use_projection && has_reading_from_mt && !applied_projection) + throw Exception( + "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", + ErrorCodes::PROJECTION_NOT_USED); } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 185cc10e5ff..15ec1a9e0ca 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -367,29 +367,29 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } -void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) - return; + return false; auto * aggregating = typeid_cast(node.step.get()); if (!aggregating) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 2"); if (!aggregating->canUseProjection()) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 3"); QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); if (!reading_node) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 4"); auto * reading = typeid_cast(reading_node->step.get()); if (!reading) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); @@ -402,14 +402,14 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) agg_projections.push_back(&projection); if (agg_projections.empty()) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); ActionsDAGPtr dag; ActionsDAG::NodeRawConstPtrs filter_nodes; if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); @@ -454,7 +454,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } if (candidates.empty()) - return; + return false; AggregateProjectionCandidate * best_candidate = nullptr; size_t best_candidate_marks = 0; @@ -533,7 +533,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } if (!best_candidate) - return; + return false; auto storage_snapshot = reading->getStorageSnapshot(); auto proj_snapshot = std::make_shared( @@ -594,6 +594,8 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); node.children.push_back(&expr_or_filter_node); } + + return true; } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 242f86e171c..e4b0d266de0 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -169,13 +169,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( enable_parallel_reading); if (!step && settings.allow_experimental_projection_optimization && settings.force_optimize_projection - && !metadata_for_reading->projections.empty()) + && !metadata_for_reading->projections.empty() && !settings.query_plan_optimize_projection) throw Exception( "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", ErrorCodes::PROJECTION_NOT_USED); auto plan = std::make_unique(); - plan->addStep(std::move(step)); + if (step) + plan->addStep(std::move(step)); return plan; } From 2b300e06d0b54d25678b44890011fb7c208d5177 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 7 Feb 2023 14:24:47 +0000 Subject: [PATCH 0042/1548] Fix setEngine() for temporary tables --- src/Interpreters/InterpreterCreateQuery.cpp | 24 +++++++++---------- .../02184_default_table_engine.sql | 2 +- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 86e4cdf83f7..b479bc96082 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -925,26 +925,24 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.temporary) { - // if (create.storage && create.storage->engine && create.storage->engine->name != "Memory") - // throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables can only be created with ENGINE = Memory, not {}", - // create.storage->engine->name); - /// It's possible if some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not. /// It makes sense when default_table_engine setting is used, but not for temporary tables. /// For temporary tables we ignore this setting to allow CREATE TEMPORARY TABLE query without specifying ENGINE - /// even if setting is set to MergeTree or something like that (otherwise MergeTree will be substituted and query will fail). - /* - if (create.storage && !create.storage->engine) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table: must be either ENGINE = Memory or empty"); - */ if (!create.cluster.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with ON CLUSTER clause"); - if (create.storage && create.storage->engine && create.storage->engine->name.starts_with("Replicated")) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated table engines"); - - if (!create.storage) + if (create.storage) + { + if (create.storage->engine) + { + if (create.storage->engine->name.starts_with("Replicated")) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated table engines"); + } + else + throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table"); + } + else { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql index 4b5ad6c008c..109875d53a5 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.sql +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -82,7 +82,7 @@ SET default_table_engine = 'Log'; CREATE TEMPORARY TABLE tmp (n int); SHOW CREATE TEMPORARY TABLE tmp; CREATE TEMPORARY TABLE tmp1 (n int) ENGINE=Memory; -CREATE TEMPORARY TABLE tmp2 (n int) ENGINE=Log; -- {serverError 80} +CREATE TEMPORARY TABLE tmp2 (n int) ENGINE=Log; CREATE TEMPORARY TABLE tmp2 (n int) ORDER BY n; -- {serverError 80} CREATE TEMPORARY TABLE tmp2 (n int, PRIMARY KEY (n)); -- {serverError 80} From f30d1efe2f0d937943adb9d8c109f69adafdd6ca Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Feb 2023 13:32:34 +0000 Subject: [PATCH 0043/1548] Fix test 01710_projections. --- .../QueryPlan/Optimizations/matchTrees.cpp | 31 +++++++++---- .../Optimizations/optimizeUseProjections.cpp | 44 +++++++++++++++++++ 2 files changed, 67 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp index 4e11f65f1e2..6c61b557477 100644 --- a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp +++ b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations @@ -101,14 +102,20 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; bool found_all_children = true; - for (const auto * child : frame.mapped_children) - if (!child) + const ActionsDAG::Node * any_child = nullptr; + size_t num_children = frame.node->children.size(); + for (size_t i = 0; i < num_children; ++i) + { + if (frame.mapped_children[i]) + any_child = frame.mapped_children[i]; + else if (!frame.node->children[i]->column || !isColumnConst(*frame.node->children[i]->column)) found_all_children = false; + } - if (found_all_children && !frame.mapped_children.empty()) + if (found_all_children && any_child) { Parents container; - Parents * intersection = &inner_parents[frame.mapped_children[0]]; + Parents * intersection = &inner_parents[any_child]; if (frame.mapped_children.size() > 1) { @@ -116,7 +123,8 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG size_t mapped_children_size = frame.mapped_children.size(); other_parents.reserve(mapped_children_size); for (size_t i = 1; i < mapped_children_size; ++i) - other_parents.push_back(&inner_parents[frame.mapped_children[i]]); + if (frame.mapped_children[i]) + other_parents.push_back(&inner_parents[frame.mapped_children[i]]); for (const auto * parent : *intersection) { @@ -148,12 +156,19 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) { const auto & children = parent->children; - size_t num_children = children.size(); - if (frame.mapped_children.size() == num_children) + if (children.size() == num_children) { bool all_children_matched = true; for (size_t i = 0; all_children_matched && i < num_children; ++i) - all_children_matched = frame.mapped_children[i] == children[i]; + { + if (frame.mapped_children[i] == nullptr) + { + all_children_matched = children[i]->column && isColumnConst(*children[i]->column) + && assert_cast(*children[i]->column).getField() == assert_cast(*frame.node->children[i]->column).getField(); + } + else + all_children_matched = frame.mapped_children[i] == children[i]; + } if (all_children_matched) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 15ec1a9e0ca..88eb366a3a8 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include namespace DB::QueryPlanOptimizations @@ -180,7 +182,10 @@ ActionsDAGPtr analyzeAggregateProjection( auto it = index.find(key); /// This should not happen ideally. if (it == index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find key {} in query DAG", key); return {}; + } key_nodes.push_back(it->second); } @@ -192,13 +197,22 @@ ActionsDAGPtr analyzeAggregateProjection( auto it = index.find(argument); /// This should not happen ideally. if (it == index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find arg {} for agg functions {}", argument, aggregate.column_name); return {}; + } aggregate_args.insert(it->second); } } MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); + for (const auto & [node, match] : matches) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Match {} {} -> {} {} (with monotonicity : {})", + static_cast(node), node->result_name, + static_cast(match.node), (match.node ? match.node->result_name : ""), match.monotonicity != std::nullopt); + } std::unordered_map> projection_aggregate_functions; for (size_t i = 0; i < info.aggregates.size(); ++i) @@ -221,7 +235,10 @@ ActionsDAGPtr analyzeAggregateProjection( { auto it = projection_aggregate_functions.find(aggregate.function->getName()); if (it == projection_aggregate_functions.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} by name {}", aggregate.column_name, aggregate.function->getName()); return {}; + } auto & candidates = it->second; std::optional match; @@ -237,7 +254,12 @@ ActionsDAGPtr analyzeAggregateProjection( /// But also functions sum(...) and sumIf(...) will have equal states, /// and we can't replace one to another from projection. if (!candidate.function->getStateType()->equals(*aggregate.function->getStateType())) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} by state {} vs {}", + aggregate.column_name, candidate.column_name, + candidate.function->getStateType()->getName(), aggregate.function->getStateType()->getName()); continue; + } ActionsDAG::NodeRawConstPtrs args; size_t num_args = aggregate.argument_names.size(); @@ -250,24 +272,40 @@ ActionsDAGPtr analyzeAggregateProjection( auto jt = index.find(query_name); /// This should not happen ideally. if (jt == index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't find arg {} in query dag", + aggregate.column_name, candidate.column_name, query_name); break; + } const auto * query_node = jt->second; auto kt = proj_index.find(proj_name); /// This should not happen ideally. if (kt == proj_index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't find arg {} in proj dag", + aggregate.column_name, candidate.column_name, proj_name); break; + } const auto * proj_node = kt->second; auto mt = matches.find(query_node); if (mt == matches.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't match arg {} vs {} : no node in map", + aggregate.column_name, candidate.column_name, query_name, proj_name); break; + } const auto & node_match = mt->second; if (node_match.node != proj_node || node_match.monotonicity) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't match arg {} vs {} : no match or monotonicity", + aggregate.column_name, candidate.column_name, query_name, proj_name); break; + } args.push_back(query_node); } @@ -557,6 +595,12 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) best_candidate->merge_tree_projection_select_result_ptr, reading->isParallelReadingEnabled()); + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()))); + projection_reading = std::make_unique(std::move(pipe)); + } + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); projection_reading->setStepDescription(best_candidate->projection->name); From d04b3a30cc8a53de6800e428706fc1147b0bd92a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Feb 2023 14:26:20 +0000 Subject: [PATCH 0044/1548] Fix 01710_projections_partial_optimize_aggregation_in_order --- .../QueryPlan/Optimizations/optimizeUseProjections.cpp | 1 + src/Processors/QueryPlan/ReadFromMergeTree.h | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 88eb366a3a8..6ec521a9d77 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -637,6 +637,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); node.children.push_back(&expr_or_filter_node); + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } return true; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 1c614ed09f1..5ab45331722 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -157,6 +157,8 @@ public: void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } + const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const MergeTreeData & getMergeTreeData() const { return data; } const Names & getRealColumnNames() const { return real_column_names; } From e7dba2a85b96b77eac4bbcd61d9894247c5a2dd5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Feb 2023 18:34:22 +0000 Subject: [PATCH 0045/1548] Partly support minmax projection. --- .../Optimizations/optimizeUseProjections.cpp | 147 +++++++++++++----- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + 2 files changed, 106 insertions(+), 42 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 6ec521a9d77..ca16bd09649 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -151,6 +152,8 @@ struct AggregateProjectionCandidate MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; + + size_t sum_marks = 0; }; ActionsDAGPtr analyzeAggregateProjection( @@ -261,6 +264,16 @@ ActionsDAGPtr analyzeAggregateProjection( continue; } + if (typeid_cast(candidate.function.get())) + { + /// we can ignore arguments for count() + match = AggFuncMatch{idx, {}}; + break; + } + + if (aggregate.argument_names.size() != candidate.argument_names.size()) + continue; + ActionsDAG::NodeRawConstPtrs args; size_t num_args = aggregate.argument_names.size(); args.reserve(num_args); @@ -317,6 +330,7 @@ ActionsDAGPtr analyzeAggregateProjection( // split_nodes.insert(node); match = AggFuncMatch{idx, std::move(args)}; + break; } if (!match) @@ -434,12 +448,14 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; + bool can_use_minmax_projection = metadata->minmax_count_projection && !reading->getMergeTreeData().has_lightweight_delete_parts.load(); + std::vector agg_projections; for (const auto & projection : projections) if (projection.type == ProjectionDescription::Type::Aggregate) agg_projections.push_back(&projection); - if (agg_projections.empty()) + if (!can_use_minmax_projection && agg_projections.empty()) return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); @@ -473,29 +489,52 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) const auto & aggregates = aggregating->getParams().aggregates; std::vector candidates; - candidates.reserve(agg_projections.size()); - for (const auto * projection : agg_projections) - { + std::optional minmax_projection; + AggregateProjectionCandidate * best_candidate = nullptr; + + if (can_use_minmax_projection) + { + const auto * projection = &*(metadata->minmax_count_projection); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - candidates.emplace_back(AggregateProjectionCandidate{ + minmax_projection.emplace(AggregateProjectionCandidate{ .info = std::move(info), .projection = projection, .dag = std::move(proj_dag), }); } + + best_candidate = &*minmax_projection; } - if (candidates.empty()) - return false; + if (!minmax_projection) + { + candidates.reserve(agg_projections.size()); + for (const auto * projection : agg_projections) + { - AggregateProjectionCandidate * best_candidate = nullptr; - size_t best_candidate_marks = 0; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + auto info = getAggregatingProjectionInfo(*projection, context, metadata); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); + if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); + candidates.emplace_back(AggregateProjectionCandidate{ + .info = std::move(info), + .projection = projection, + .dag = std::move(proj_dag), + }); + } + } + + if (candidates.empty()) + return false; + } const auto & parts = reading->getParts(); const auto & query_info = reading->getQueryInfo(); @@ -531,7 +570,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) added_filter_nodes.nodes.push_back(candidate.dag->getOutputs().front()); auto projection_result_ptr = reader.estimateNumMarksToRead( - projection_parts, + std::move(projection_parts), nullptr, candidate.dag->getRequiredColumnsNames(), metadata, @@ -545,7 +584,8 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (projection_result_ptr->error()) continue; - size_t sum_marks = projection_result_ptr->marks(); + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); if (!normal_parts.empty()) { @@ -556,49 +596,73 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (normal_result_ptr->marks() != 0) { - sum_marks += normal_result_ptr->marks(); + candidate.sum_marks += normal_result_ptr->marks(); candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); } } - candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); - - if (best_candidate == nullptr || best_candidate_marks > sum_marks) - { + if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) best_candidate = &candidate; - best_candidate_marks = sum_marks; - } } - if (!best_candidate) + if (!best_candidate && !minmax_projection) return false; - auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); - proj_snapshot->addProjection(best_candidate->projection); - - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); - + QueryPlanStepPtr projection_reading; auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; + bool has_nornal_parts; - auto projection_reading = reader.readFromParts( - {}, - best_candidate->dag->getRequiredColumnsNames(), - proj_snapshot, - query_info_copy, - context, - reading->getMaxBlockSize(), - reading->getNumStreams(), - max_added_blocks, - best_candidate->merge_tree_projection_select_result_ptr, - reading->isParallelReadingEnabled()); - - if (!projection_reading) + if (minmax_projection) { - Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()))); + MergeTreeData::DataPartsVector normal_parts; + auto minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + minmax_projection->dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + normal_parts, + max_added_blocks.get(), + context); + + Pipe pipe(std::make_shared(std::move(minmax_count_projection_block))); projection_reading = std::make_unique(std::move(pipe)); + + has_nornal_parts = !normal_parts.empty(); + if (has_nornal_parts) + reading->resetParts(std::move(normal_parts)); + } + else + { + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + projection_reading = reader.readFromParts( + {}, + best_candidate->dag->getRequiredColumnsNames(), + proj_snapshot, + query_info_copy, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()))); + projection_reading = std::make_unique(std::move(pipe)); + } + + has_nornal_parts = best_candidate->merge_tree_normal_select_result_ptr != nullptr; + if (has_nornal_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); @@ -623,7 +687,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) expr_or_filter_node.children.push_back(&projection_reading_node); - if (!best_candidate->merge_tree_normal_select_result_ptr) + if (!has_nornal_parts) { /// All parts are taken from projection @@ -637,7 +701,6 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); node.children.push_back(&expr_or_filter_node); - reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } return true; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5ab45331722..90832d8a7ae 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -158,6 +158,7 @@ public: void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } + void resetParts(MergeTreeData::DataPartsVector parts) { prepared_parts = std::move(parts); } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const MergeTreeData & getMergeTreeData() const { return data; } From f9e81ca7dea94e019062655f08a70aaa152260a5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Feb 2023 23:30:16 +0300 Subject: [PATCH 0046/1548] 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 9e04e57949674123b9b35b4070b387cec7353092 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 9 Feb 2023 15:52:11 +0100 Subject: [PATCH 0047/1548] Allow to give access only to certain named collections --- src/Access/AccessRights.cpp | 38 +++++++--- src/Access/Common/AccessFlags.cpp | 10 ++- src/Access/Common/AccessFlags.h | 5 ++ src/Access/Common/AccessRightsElement.cpp | 23 +++--- src/Access/Common/AccessRightsElement.h | 2 + src/Access/Common/AccessType.h | 10 +-- src/Access/ContextAccess.cpp | 4 +- src/Parsers/Access/ASTGrantQuery.cpp | 21 +++--- src/Parsers/Access/ParserGrantQuery.cpp | 34 ++++++++- .../System/StorageSystemNamedCollections.cpp | 6 +- .../System/StorageSystemPrivileges.cpp | 3 + .../test_named_collections/test.py | 70 +++++++++++++++++++ 12 files changed, 189 insertions(+), 37 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 7e21e3c2c4d..595afadfe1c 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -61,12 +61,22 @@ namespace res.any_database = true; res.any_table = true; res.any_column = true; + res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); break; } case 1: { - res.any_database = false; - res.database = full_name[0]; + res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); + if (!res.any_named_collection) + { + res.any_database = true; + res.named_collection = full_name[0]; + } + else + { + res.any_database = false; + res.database = full_name[0]; + } res.any_table = true; res.any_column = true; break; @@ -317,8 +327,8 @@ public: const Node * child = tryGetChild(name); if (child) return child->isGranted(flags_to_check, subnames...); - else - return flags.contains(flags_to_check); + + return flags.contains(flags_to_check); } template @@ -783,7 +793,9 @@ void AccessRights::grantImplHelper(const AccessRightsElement & element) { assert(!element.is_partial_revoke); assert(!element.grant_option || with_grant_option); - if (element.any_database) + if (!element.any_named_collection) + grantImpl(element.access_flags, element.named_collection); + else if (element.any_database) grantImpl(element.access_flags); else if (element.any_table) grantImpl(element.access_flags, element.database); @@ -825,7 +837,10 @@ void AccessRights::grant(const AccessFlags & flags, std::string_view database, s void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessRightsElement & element) { grantImpl(element); } -void AccessRights::grant(const AccessRightsElements & elements) { grantImpl(elements); } +void AccessRights::grant(const AccessRightsElements & elements) +{ + grantImpl(elements); +} void AccessRights::grantWithGrantOption(const AccessFlags & flags) { grantImpl(flags); } void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } @@ -858,7 +873,9 @@ template void AccessRights::revokeImplHelper(const AccessRightsElement & element) { assert(!element.grant_option || grant_option); - if (element.any_database) + if (!element.any_named_collection) + revokeImpl(element.access_flags, element.named_collection); + else if (element.any_database) revokeImpl(element.access_flags); else if (element.any_table) revokeImpl(element.access_flags, element.database); @@ -912,7 +929,7 @@ void AccessRights::revokeGrantOption(const AccessRightsElements & elements) { re AccessRightsElements AccessRights::getElements() const { -#if 0 +#if 1 logTree(); #endif if (!root) @@ -934,6 +951,7 @@ bool AccessRights::isGrantedImpl(const AccessFlags & flags, const Args &... args { auto helper = [&](const std::unique_ptr & root_node) -> bool { + logTree(); if (!root_node) return flags.isEmpty(); return root_node->isGranted(flags, args...); @@ -948,7 +966,9 @@ template bool AccessRights::isGrantedImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (element.any_database) + if (!element.any_named_collection) + return isGrantedImpl(element.access_flags, element.named_collection); + else if (element.any_database) return isGrantedImpl(element.access_flags); else if (element.any_table) return isGrantedImpl(element.access_flags, element.database); diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index bef165ba4e6..533fbb0e524 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -96,11 +96,13 @@ namespace const Flags & getAllFlags() const { return all_flags; } const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } + const Flags & getGlobalWithParameterFlags() const { return all_flags_for_target[GLOBAL_WITH_PARAMETER]; } const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } + const Flags & getAllFlagsGrantableOnNamedCollectionLevel() const { return all_flags_for_target[NAMED_COLLECTION]; } const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } @@ -116,6 +118,8 @@ namespace VIEW = TABLE, COLUMN, DICTIONARY, + GLOBAL_WITH_PARAMETER, + NAMED_COLLECTION, }; struct Node; @@ -295,7 +299,7 @@ namespace collectAllFlags(child.get()); all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; - all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; + all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_for_target[NAMED_COLLECTION] | all_flags_grantable_on_table_level; } Helper() @@ -345,7 +349,7 @@ namespace std::unordered_map keyword_to_flags_map; std::vector access_type_to_flags_mapping; Flags all_flags; - Flags all_flags_for_target[static_cast(DICTIONARY) + 1]; + Flags all_flags_for_target[static_cast(NAMED_COLLECTION) + 1]; Flags all_flags_grantable_on_database_level; Flags all_flags_grantable_on_table_level; }; @@ -361,11 +365,13 @@ std::vector AccessFlags::toAccessTypes() const { return Helper::inst std::vector AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); } AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); } AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); } +AccessFlags AccessFlags::allGlobalWithParameterFlags() { return Helper::instance().getGlobalWithParameterFlags(); } AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); } AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); } AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); } AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); } AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); } +AccessFlags AccessFlags::allFlagsGrantableOnNamedCollectionLevel() { return Helper::instance().getAllFlagsGrantableOnNamedCollectionLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Helper::instance().getAllFlagsGrantableOnTableLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Helper::instance().getAllFlagsGrantableOnColumnLevel(); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index c4e0b7ac281..5443c505245 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,6 +50,7 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } + bool isNamedCollectionAccessOnly() const { return (flags & ~allFlagsGrantableOnNamedCollectionLevel()).isEmpty(); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } @@ -76,6 +77,8 @@ public: /// Returns all the global flags. static AccessFlags allGlobalFlags(); + static AccessFlags allGlobalWithParameterFlags(); + /// Returns all the flags related to a database. static AccessFlags allDatabaseFlags(); @@ -104,6 +107,8 @@ public: /// The same as allColumnFlags(). static AccessFlags allFlagsGrantableOnColumnLevel(); + static AccessFlags allFlagsGrantableOnNamedCollectionLevel(); + static constexpr size_t SIZE = 256; private: using Flags = std::bitset; diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 69a2354f25d..e2eb14ad9cb 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -21,24 +21,31 @@ namespace result += ")"; } - void formatONClause(const String & database, bool any_database, const String & table, bool any_table, String & result) + void formatONClause(const AccessRightsElement & element, String & result) { result += "ON "; - if (any_database) + if (!element.any_named_collection) + { + if (element.named_collection.empty()) + result += "*"; + else + result += backQuoteIfNeed(element.named_collection); + } + else if (element.any_database) { result += "*.*"; } else { - if (!database.empty()) + if (!element.database.empty()) { - result += backQuoteIfNeed(database); + result += backQuoteIfNeed(element.database); result += "."; } - if (any_table) + if (element.any_table) result += "*"; else - result += backQuoteIfNeed(table); + result += backQuoteIfNeed(element.table); } } @@ -96,7 +103,7 @@ namespace String result; formatAccessFlagsWithColumns(element.access_flags, element.columns, element.any_column, result); result += " "; - formatONClause(element.database, element.any_database, element.table, element.any_table, result); + formatONClause(element, result); if (with_options) formatOptions(element.grant_option, element.is_partial_revoke, result); return result; @@ -129,7 +136,7 @@ namespace if (!next_element_uses_same_table_and_options) { part += " "; - formatONClause(element.database, element.any_database, element.table, element.any_table, part); + formatONClause(element, part); if (with_options) formatOptions(element.grant_option, element.is_partial_revoke, part); if (result.empty()) diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 5f65b6bcd12..27657ea3960 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -14,9 +14,11 @@ struct AccessRightsElement String database; String table; Strings columns; + String named_collection; bool any_database = true; bool any_table = true; bool any_column = true; + bool any_named_collection = true; bool grant_option = false; bool is_partial_revoke = false; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 497327c1bad..703b3106a9a 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -12,7 +12,7 @@ enum class AccessType /// Macro M should be defined as M(name, aliases, node_type, parent_group_name) /// where name is identifier with underscores (instead of spaces); /// aliases is a string containing comma-separated list; -/// node_type either specifies access type's level (GLOBAL/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS), +/// node_type either specifies access type's level (GLOBAL/NAMED_COLLECTION/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS), /// or specifies that the access type is a GROUP of other access types; /// parent_group_name is the name of the group containing this access type (or NONE if there is no such group). #define APPLY_FOR_ACCESS_TYPES(M) \ @@ -69,7 +69,7 @@ enum class AccessType M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ \ M(ALTER_DATABASE_SETTINGS, "ALTER DATABASE SETTING, ALTER MODIFY DATABASE SETTING, MODIFY DATABASE SETTING", DATABASE, ALTER_DATABASE) /* allows to execute ALTER MODIFY SETTING */\ - M(ALTER_NAMED_COLLECTION, "", GROUP, ALTER) /* allows to execute ALTER NAMED COLLECTION */\ + M(ALTER_NAMED_COLLECTION, "", NAMED_COLLECTION, ALTER) /* allows to execute ALTER NAMED COLLECTION */\ \ M(ALTER_TABLE, "", GROUP, ALTER) \ M(ALTER_DATABASE, "", GROUP, ALTER) \ @@ -89,7 +89,7 @@ enum class AccessType M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables; implicitly enabled by the grant CREATE_TABLE on any table */ \ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ - M(CREATE_NAMED_COLLECTION, "", GLOBAL, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \ + M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\ @@ -98,7 +98,7 @@ enum class AccessType implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ - M(DROP_NAMED_COLLECTION, "", GLOBAL, DROP) /* allows to execute DROP NAMED COLLECTION */\ + M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, DROP) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \ @@ -134,7 +134,7 @@ enum class AccessType M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ - M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", GLOBAL, ACCESS_MANAGEMENT) \ + M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", NAMED_COLLECTION, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ \ M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index fbaacb2263b..6d6362a98b2 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -606,7 +606,9 @@ template bool ContextAccess::checkAccessImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (element.any_database) + if (!element.any_named_collection) + return checkAccessImpl(element.access_flags, element.named_collection); + else if (element.any_database) return checkAccessImpl(element.access_flags); else if (element.any_table) return checkAccessImpl(element.access_flags, element.database); diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index f92541ec672..e4aa11967c6 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -27,21 +27,25 @@ namespace } - void formatONClause(const String & database, bool any_database, const String & table, bool any_table, const IAST::FormatSettings & settings) + void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : ""); - if (any_database) + if (!element.any_named_collection) + { + settings.ostr << backQuoteIfNeed(element.named_collection); + } + else if (element.any_database) { settings.ostr << "*.*"; } else { - if (!database.empty()) - settings.ostr << backQuoteIfNeed(database) << "."; - if (any_table) + if (!element.database.empty()) + settings.ostr << backQuoteIfNeed(element.database) << "."; + if (element.any_table) settings.ostr << "*"; else - settings.ostr << backQuoteIfNeed(table); + settings.ostr << backQuoteIfNeed(element.table); } } @@ -71,14 +75,15 @@ namespace { const auto & next_element = elements[i + 1]; if ((element.database == next_element.database) && (element.any_database == next_element.any_database) - && (element.table == next_element.table) && (element.any_table == next_element.any_table)) + && (element.table == next_element.table) && (element.any_table == next_element.any_table) + && (element.named_collection == next_element.named_collection)) next_element_on_same_db_and_table = true; } if (!next_element_on_same_db_and_table) { settings.ostr << " "; - formatONClause(element.database, element.any_database, element.table, element.any_table, settings); + formatONClause(element, settings); } } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index 2211969c61e..a2f4e2a4921 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -123,12 +123,38 @@ namespace if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns)) return false; + String database_name, table_name, collection_name; + bool any_database = false, any_table = false, any_named_collection = true; + + size_t named_collection_access = 0; + for (const auto & elem : access_and_columns) + { + if (elem.first.isNamedCollectionAccessOnly()) + ++named_collection_access; + } + const bool grant_named_collection_access = named_collection_access == access_and_columns.size(); + if (!ParserKeyword{"ON"}.ignore(pos, expected)) return false; - String database_name, table_name; - bool any_database = false, any_table = false; - if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, any_database, table_name, any_table)) + if (grant_named_collection_access) + { + ASTPtr collection; + if (ParserToken{TokenType::Asterisk}.ignore(pos, expected)) + { + any_named_collection = true; + } + else if (ParserIdentifier{}.parse(pos, collection, expected)) + { + any_named_collection = false; + collection_name = getIdentifierName(collection); + } + else + return false; + + any_database = any_table = true; + } + else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, any_database, table_name, any_table)) return false; for (auto & [access_flags, columns] : access_and_columns) @@ -140,6 +166,8 @@ namespace element.any_database = any_database; element.database = database_name; element.any_table = any_table; + element.any_named_collection = any_named_collection; + element.named_collection = collection_name; element.table = table_name; res_elements.emplace_back(std::move(element)); } diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index bc1e3a45e6b..0b7522d3845 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -29,11 +30,14 @@ StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & t void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); + const auto & access = context->getAccess(); auto collections = NamedCollectionFactory::instance().getAll(); for (const auto & [name, collection] : collections) { + if (!access->isGranted(AccessType::SHOW_NAMED_COLLECTIONS, name)) + continue; + res_columns[0]->insert(name); auto * column_map = typeid_cast(res_columns[1].get()); diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 70163979f72..14ca76df7d8 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -28,6 +28,8 @@ namespace DICTIONARY, VIEW, COLUMN, + GLOBAL_WITH_PARAMETER, + NAMED_COLLECTION, }; DataTypeEnum8::Values getLevelEnumValues() @@ -39,6 +41,7 @@ namespace enum_values.emplace_back("DICTIONARY", static_cast(DICTIONARY)); enum_values.emplace_back("VIEW", static_cast(VIEW)); enum_values.emplace_back("COLUMN", static_cast(COLUMN)); + enum_values.emplace_back("NAMED_COLLECTION", static_cast(NAMED_COLLECTION)); return enum_values; } } diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 3b102f1aa70..0f8999f43dd 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -105,6 +105,76 @@ def test_access(cluster): assert int(node.query("select count() from system.named_collections")) > 0 +def test_granular_access(cluster): + node = cluster.instances["node"] + assert 1 == int(node.query("SELECT count() FROM system.named_collections")) + assert ( + "collection1" == node.query("SELECT name FROM system.named_collections").strip() + ) + + node.query("CREATE USER kek") + node.query("GRANT select ON *.* TO kek") + assert 0 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + + node.query("GRANT show named collections ON collection1 TO kek") + assert 1 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + assert ( + "collection1" + == node.query("SELECT name FROM system.named_collections", user="kek").strip() + ) + + node.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'") + assert 2 == int(node.query("SELECT count() FROM system.named_collections")) + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections").strip() + ) + + assert 1 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + assert ( + "collection1" + == node.query("select name from system.named_collections", user="kek").strip() + ) + + node.query("GRANT show named collections ON collection2 TO kek") + assert 2 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="kek").strip() + ) + node.restart_clickhouse() + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="kek").strip() + ) + + node.query("CREATE USER koko") + node.query("GRANT select ON *.* TO koko") + assert 0 == int( + node.query("SELECT count() FROM system.named_collections", user="koko") + ) + node.query("GRANT show named collections ON * TO koko") + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + node.restart_clickhouse() + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + + node.query("DROP NAMED COLLECTION collection2") + + def test_config_reload(cluster): node = cluster.instances["node"] assert ( From 214ffe0bb0f7d19b70649fac60886c900fb97925 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 9 Feb 2023 17:44:26 +0100 Subject: [PATCH 0048/1548] Remove unused code --- src/Access/Common/AccessFlags.cpp | 3 --- src/Access/Common/AccessFlags.h | 2 -- src/Access/Common/AccessRightsElement.cpp | 8 ++++---- src/Access/Common/AccessRightsElement.h | 2 ++ src/Storages/System/StorageSystemPrivileges.cpp | 1 - 5 files changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index 533fbb0e524..c8d2b1ca4d9 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -96,7 +96,6 @@ namespace const Flags & getAllFlags() const { return all_flags; } const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } - const Flags & getGlobalWithParameterFlags() const { return all_flags_for_target[GLOBAL_WITH_PARAMETER]; } const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } @@ -118,7 +117,6 @@ namespace VIEW = TABLE, COLUMN, DICTIONARY, - GLOBAL_WITH_PARAMETER, NAMED_COLLECTION, }; @@ -365,7 +363,6 @@ std::vector AccessFlags::toAccessTypes() const { return Helper::inst std::vector AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); } AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); } AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); } -AccessFlags AccessFlags::allGlobalWithParameterFlags() { return Helper::instance().getGlobalWithParameterFlags(); } AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); } AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); } AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 5443c505245..b81b73b8350 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -77,8 +77,6 @@ public: /// Returns all the global flags. static AccessFlags allGlobalFlags(); - static AccessFlags allGlobalWithParameterFlags(); - /// Returns all the flags related to a database. static AccessFlags allDatabaseFlags(); diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index e2eb14ad9cb..011db851bc4 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -24,12 +24,12 @@ namespace void formatONClause(const AccessRightsElement & element, String & result) { result += "ON "; - if (!element.any_named_collection) + if (element.isNamedCollectionAccess()) { - if (element.named_collection.empty()) - result += "*"; - else + if (!element.any_named_collection) result += backQuoteIfNeed(element.named_collection); + else + result += "*"; } else if (element.any_database) { diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 27657ea3960..653f813ff35 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -69,6 +69,8 @@ struct AccessRightsElement /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & current_database); + bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccessOnly(); } + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 14ca76df7d8..ee412d0e648 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -28,7 +28,6 @@ namespace DICTIONARY, VIEW, COLUMN, - GLOBAL_WITH_PARAMETER, NAMED_COLLECTION, }; From 3b72b3f13b36bde864995ce492b0892f4f2d88b0 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 10 Feb 2023 13:42:11 +0300 Subject: [PATCH 0049/1548] 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 c13b0b8a065ec5fa2275fa7ed27705c1f481eebc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 13:14:11 +0100 Subject: [PATCH 0050/1548] Fix tests --- src/Access/AccessRights.cpp | 7 +++---- src/Access/Common/AccessFlags.h | 3 ++- src/Access/Common/AccessRightsElement.h | 2 +- src/Parsers/Access/ParserGrantQuery.cpp | 4 ++-- tests/queries/0_stateless/01271_show_privileges.reference | 8 ++++---- .../0_stateless/02117_show_create_table_system.reference | 2 +- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 595afadfe1c..ca118203541 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -61,15 +61,14 @@ namespace res.any_database = true; res.any_table = true; res.any_column = true; - res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); + res.any_named_collection = true; break; } case 1: { - res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); - if (!res.any_named_collection) + if (access_flags.isNamedCollectionAccess()) { - res.any_database = true; + res.any_named_collection = false; res.named_collection = full_name[0]; } else diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index b81b73b8350..f475896b6e5 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,7 +50,8 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool isNamedCollectionAccessOnly() const { return (flags & ~allFlagsGrantableOnNamedCollectionLevel()).isEmpty(); } + bool containsOnly(const AccessFlags & other) const { return flags == other.flags; } + bool isNamedCollectionAccess() const { return containsOnly(AccessFlags::allFlagsGrantableOnNamedCollectionLevel()); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 653f813ff35..db6a1f6872f 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -69,7 +69,7 @@ struct AccessRightsElement /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & current_database); - bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccessOnly(); } + bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccess(); } /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index a2f4e2a4921..06660642fbf 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -129,14 +129,14 @@ namespace size_t named_collection_access = 0; for (const auto & elem : access_and_columns) { - if (elem.first.isNamedCollectionAccessOnly()) + if (elem.first.isNamedCollectionAccess()) ++named_collection_access; } - const bool grant_named_collection_access = named_collection_access == access_and_columns.size(); if (!ParserKeyword{"ON"}.ignore(pos, expected)) return false; + const bool grant_named_collection_access = named_collection_access && named_collection_access == access_and_columns.size(); if (grant_named_collection_access) { ASTPtr collection; diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 58b1cab6e20..518b1a84abb 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -39,7 +39,7 @@ ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER DATABASE SETTINGS ['ALTER DATABASE SETTING','ALTER MODIFY DATABASE SETTING','MODIFY DATABASE SETTING'] DATABASE ALTER DATABASE -ALTER NAMED COLLECTION [] \N ALTER +ALTER NAMED COLLECTION [] NAMED_COLLECTION ALTER ALTER TABLE [] \N ALTER ALTER DATABASE [] \N ALTER ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW @@ -52,14 +52,14 @@ CREATE VIEW [] VIEW CREATE CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE -CREATE NAMED COLLECTION [] GLOBAL CREATE +CREATE NAMED COLLECTION [] NAMED_COLLECTION CREATE CREATE [] \N ALL DROP DATABASE [] DATABASE DROP DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP -DROP NAMED COLLECTION [] GLOBAL DROP +DROP NAMED COLLECTION [] NAMED_COLLECTION DROP DROP [] \N ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL @@ -89,7 +89,7 @@ SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY' SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT -SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] GLOBAL ACCESS MANAGEMENT +SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] NAMED_COLLECTION ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index dc7cdddf8ec..484243bd523 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -571,7 +571,7 @@ CREATE TABLE system.privileges ( `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157), `aliases` Array(String), - `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), + `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157)) ) ENGINE = SystemPrivileges From 6f985b8ae0b8546dc0022862ae5ad913b83d6615 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 18:49:18 +0100 Subject: [PATCH 0051/1548] Fix unit test, cleanup code --- src/Access/AccessRights.cpp | 12 ++++-------- src/Access/Common/AccessRightsElement.cpp | 6 +++--- src/Access/tests/gtest_access_rights_ops.cpp | 4 ++-- src/Parsers/Access/ASTGrantQuery.cpp | 7 +++++-- 4 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index ca118203541..65363babb4f 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -326,8 +326,8 @@ public: const Node * child = tryGetChild(name); if (child) return child->isGranted(flags_to_check, subnames...); - - return flags.contains(flags_to_check); + else + return flags.contains(flags_to_check); } template @@ -836,10 +836,7 @@ void AccessRights::grant(const AccessFlags & flags, std::string_view database, s void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessRightsElement & element) { grantImpl(element); } -void AccessRights::grant(const AccessRightsElements & elements) -{ - grantImpl(elements); -} +void AccessRights::grant(const AccessRightsElements & elements) { grantImpl(elements); } void AccessRights::grantWithGrantOption(const AccessFlags & flags) { grantImpl(flags); } void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } @@ -928,7 +925,7 @@ void AccessRights::revokeGrantOption(const AccessRightsElements & elements) { re AccessRightsElements AccessRights::getElements() const { -#if 1 +#if 0 logTree(); #endif if (!root) @@ -950,7 +947,6 @@ bool AccessRights::isGrantedImpl(const AccessFlags & flags, const Args &... args { auto helper = [&](const std::unique_ptr & root_node) -> bool { - logTree(); if (!root_node) return flags.isEmpty(); return root_node->isGranted(flags, args...); diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 011db851bc4..70a6b3bea57 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -26,10 +26,10 @@ namespace result += "ON "; if (element.isNamedCollectionAccess()) { - if (!element.any_named_collection) - result += backQuoteIfNeed(element.named_collection); - else + if (element.any_named_collection) result += "*"; + else + result += backQuoteIfNeed(element.named_collection); } else if (element.any_database) { diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index e21ebda2a31..d6f827a02c5 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -48,9 +48,9 @@ TEST(AccessRights, Union) ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, " "GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, " - "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, " + "CREATE DICTIONARY, CREATE NAMED COLLECTION, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, DROP NAMED COLLECTION, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " - "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " + "SHOW ROW POLICIES, SHOW NAMED COLLECTIONS, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index e4aa11967c6..8d66ac991be 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -30,9 +30,12 @@ namespace void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : ""); - if (!element.any_named_collection) + if (element.isNamedCollectionAccess()) { - settings.ostr << backQuoteIfNeed(element.named_collection); + if (element.any_named_collection) + settings.ostr << "*"; + else + settings.ostr << backQuoteIfNeed(element.named_collection); } else if (element.any_database) { From 78c809608cb7b0b8b81bc339062d6535af192880 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 23:37:34 +0100 Subject: [PATCH 0052/1548] Fix --- src/Access/Common/AccessFlags.h | 3 +-- src/Parsers/Access/ParserGrantQuery.cpp | 2 ++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index f475896b6e5..68ff28516e5 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,8 +50,7 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool containsOnly(const AccessFlags & other) const { return flags == other.flags; } - bool isNamedCollectionAccess() const { return containsOnly(AccessFlags::allFlagsGrantableOnNamedCollectionLevel()); } + bool isNamedCollectionAccess() const { return AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index 06660642fbf..efc2e1c3ea5 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -155,7 +155,9 @@ namespace any_database = any_table = true; } else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, any_database, table_name, any_table)) + { return false; + } for (auto & [access_flags, columns] : access_and_columns) { From c79f252a568b12989eb0ab5f17caee949324ba5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 11 Feb 2023 12:14:37 +0100 Subject: [PATCH 0053/1548] Fix test --- src/Access/Common/AccessFlags.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 68ff28516e5..f43e54f3f33 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,7 +50,7 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool isNamedCollectionAccess() const { return AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } + bool isNamedCollectionAccess() const { return !isEmpty() && AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } From 7e0a98cbf190e4bfd0d695285ae14db509e9e5d4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 11 Feb 2023 12:36:25 +0100 Subject: [PATCH 0054/1548] Add test --- .../test_named_collections/test.py | 52 ++++++++++++++++++- 1 file changed, 51 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 0f8999f43dd..d2d6455caec 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -105,7 +105,7 @@ def test_access(cluster): assert int(node.query("select count() from system.named_collections")) > 0 -def test_granular_access(cluster): +def test_granular_access_show_query(cluster): node = cluster.instances["node"] assert 1 == int(node.query("SELECT count() FROM system.named_collections")) assert ( @@ -175,6 +175,56 @@ def test_granular_access(cluster): node.query("DROP NAMED COLLECTION collection2") +def test_granular_access_create_alter_drop_query(cluster): + node = cluster.instances["node"] + node.query("CREATE USER kek") + node.query("GRANT select ON *.* TO kek") + assert 0 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant CREATE NAMED COLLECTION" + in node.query_and_get_error( + "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" + ) + ) + node.query("GRANT create named collection ON collection2 TO kek") + node.query_and_get_error( + "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" + ) + assert 0 == int( + node.query("select count() from system.named_collections", user="kek") + ) + + node.query("GRANT show named collections ON collection2 TO kek") + # assert ( + # "collection2" + # == node.query("select name from system.named_collections", user="kek").strip() + # ) + # assert ( + # "1" + # == node.query( + # "select collection['key1'] from system.named_collections where name = 'collection2'" + # ).strip() + # ) + + # assert ( + # "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + # in node.query_and_get_error( + # "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" + # ) + # ) + # node.query("GRANT alter named collection ON collection2 TO kek") + # node.query("ALTER NAMED COLLECTION collection2 SET key1=2", user="kek") + # assert ( + # "2" + # == node.query( + # "select collection['key1'] from system.named_collections where name = 'collection2'" + # ).strip() + # ) + + def test_config_reload(cluster): node = cluster.instances["node"] assert ( From 7beb84365019617bfeeab5487a3e8dadfa647ca7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 11 Feb 2023 19:57:41 +0100 Subject: [PATCH 0055/1548] Fix --- .../InterpreterAlterNamedCollectionQuery.cpp | 5 +- .../InterpreterCreateNamedCollectionQuery.cpp | 4 +- .../InterpreterDropNamedCollectionQuery.cpp | 5 +- .../test_named_collections/test.py | 82 ++++++++++--------- 4 files changed, 52 insertions(+), 44 deletions(-) diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index 040a8714983..478735c432b 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -12,9 +12,10 @@ namespace DB BlockIO InterpreterAlterNamedCollectionQuery::execute() { auto current_context = getContext(); - current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION); - const auto & query = query_ptr->as(); + + current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION, query.collection_name); + if (!query.cluster.empty()) { DDLQueryOnClusterParams params; diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index 8a1a8d9dde6..bac59998062 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -13,10 +13,10 @@ namespace DB BlockIO InterpreterCreateNamedCollectionQuery::execute() { auto current_context = getContext(); - current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION); - const auto & query = query_ptr->as(); + current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION, query.collection_name); + if (!query.cluster.empty()) { DDLQueryOnClusterParams params; diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index 064a13012a6..cc3444bb4df 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -12,9 +12,10 @@ namespace DB BlockIO InterpreterDropNamedCollectionQuery::execute() { auto current_context = getContext(); - current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION); - const auto & query = query_ptr->as(); + + current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION, query.collection_name); + if (!query.cluster.empty()) { DDLQueryOnClusterParams params; diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index d2d6455caec..ba403d3f48b 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -70,20 +70,15 @@ def replace_in_users_config(node, old, new): ) -def test_access(cluster): +def test_default_access(cluster): node = cluster.instances["node_no_default_access"] - assert ( - "DB::Exception: default: Not enough privileges. To execute this query it's necessary to have grant SHOW NAMED COLLECTIONS ON *.*" - in node.query_and_get_error("select count() from system.named_collections") - ) + assert 0 == int(node.query("select count() from system.named_collections")) node = cluster.instances["node_no_default_access_but_with_access_management"] - assert ( - "DB::Exception: default: Not enough privileges. To execute this query it's necessary to have grant SHOW NAMED COLLECTIONS ON *.*" - in node.query_and_get_error("select count() from system.named_collections") - ) + assert 0 == int(node.query("select count() from system.named_collections")) node = cluster.instances["node"] assert int(node.query("select count() from system.named_collections")) > 0 + replace_in_users_config( node, "show_named_collections>1", "show_named_collections>0" ) @@ -91,10 +86,8 @@ def test_access(cluster): ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() - assert ( - "DB::Exception: default: Not enough privileges. To execute this query it's necessary to have grant SHOW NAMED COLLECTIONS ON *.*" - in node.query_and_get_error("select count() from system.named_collections") - ) + assert 0 == int(node.query("select count() from system.named_collections")) + replace_in_users_config( node, "show_named_collections>0", "show_named_collections>1" ) @@ -112,6 +105,7 @@ def test_granular_access_show_query(cluster): "collection1" == node.query("SELECT name FROM system.named_collections").strip() ) + node.query("DROP USER IF EXISTS kek") node.query("CREATE USER kek") node.query("GRANT select ON *.* TO kek") assert 0 == int( @@ -156,6 +150,7 @@ def test_granular_access_show_query(cluster): == node.query("select name from system.named_collections", user="kek").strip() ) + node.query("DROP USER IF EXISTS koko") node.query("CREATE USER koko") node.query("GRANT select ON *.* TO koko") assert 0 == int( @@ -177,6 +172,7 @@ def test_granular_access_show_query(cluster): def test_granular_access_create_alter_drop_query(cluster): node = cluster.instances["node"] + node.query("DROP USER IF EXISTS kek") node.query("CREATE USER kek") node.query("GRANT select ON *.* TO kek") assert 0 == int( @@ -190,7 +186,7 @@ def test_granular_access_create_alter_drop_query(cluster): ) ) node.query("GRANT create named collection ON collection2 TO kek") - node.query_and_get_error( + node.query( "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" ) assert 0 == int( @@ -198,31 +194,41 @@ def test_granular_access_create_alter_drop_query(cluster): ) node.query("GRANT show named collections ON collection2 TO kek") - # assert ( - # "collection2" - # == node.query("select name from system.named_collections", user="kek").strip() - # ) - # assert ( - # "1" - # == node.query( - # "select collection['key1'] from system.named_collections where name = 'collection2'" - # ).strip() - # ) + assert ( + "collection2" + == node.query("select name from system.named_collections", user="kek").strip() + ) + assert ( + "1" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) - # assert ( - # "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" - # in node.query_and_get_error( - # "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" - # ) - # ) - # node.query("GRANT alter named collection ON collection2 TO kek") - # node.query("ALTER NAMED COLLECTION collection2 SET key1=2", user="kek") - # assert ( - # "2" - # == node.query( - # "select collection['key1'] from system.named_collections where name = 'collection2'" - # ).strip() - # ) + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + in node.query_and_get_error( + "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" + ) + ) + node.query("GRANT alter named collection ON collection2 TO kek") + node.query("ALTER NAMED COLLECTION collection2 SET key1=2", user="kek") + assert ( + "2" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant DROP NAMED COLLECTION" + in node.query_and_get_error("DROP NAMED COLLECTION collection2", user="kek") + ) + node.query("GRANT drop named collection ON collection2 TO kek") + node.query("DROP NAMED COLLECTION collection2", user="kek") + assert 0 == int( + node.query("select count() from system.named_collections", user="kek") + ) def test_config_reload(cluster): From 38ea27489c018771c3773449d77cd0e469c92693 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 13 Feb 2023 01:04:38 +0300 Subject: [PATCH 0056/1548] 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 0057/1548] 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 0058/1548] 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 8cc768fde2d72c6e5d94c11ac3e9f8a45f87ba77 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Mon, 13 Feb 2023 16:23:04 +0000 Subject: [PATCH 0059/1548] Add KeeperMap as disallowed engine for temporary tables --- docs/en/sql-reference/statements/create/table.md | 2 +- docs/ru/sql-reference/statements/create/table.md | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- .../02525_different_engines_in_temporary_tables.sql | 8 ++++++++ 4 files changed, 12 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 463748ab9a4..efa8c420f1a 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -370,7 +370,7 @@ ENGINE = MergeTree ORDER BY x; ClickHouse supports temporary tables which have the following characteristics: - Temporary tables disappear when the session ends, including if the connection is lost. -- A temporary table uses the Memory table engine when engine is not specified and it may use any table engine except for Replicated engines. +- A temporary table uses the Memory table engine when engine is not specified and it may use any table engine except for Replicated and `KeeperMap` engines. - The DB can’t be specified for a temporary table. It is created outside of databases. - Impossible to create a temporary table with distributed DDL query on all cluster servers (by using `ON CLUSTER`): this table exists only in the current session. - If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used. diff --git a/docs/ru/sql-reference/statements/create/table.md b/docs/ru/sql-reference/statements/create/table.md index 558f9472b30..2ba4c94a2f4 100644 --- a/docs/ru/sql-reference/statements/create/table.md +++ b/docs/ru/sql-reference/statements/create/table.md @@ -299,7 +299,7 @@ ENGINE = MergeTree ORDER BY x; ClickHouse поддерживает временные таблицы Ñо Ñледующими характериÑтиками: - Временные таблицы иÑчезают поÑле Ð·Ð°Ð²ÐµÑ€ÑˆÐµÐ½Ð¸Ñ ÑеÑÑии, в том чиÑле при обрыве ÑоединениÑ. -- Ð’Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° иÑпользует движок таблиц Memory когда движок не указан и она может иÑпользовать любой движок таблиц за иÑключением движков Replicated. +- Ð’Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° иÑпользует движок таблиц Memory когда движок не указан и она может иÑпользовать любой движок таблиц за иÑключением движков Replicated и `KeeperMap`. - Ðевозможно указать базу данных Ð´Ð»Ñ Ð²Ñ€ÐµÐ¼ÐµÐ½Ð½Ð¾Ð¹ таблицы. Она ÑоздаетÑÑ Ð²Ð½Ðµ баз данных. - Ðевозможно Ñоздать временную таблицу раÑпределённым DDL запроÑом на вÑех Ñерверах клаÑтера (Ñ Ð¾Ð¿Ñ†Ð¸ÐµÐ¹ `ON CLUSTER`): Ñ‚Ð°ÐºÐ°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° ÑущеÑтвует только в рамках ÑущеÑтвующей ÑеÑÑии. - ЕÑли Ð²Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð° имеет то же имÑ, что и Ð½ÐµÐºÐ¾Ñ‚Ð¾Ñ€Ð°Ñ Ð´Ñ€ÑƒÐ³Ð°Ñ, то, при упоминании в запроÑе без ÑƒÐºÐ°Ð·Ð°Ð½Ð¸Ñ Ð‘Ð”, будет иÑпользована Ð²Ñ€ÐµÐ¼ÐµÐ½Ð½Ð°Ñ Ñ‚Ð°Ð±Ð»Ð¸Ñ†Ð°. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b479bc96082..9f75d9c7835 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -936,8 +936,8 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.storage->engine) { - if (create.storage->engine->name.starts_with("Replicated")) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated table engines"); + if (create.storage->engine->name.starts_with("Replicated") || create.storage->engine->name == "KeeperMap") + throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated or KeeperMap table engines"); } else throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table"); diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql index 2913a38865e..8d8381e92c4 100644 --- a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql @@ -43,3 +43,11 @@ ENGINE = TinyLog; INSERT INTO table_tiny_log_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); SELECT * FROM table_tiny_log_02525; DROP TEMPORARY TABLE table_tiny_log_02525; + +DROP TEMPORARY TABLE IF EXISTS table_keeper_map_02525; +CREATE TEMPORARY TABLE table_keeper_map_02525 +( + key String, + value UInt32 +) Engine=KeeperMap('/' || currentDatabase() || '/test02525') +PRIMARY KEY(key); -- { serverError 80 } From d6ea566b20673cc935823a7b7560dc76678f403c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 16:52:21 +0000 Subject: [PATCH 0060/1548] Mostly support minmax projection. --- .../Optimizations/optimizeUseProjections.cpp | 62 ++++++++++++------- 1 file changed, 40 insertions(+), 22 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index ca16bd09649..28bce3dea51 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations @@ -121,8 +122,11 @@ struct AggregateProjectionInfo AggregateProjectionInfo getAggregatingProjectionInfo( const ProjectionDescription & projection, const ContextPtr & context, - const StorageMetadataPtr & metadata_snapshot) + const StorageMetadataPtr & metadata_snapshot, + const Block & key_virtual_columns) { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj query : {}", queryToString(projection.query_ast)); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Sample for keys : {}", projection.sample_block_for_keys.dumpStructure()); /// 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. @@ -141,6 +145,13 @@ AggregateProjectionInfo getAggregatingProjectionInfo( info.keys = query_analyzer->aggregationKeys(); info.aggregates = query_analyzer->aggregates(); + for (const auto & virt_column : key_virtual_columns) + { + const auto * input = &info.before_aggregation->addInput(virt_column); + info.before_aggregation->getOutputs().push_back(input); + info.keys.push_back(NameAndTypePair{virt_column.name, virt_column.type}); + } + return info; } @@ -487,9 +498,18 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) const auto & keys = aggregating->getParams().keys; const auto & aggregates = aggregating->getParams().aggregates; + Block key_virtual_columns = reading->getMergeTreeData().getSampleBlockWithVirtualColumns(); std::vector candidates; std::optional minmax_projection; + Block minmax_count_projection_block; + MergeTreeData::DataPartsVector minmax_projection_normal_parts; + + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + std::shared_ptr max_added_blocks; AggregateProjectionCandidate * best_candidate = nullptr; @@ -497,7 +517,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { const auto * projection = &*(metadata->minmax_count_projection); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); - auto info = getAggregatingProjectionInfo(*projection, context, metadata); + auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { @@ -509,7 +529,20 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) }); } - best_candidate = &*minmax_projection; + minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + minmax_projection->dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + minmax_projection_normal_parts, + max_added_blocks.get(), + context); + + if (!minmax_count_projection_block) + minmax_projection.reset(); + else + best_candidate = &*minmax_projection; } if (!minmax_projection) @@ -519,7 +552,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); - auto info = getAggregatingProjectionInfo(*projection, context, metadata); + auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { @@ -536,12 +569,8 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return false; } - const auto & parts = reading->getParts(); - const auto & query_info = reading->getQueryInfo(); - MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - std::shared_ptr max_added_blocks; if (context->getSettingsRef().select_sequential_consistency) { if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) @@ -609,29 +638,18 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return false; QueryPlanStepPtr projection_reading; - auto query_info_copy = query_info; - query_info_copy.prewhere_info = nullptr; bool has_nornal_parts; if (minmax_projection) { - MergeTreeData::DataPartsVector normal_parts; - auto minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( - metadata, - minmax_projection->dag->getRequiredColumnsNames(), - filter_node != nullptr, - query_info, - parts, - normal_parts, - max_added_blocks.get(), - context); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", minmax_count_projection_block.dumpStructure()); Pipe pipe(std::make_shared(std::move(minmax_count_projection_block))); projection_reading = std::make_unique(std::move(pipe)); - has_nornal_parts = !normal_parts.empty(); + has_nornal_parts = !minmax_projection_normal_parts.empty(); if (has_nornal_parts) - reading->resetParts(std::move(normal_parts)); + reading->resetParts(std::move(minmax_projection_normal_parts)); } else { From e6e486c4a14932db5516aa79a71a3a127fd6af2e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 17:10:10 +0000 Subject: [PATCH 0061/1548] Not crash with minmax projection. --- .../Optimizations/optimizeUseProjections.cpp | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 28bce3dea51..968dfa7631d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -527,22 +527,22 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) .projection = projection, .dag = std::move(proj_dag), }); + + minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + minmax_projection->dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + minmax_projection_normal_parts, + max_added_blocks.get(), + context); + + if (!minmax_count_projection_block) + minmax_projection.reset(); + else + best_candidate = &*minmax_projection; } - - minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( - metadata, - minmax_projection->dag->getRequiredColumnsNames(), - filter_node != nullptr, - query_info, - parts, - minmax_projection_normal_parts, - max_added_blocks.get(), - context); - - if (!minmax_count_projection_block) - minmax_projection.reset(); - else - best_candidate = &*minmax_projection; } if (!minmax_projection) From 522a39f93f3850534f81e40b1677b0c4ed1d20f0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 19:56:31 +0000 Subject: [PATCH 0062/1548] Normal Projections analysis using query plan [In progress] --- .../QueryPlan/Optimizations/Optimizations.h | 3 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 10 +- .../Optimizations/optimizeUseProjections.cpp | 235 +++++++++++++++++- 3 files changed, 245 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 0ee2cecb4df..fb791a4bd09 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -93,7 +93,8 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); +bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index d0d634d931d..f8eb4b34316 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -122,7 +122,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - applied_projection |= optimizeUseProjections(*frame.node, nodes); + applied_projection |= optimizeUseAggProjections(*frame.node, nodes); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -140,6 +140,14 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } + if (optimization_settings.optimize_projection) + { + bool applied = optimizeUseNormalProjections(stack, nodes); + applied_projection |= applied; + if (applied && stack.back().next_child == 0) + continue; + } + optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 968dfa7631d..b16fa317ef7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -167,6 +168,16 @@ struct AggregateProjectionCandidate size_t sum_marks = 0; }; +struct NormalProjectionCandidate +{ + const ProjectionDescription * projection; + + MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; + MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; + + size_t sum_marks = 0; +}; + ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, ActionsDAG & query_dag, @@ -430,7 +441,7 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } -bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) return false; @@ -724,4 +735,226 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return true; } + +bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) +{ + const auto & frame = stack.back(); + + auto * reading = typeid_cast(frame.node->step.get()); + if (!reading) + return false; + + auto iter = stack.rbegin(); + while (iter != stack.rend()) + { + auto next = std::next(iter); + + if (!typeid_cast(next->node->step.get()) && + !typeid_cast(next->node->step.get())) + break; + + iter = next; + } + + if (iter == stack.rbegin()) + return false; + + const auto metadata = reading->getStorageMetadata(); + const auto & projections = metadata->projections; + + std::vector normal_projections; + for (const auto & projection : projections) + if (projection.type == ProjectionDescription::Type::Normal) + normal_projections.push_back(&projection); + + if (normal_projections.empty()) + return false; + + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes)) + return false; + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + + const ActionsDAG::Node * filter_node = nullptr; + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().push_back(filter_node); + } + + std::list candidates; + NormalProjectionCandidate * best_candidate = nullptr; + + const Block & header = frame.node->step->getOutputStream().header; + const Names & required_columns = reading->getRealColumnNames(); + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + ContextPtr context = reading->getContext(); + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + + std::shared_ptr max_added_blocks; + if (context->getSettingsRef().select_sequential_consistency) + { + if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); + } + + for (const auto * projection : normal_projections) + { + bool has_all_columns = true; + for (const auto & col : required_columns) + { + if (!projection->sample_block.has(col)) + { + has_all_columns = false; + break; + } + } + + if (!has_all_columns) + continue; + + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & created_projections = part->getProjectionParts(); + auto it = created_projections.find(projection->name); + if (it != created_projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } + + if (projection_parts.empty()) + continue; + + ActionDAGNodes added_filter_nodes; + if (filter_node) + added_filter_nodes.nodes.push_back(filter_node); + + auto projection_result_ptr = reader.estimateNumMarksToRead( + std::move(projection_parts), + nullptr, + header.getNames(), + metadata, + projection->metadata, + query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes + added_filter_nodes, + context, + context->getSettingsRef().max_threads, + max_added_blocks); + + if (projection_result_ptr->error()) + continue; + + auto & candidate = candidates.emplace_back(); + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); + + if (!normal_parts.empty()) + { + auto normal_result_ptr = reading->selectRangesToRead(std::move(normal_parts)); + + if (normal_result_ptr->error()) + continue; + + if (normal_result_ptr->marks() != 0) + { + candidate.sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); + } + } + + if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) + best_candidate = &candidate; + } + + if (!best_candidate) + return false; + + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + auto projection_reading = reader.readFromParts( + {}, + header.getNames(), + proj_snapshot, + query_info, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(header.getNames()))); + projection_reading = std::make_unique(std::move(pipe)); + } + + bool has_nornal_parts = best_candidate->merge_tree_normal_select_result_ptr != nullptr; + if (has_nornal_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); + + projection_reading->setStepDescription(best_candidate->projection->name); + + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto & expr_or_filter_node = nodes.emplace_back(); + + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + dag, + dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + + if (!has_nornal_parts) + { + /// All parts are taken from projection + iter->node->children.front() = &expr_or_filter_node; + + //optimizeAggregationInOrder(node, nodes); + } + else + { + auto & union_node = nodes.emplace_back(); + DataStreams input_streams = {iter->node->children.front()->step->getOutputStream(), expr_or_filter_node.step->getOutputStream()}; + union_node.step = std::make_unique(std::move(input_streams)); + union_node.children = {iter->node->children.front(), &expr_or_filter_node}; + iter->node->children.front() = &union_node; + + iter->next_child = 0; + stack.resize(iter.base() - stack.begin() + 1); + } + + return true; +} + } From 53298f5a17bc88faffef80b1a47f2920912f5bda Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 14 Feb 2023 11:16:00 +0000 Subject: [PATCH 0063/1548] Add test for ReplicatedMergeTree --- .../02525_different_engines_in_temporary_tables.sql | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql index 8d8381e92c4..1535a190db5 100644 --- a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql @@ -44,6 +44,16 @@ INSERT INTO table_tiny_log_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); SELECT * FROM table_tiny_log_02525; DROP TEMPORARY TABLE table_tiny_log_02525; +DROP TEMPORARY TABLE IF EXISTS table_replicated_merge_tree_02525; +CREATE TEMPORARY TABLE table_replicated_merge_tree_02525 +( + id UInt64, + info String +) +ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02525/table_replicated_merge_tree_02525', 'r1') +ORDER BY id +PRIMARY KEY id; -- { serverError 80 } + DROP TEMPORARY TABLE IF EXISTS table_keeper_map_02525; CREATE TEMPORARY TABLE table_keeper_map_02525 ( From 42f2c537e21a9a9c7fa067e175916af95c327e90 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 14 Feb 2023 12:10:44 +0000 Subject: [PATCH 0064/1548] Remove setting database for create query --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +-- src/Parsers/ASTCreateQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 60662346f68..71812c1ab0e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1268,8 +1268,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.if_not_exists && getContext()->tryResolveStorageID({"", create.getTable()}, Context::ResolveExternal)) return false; - create.setDatabase(DatabaseCatalog::TEMPORARY_DATABASE); - DatabasePtr database = DatabaseCatalog::instance().getDatabase(create.getDatabase()); + DatabasePtr database = DatabaseCatalog::instance().getDatabase(DatabaseCatalog::TEMPORARY_DATABASE); String temporary_table_name = create.getTable(); auto creator = [&](const StorageID & table_id) diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index a5a1b25de8b..e74be6c66c5 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -293,7 +293,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat << what << " " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") - << (database && !temporary ? backQuoteIfNeed(getDatabase()) + "." : "") << backQuoteIfNeed(getTable()); + << (database ? backQuoteIfNeed(getDatabase()) + "." : "") << backQuoteIfNeed(getTable()); if (uuid != UUIDHelpers::Nil) settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") From b0ced21c30043856b75babea261012e9f45d7d81 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 14 Feb 2023 13:22:15 +0000 Subject: [PATCH 0065/1548] Move temp database clean up into new function removeDataPath() --- src/Databases/DatabaseMemory.cpp | 14 ++++++++++++-- src/Databases/DatabaseMemory.h | 2 ++ src/Interpreters/DatabaseCatalog.cpp | 5 ----- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 1aae2b81ccc..3ede69d5362 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -26,7 +26,12 @@ namespace ErrorCodes DatabaseMemory::DatabaseMemory(const String & name_, ContextPtr context_) : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context_) , data_path("data/" + escapeForFileName(database_name) + "/") -{} +{ + /// Temporary database should not have any data on the moment of its creation + /// In case of sudden server shutdown remove database folder of temporary database + if (name_ == DatabaseCatalog::TEMPORARY_DATABASE) + removeDataPath(context_); +} void DatabaseMemory::createTable( ContextPtr /*context*/, @@ -127,10 +132,15 @@ UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const return UUIDHelpers::Nil; } +void DatabaseMemory::removeDataPath(ContextPtr local_context) +{ + std::filesystem::remove_all(local_context->getPath() + data_path); +} + void DatabaseMemory::drop(ContextPtr local_context) { /// Remove data on explicit DROP DATABASE - std::filesystem::remove_all(local_context->getPath() + data_path); + removeDataPath(local_context); } void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index 6262543b0c1..0f703a0b46e 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -53,6 +53,8 @@ public: std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override; private: + void removeDataPath(ContextPtr local_context); + const String data_path; using NameToASTCreate = std::unordered_map; NameToASTCreate create_queries TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index eb5c97bc583..ac16431b293 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -154,11 +154,6 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase() unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, getContext()); - - /// Temporary database should not have any data on the moment of its creation - /// In case of sudden server shutdown remove database folder of temporary database - db_for_temporary_and_external_tables->drop(getContext()); - attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); } From 5ff1f7237a4401fe44304f7cc631f2ac8a050c61 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 14 Feb 2023 21:35:16 +0800 Subject: [PATCH 0066/1548] initial commit of quantileGK --- .../AggregateFunctionQuantile.h | 3 + .../AggregateFunctionQuantileGK.cpp | 70 +++ src/AggregateFunctions/QuantileGK.h | 442 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + 4 files changed, 517 insertions(+) create mode 100644 src/AggregateFunctions/AggregateFunctionQuantileGK.cpp create mode 100644 src/AggregateFunctions/QuantileGK.h diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 49157acf690..ab7edcd082e 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -250,4 +250,7 @@ struct NameQuantilesBFloat16 { static constexpr auto name = "quantilesBFloat16"; struct NameQuantileBFloat16Weighted { static constexpr auto name = "quantileBFloat16Weighted"; }; struct NameQuantilesBFloat16Weighted { static constexpr auto name = "quantilesBFloat16Weighted"; }; +struct NameQuantileGK { static constexpr auto name = "quantileGK"; }; +struct NameQuantilesGK { static constexpr auto name = "quantilesGK"; }; + } diff --git a/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp b/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp new file mode 100644 index 00000000000..54174f3be44 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +template using FuncQuantileGK = AggregateFunctionQuantile, NameQuantileGK, true, void, false>; +template using FuncQuantilesGK = AggregateFunctionQuantile, NameQuantilesGK, true, void, true>; + + +template