From 0a46532005b5be94c85d264de7cb8a21a2a3dee0 Mon Sep 17 00:00:00 2001 From: liyang830 Date: Mon, 14 Mar 2022 18:25:28 +0800 Subject: [PATCH 001/249] add check copier drop partition success --- programs/copier/ClusterCopier.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 4d491a06795..b1cafd4ad6d 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -598,6 +598,8 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ss << "ALTER TABLE " << getQuotedTable(original_table) << ((partition_name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") << partition_name; UInt64 num_shards_drop_partition = executeQueryOnCluster(task_table.cluster_push, ss.str(), task_cluster->settings_push, ClusterExecutionMode::ON_EACH_SHARD); + if (num_shards_drop_partition != task_table.cluster_push->getShardCount()) + return TaskStatus::Error; LOG_INFO(log, "Drop partition {} in original table {} have been executed successfully on {} shards of {}", partition_name, getQuotedTable(original_table), num_shards_drop_partition, task_table.cluster_push->getShardCount()); From 230f79a3229daf11c169c20be909c0ea4f04958e Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 10 May 2023 11:51:25 +0300 Subject: [PATCH 002/249] Fix bug in StorageBuffer::reschedule() --- src/Storages/StorageBuffer.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a4cb15d5711..757d3161067 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -996,8 +996,11 @@ void StorageBuffer::reschedule() std::unique_lock lock(buffer.tryLock()); if (lock.owns_lock()) { - min_first_write_time = buffer.first_write_time; - rows += buffer.data.rows(); + if (buffer.data) + { + min_first_write_time = std::min(min_first_write_time, buffer.first_write_time); + rows += buffer.data.rows(); + } } } From 28fb99cd260dcf7d6c36386364645f19c3ef7ab6 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Mon, 5 Jun 2023 13:12:19 -0700 Subject: [PATCH 003/249] Maintain same aggregate function mergeY behavior for small and big endian machine --- src/AggregateFunctions/UniquesHashSet.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index ca6d31a716d..369fce9374f 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -108,7 +108,13 @@ private: inline size_t buf_size() const { return 1ULL << size_degree; } /// NOLINT inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); } + + inline size_t place(HashValue x) const { + if constexpr (std::endian::native == std::endian::little) + return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + else + return (std::byteswap(x) >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + } /// The value is divided by 2 ^ skip_degree inline bool good(HashValue hash) const From 55c526b6e644c9bab04972711efc2512a64cbfdd Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Wed, 5 Jul 2023 10:20:37 -0400 Subject: [PATCH 004/249] Update src/AggregateFunctions/UniquesHashSet.h Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/AggregateFunctions/UniquesHashSet.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index 369fce9374f..3e501b29414 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -109,7 +109,8 @@ private: inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { + inline size_t place(HashValue x) const + { if constexpr (std::endian::native == std::endian::little) return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); else From 77e45e5669eae8ae05c5acdc5b0cbf6647d1f50a Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 10 Jul 2023 20:23:03 -0400 Subject: [PATCH 005/249] Add `DROP ROLE ... FROM` and `CREATE ROLE ... AT` syntax --- src/Access/AccessControl.cpp | 2 +- src/Access/DiskAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 35 +++++++++ src/Access/MultipleAccessStorage.h | 5 ++ src/Access/UsersConfigAccessStorage.h | 2 +- .../Access/InterpreterCreateRoleQuery.cpp | 42 +++++++++-- .../InterpreterDropAccessEntityQuery.cpp | 18 +++-- src/Parsers/Access/ASTCreateRoleQuery.cpp | 6 ++ src/Parsers/Access/ASTCreateRoleQuery.h | 1 + .../Access/ASTDropAccessEntityQuery.cpp | 5 ++ src/Parsers/Access/ASTDropAccessEntityQuery.h | 1 + src/Parsers/Access/ParserCreateRoleQuery.cpp | 5 ++ .../Access/ParserDropAccessEntityQuery.cpp | 5 ++ src/Parsers/Access/parseUserName.h | 5 ++ .../System/StorageSystemRoleGrants.cpp | 7 +- .../__init__.py | 0 .../configs/roles.xml | 13 ++++ .../test.py | 72 +++++++++++++++++++ 18 files changed, 211 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_multi_access_storage_role_management/__init__.py create mode 100644 tests/integration/test_multi_access_storage_role_management/configs/roles.xml create mode 100644 tests/integration/test_multi_access_storage_role_management/test.py diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 6179c823b56..0c61a9e3e1b 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -415,7 +415,7 @@ void AccessControl::addStoragesFromUserDirectoriesConfig( String type = key_in_user_directories; if (size_t bracket_pos = type.find('['); bracket_pos != String::npos) type.resize(bracket_pos); - if ((type == "users_xml") || (type == "users_config")) + if ((type == "users.xml") || (type == "users_config")) type = UsersConfigAccessStorage::STORAGE_TYPE; else if ((type == "local") || (type == "local_directory")) type = DiskAccessStorage::STORAGE_TYPE; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 069a966c8e9..e3bd32c0292 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -13,7 +13,7 @@ class AccessChangesNotifier; class DiskAccessStorage : public IAccessStorage { public: - static constexpr char STORAGE_TYPE[] = "local directory"; + static constexpr char STORAGE_TYPE[] = "local_directory"; DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_); ~DiskAccessStorage() override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36a670ddbf..f36beb3f80b 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes { extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND; + extern const int ACCESS_ENTITY_NOT_FOUND; } using Storage = IAccessStorage; @@ -178,6 +179,40 @@ ConstStoragePtr MultipleAccessStorage::getStorage(const UUID & id) const return const_cast(this)->getStorage(id); } +StoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + if (storage->getStorageName() == storage_name) + return storage; + } + + return nullptr; +} + + +ConstStoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) const +{ + return const_cast(this)->findStorageByName(storage_name); +} + + +StoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) +{ + auto storage = findStorageByName(storage_name); + if (storage) + return storage; + + throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "Access storage with name {} is not found", storage_name); +} + + +ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) const +{ + return const_cast(this)->getStorageByName(storage_name); +} + AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { if (auto storage = findStorage(id)) diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 6a0c1bdfc02..1898c6ba5d3 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -41,6 +41,11 @@ public: ConstStoragePtr getStorage(const UUID & id) const; StoragePtr getStorage(const UUID & id); + ConstStoragePtr findStorageByName(const String & storage_name) const; + StoragePtr findStorageByName(const String & storage_name); + ConstStoragePtr getStorageByName(const String & storage_name) const; + StoragePtr getStorageByName(const String & storage_name); + bool exists(const UUID & id) const override; bool isBackupAllowed() const override; diff --git a/src/Access/UsersConfigAccessStorage.h b/src/Access/UsersConfigAccessStorage.h index b533ccbf200..d8ddf07ff40 100644 --- a/src/Access/UsersConfigAccessStorage.h +++ b/src/Access/UsersConfigAccessStorage.h @@ -20,7 +20,7 @@ class UsersConfigAccessStorage : public IAccessStorage { public: - static constexpr char STORAGE_TYPE[] = "users.xml"; + static constexpr char STORAGE_TYPE[] = "users_xml"; UsersConfigAccessStorage(const String & storage_name_, AccessControl & access_control_, bool allow_backup_); ~UsersConfigAccessStorage() override; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 3386dfb8792..87abe002d3d 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -8,6 +8,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateRoleFromQueryImpl( @@ -52,6 +58,15 @@ BlockIO InterpreterCreateRoleQuery::execute() if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext()); + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -62,11 +77,11 @@ BlockIO InterpreterCreateRoleQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -78,12 +93,27 @@ BlockIO InterpreterCreateRoleQuery::execute() new_roles.emplace_back(std::move(new_role)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + auto id = access_control.find(name); + + if (!id) + continue; + + auto another_storage_ptr = access_control.findStorage(*id); + if (another_storage_ptr != storage_ptr) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Role {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_roles); + storage->tryInsert(new_roles); else if (query.or_replace) - access_control.insertOrReplace(new_roles); + storage->insertOrReplace(new_roles); else - access_control.insert(new_roles); + storage->insert(new_roles); } return {}; diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index d4c37064065..54e3b95226c 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -26,18 +26,26 @@ BlockIO InterpreterDropAccessEntityQuery::execute() query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); - auto do_drop = [&](const Strings & names) + auto do_drop = [&](const Strings & names, const String & storage_name) { + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + if (!storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(storage_name); + storage = storage_ptr.get(); + } + if (query.if_exists) - access_control.tryRemove(access_control.find(query.type, names)); + storage->tryRemove(storage->find(query.type, names)); else - access_control.remove(access_control.getIDs(query.type, names)); + storage->remove(storage->getIDs(query.type, names)); }; if (query.type == AccessEntityType::ROW_POLICY) - do_drop(query.row_policy_names->toStrings()); + do_drop(query.row_policy_names->toStrings(), query.storage_name); else - do_drop(query.names); + do_drop(query.names, query.storage_name); return {}; } diff --git a/src/Parsers/Access/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp index d624b9a9157..14946f2d3cd 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -71,6 +71,12 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); formatNames(names, format); + + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " AT " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateRoleQuery.h b/src/Parsers/Access/ASTCreateRoleQuery.h index 42d1a4031b6..4e465553164 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.h +++ b/src/Parsers/Access/ASTCreateRoleQuery.h @@ -28,6 +28,7 @@ public: Strings names; String new_name; + String storage_name; std::shared_ptr settings; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 88f2d7bce63..02b29f3fd2c 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -53,6 +53,11 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma else formatNames(names, settings); + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " FROM " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); } diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h index f3a065c50df..32f4a8f8047 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -21,6 +21,7 @@ public: AccessEntityType type; bool if_exists = false; Strings names; + String storage_name; std::shared_ptr row_policy_names; String getID(char) const override; diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index da9749958ee..de9e5baed18 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -91,6 +91,10 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String new_name; std::shared_ptr settings; String cluster; + String storage_name; + + if (ParserKeyword{"AT"}.ignore(pos, expected)) + parseStorageName(pos, expected, storage_name); while (true) { @@ -125,6 +129,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->names = std::move(names); query->new_name = std::move(new_name); query->settings = std::move(settings); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d21164bc1a2..d5ecdf27449 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -53,6 +53,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & Strings names; std::shared_ptr row_policy_names; + String storage_name; String cluster; if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) @@ -76,6 +77,9 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } + if (ParserKeyword{"FROM"}.ignore(pos, expected)) + parseStorageName(pos, expected, storage_name); + if (cluster.empty()) parseOnCluster(pos, expected, cluster); @@ -87,6 +91,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->cluster = std::move(cluster); query->names = std::move(names); query->row_policy_names = std::move(row_policy_names); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index fbad78e7bce..717911bf2f1 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -34,4 +34,9 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro return parseUserNames(pos, expected, role_names); } +inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) +{ + return parseUserName(pos, expected, storage_name); +} + } diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index cf5a24f88cd..8fcd0fd7cf5 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes() {"user_name", std::make_shared(std::make_shared())}, {"role_name", std::make_shared(std::make_shared())}, {"granted_role_name", std::make_shared()}, + {"granted_role_id", std::make_shared()}, {"granted_role_is_default", std::make_shared()}, {"with_admin_option", std::make_shared()}, }; @@ -45,12 +47,14 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr auto & column_role_name = assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()); auto & column_role_name_null_map = assert_cast(*res_columns[column_index++]).getNullMapData(); auto & column_granted_role_name = assert_cast(*res_columns[column_index++]); + auto & column_granted_role_id = assert_cast(*res_columns[column_index++]).getData(); auto & column_is_default = assert_cast(*res_columns[column_index++]).getData(); auto & column_admin_option = assert_cast(*res_columns[column_index++]).getData(); auto add_row = [&](const String & grantee_name, AccessEntityType grantee_type, const String & granted_role_name, + const UUID & granted_role_id, bool is_default, bool with_admin_option) { @@ -72,6 +76,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr assert(false); column_granted_role_name.insertData(granted_role_name.data(), granted_role_name.length()); + column_granted_role_id.push_back(granted_role_id.toUnderType()); column_is_default.push_back(is_default); column_admin_option.push_back(with_admin_option); }; @@ -90,7 +95,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr continue; bool is_default = !default_roles || default_roles->match(role_id); - add_row(grantee_name, grantee_type, *role_name, is_default, element.admin_option); + add_row(grantee_name, grantee_type, *role_name, role_id, is_default, element.admin_option); } } }; diff --git a/tests/integration/test_multi_access_storage_role_management/__init__.py b/tests/integration/test_multi_access_storage_role_management/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml new file mode 100644 index 00000000000..672c7e24728 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml @@ -0,0 +1,13 @@ + + + + + + + + + GRANT ALL ON *.* WITH GRANT OPTION + + + + diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py new file mode 100644 index 00000000000..473a37695e9 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -0,0 +1,72 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + stay_alive=True, +) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + node.query("CREATE DATABASE mydb") + node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log") + + node.query("CREATE USER test_user") + node.query("CREATE USER test_user2") + yield cluster + + finally: + cluster.shutdown() + + +def test_role_from_different_storages(): + node.query("CREATE ROLE default_role") + node.query("GRANT SELECT ON system.* TO default_role") + + assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT SELECT ON system.* TO default_role"]) + assert node.query("SHOW ROLES") == TSV(["default_role"]) + + node.query("GRANT default_role TO test_user") + + node.copy_file_to_container( + os.path.join(SCRIPT_DIR, "configs/roles.xml"), + "/etc/clickhouse-server/users.d/roles.xml", + ) + + node.restart_clickhouse() + + assert node.query("SELECT name, storage FROM system.roles") == TSV( + [ + ["default_role", "users_xml"], + ["default_role", "local_directory"] + ] + ) + + # Role from users.xml will have priority + assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"]) + + node.query("GRANT default_role TO test_user") + node.query("GRANT default_role TO test_user2") + assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'") == TSV( + ["62bedbf3-7fb1-94cb-3a35-e479693223b3"] # roles from users.xml have deterministic ids + ) + + node.query("DROP ROLE default_role FROM local_directory") + assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'") == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) + + # Already exists + with pytest.raises(QueryRuntimeException): + node.query("CREATE ROLE default_role AT memory") + + node.query("CREATE ROLE other_role AT memory") + + assert node.query("SELECT storage FROM system.roles WHERE name = 'other_role'") == TSV(["memory"]) From ee591ef7f52f3f53dd8cb59a6acf33a734561f65 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 11 Jul 2023 00:33:34 +0000 Subject: [PATCH 006/249] Automatic style fix --- .../test.py | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index 473a37695e9..d9a4a73c56b 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -32,7 +32,9 @@ def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") - assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT SELECT ON system.* TO default_role"]) + assert node.query("SHOW GRANTS FOR default_role") == TSV( + ["GRANT SELECT ON system.* TO default_role"] + ) assert node.query("SHOW ROLES") == TSV(["default_role"]) node.query("GRANT default_role TO test_user") @@ -45,23 +47,28 @@ def test_role_from_different_storages(): node.restart_clickhouse() assert node.query("SELECT name, storage FROM system.roles") == TSV( - [ - ["default_role", "users_xml"], - ["default_role", "local_directory"] - ] + [["default_role", "users_xml"], ["default_role", "local_directory"]] ) # Role from users.xml will have priority - assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"]) + assert node.query("SHOW GRANTS FOR default_role") == TSV( + ["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"] + ) node.query("GRANT default_role TO test_user") node.query("GRANT default_role TO test_user2") - assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'") == TSV( - ["62bedbf3-7fb1-94cb-3a35-e479693223b3"] # roles from users.xml have deterministic ids + assert node.query( + "SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'" + ) == TSV( + [ + "62bedbf3-7fb1-94cb-3a35-e479693223b3" + ] # roles from users.xml have deterministic ids ) node.query("DROP ROLE default_role FROM local_directory") - assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'") == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) + assert node.query( + "SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'" + ) == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) # Already exists with pytest.raises(QueryRuntimeException): @@ -69,4 +76,6 @@ def test_role_from_different_storages(): node.query("CREATE ROLE other_role AT memory") - assert node.query("SELECT storage FROM system.roles WHERE name = 'other_role'") == TSV(["memory"]) + assert node.query( + "SELECT storage FROM system.roles WHERE name = 'other_role'" + ) == TSV(["memory"]) From 0e9f0e6837cbbb9c7fa784fc9139451ce9779c57 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 11 Jul 2023 11:09:01 -0700 Subject: [PATCH 007/249] Fix ForEach Aggregate state for s390x --- src/AggregateFunctions/AggregateFunctionForEach.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 480b4cc690e..ec59fd2e616 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -240,7 +240,7 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const AggregateFunctionForEachData & state = data(place); - writeBinary(state.dynamic_array_size, buf); + writeBinaryLittleEndian(state.dynamic_array_size, buf); const char * nested_state = state.array_of_aggregate_datas; for (size_t i = 0; i < state.dynamic_array_size; ++i) @@ -255,7 +255,7 @@ public: AggregateFunctionForEachData & state = data(place); size_t new_size = 0; - readBinary(new_size, buf); + readBinaryLittleEndian(new_size, buf); ensureAggregateData(place, new_size, *arena); From 1a4d48324e3ffe1f8c0c05a337b590ce8aa9ff33 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 11 Jul 2023 19:58:28 -0400 Subject: [PATCH 008/249] Fix fast tests --- .../queries/0_stateless/01293_create_role.reference | 2 +- .../01294_create_settings_profile.reference | 12 ++++++------ .../0_stateless/01295_create_row_policy.reference | 6 +++--- .../queries/0_stateless/01297_create_quota.reference | 8 ++++---- .../02117_show_create_table_system.reference | 1 + 5 files changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01293_create_role.reference b/tests/queries/0_stateless/01293_create_role.reference index 9b3c4eabd47..42f091bddd5 100644 --- a/tests/queries/0_stateless/01293_create_role.reference +++ b/tests/queries/0_stateless/01293_create_role.reference @@ -28,7 +28,7 @@ CREATE ROLE r2_01293 CREATE ROLE r1_01293 SETTINGS readonly = 1 CREATE ROLE r2_01293 SETTINGS readonly = 1 -- system.roles -r1_01293 local directory +r1_01293 local_directory -- system.settings_profile_elements \N \N r1_01293 0 readonly 1 \N \N \N \N \N \N r2_01293 0 \N \N \N \N \N default diff --git a/tests/queries/0_stateless/01294_create_settings_profile.reference b/tests/queries/0_stateless/01294_create_settings_profile.reference index dbb73bca851..a10d5758752 100644 --- a/tests/queries/0_stateless/01294_create_settings_profile.reference +++ b/tests/queries/0_stateless/01294_create_settings_profile.reference @@ -53,12 +53,12 @@ CREATE SETTINGS PROFILE s4_01294 SETTINGS INHERIT readonly, INHERIT readonly CREATE SETTINGS PROFILE s5_01294 SETTINGS INHERIT readonly, readonly = 1 CREATE SETTINGS PROFILE s6_01294 SETTINGS INHERIT readonly, readonly CONST -- system.settings_profiles -s1_01294 local directory 0 0 [] [] -s2_01294 local directory 1 0 ['r1_01294'] [] -s3_01294 local directory 1 0 ['r1_01294'] [] -s4_01294 local directory 1 0 ['r1_01294'] [] -s5_01294 local directory 3 0 ['u1_01294'] [] -s6_01294 local directory 0 1 [] ['r1_01294','u1_01294'] +s1_01294 local_directory 0 0 [] [] +s2_01294 local_directory 1 0 ['r1_01294'] [] +s3_01294 local_directory 1 0 ['r1_01294'] [] +s4_01294 local_directory 1 0 ['r1_01294'] [] +s5_01294 local_directory 3 0 ['u1_01294'] [] +s6_01294 local_directory 0 1 [] ['r1_01294','u1_01294'] -- system.settings_profile_elements s2_01294 \N \N 0 readonly 0 \N \N \N \N s3_01294 \N \N 0 max_memory_usage 5000000 4000000 6000000 CONST \N diff --git a/tests/queries/0_stateless/01295_create_row_policy.reference b/tests/queries/0_stateless/01295_create_row_policy.reference index 6e3169b7fec..d73d9752bc1 100644 --- a/tests/queries/0_stateless/01295_create_row_policy.reference +++ b/tests/queries/0_stateless/01295_create_row_policy.reference @@ -30,6 +30,6 @@ CREATE ROW POLICY p5_01295 ON db2.table2 FOR SELECT USING a = b CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 TO ALL CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 TO ALL -- system.row_policies -p1_01295 ON db.table p1_01295 db table local directory (a < b) AND (c > d) 0 0 [] [] -p2_01295 ON db.table p2_01295 db table local directory id = currentUser() 1 0 ['u1_01295'] [] -p3_01295 ON db.table p3_01295 db table local directory 1 0 1 [] ['r1_01295'] +p1_01295 ON db.table p1_01295 db table local_directory (a < b) AND (c > d) 0 0 [] [] +p2_01295 ON db.table p2_01295 db table local_directory id = currentUser() 1 0 ['u1_01295'] [] +p3_01295 ON db.table p3_01295 db table local_directory 1 0 1 [] ['r1_01295'] diff --git a/tests/queries/0_stateless/01297_create_quota.reference b/tests/queries/0_stateless/01297_create_quota.reference index 6b844e0d2f6..308bbf79024 100644 --- a/tests/queries/0_stateless/01297_create_quota.reference +++ b/tests/queries/0_stateless/01297_create_quota.reference @@ -52,10 +52,10 @@ CREATE QUOTA q2_01297 FOR INTERVAL 1 day MAX errors = 5 CREATE QUOTA q1_01297 FOR INTERVAL 1 day TRACKING ONLY TO r1_01297 CREATE QUOTA q2_01297 FOR INTERVAL 1 day TRACKING ONLY TO r1_01297 -- system.quotas -q1_01297 local directory ['user_name'] [] 0 ['r1_01297'] [] -q2_01297 local directory [] [5259492] 0 ['r1_01297','u1_01297'] [] -q3_01297 local directory ['client_key','user_name'] [5259492,15778476] 0 [] [] -q4_01297 local directory [] [604800] 1 [] ['u1_01297'] +q1_01297 local_directory ['user_name'] [] 0 ['r1_01297'] [] +q2_01297 local_directory [] [5259492] 0 ['r1_01297','u1_01297'] [] +q3_01297 local_directory ['client_key','user_name'] [5259492,15778476] 0 [] [] +q4_01297 local_directory [] [604800] 1 [] ['u1_01297'] -- system.quota_limits q2_01297 5259492 0 100 \N \N 11 1000 10000 1001 10001 2.5 \N q3_01297 5259492 0 \N \N \N \N 1002 \N \N \N \N \N 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 3834b05601f..aaff6d92d32 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -955,6 +955,7 @@ CREATE TABLE system.role_grants `user_name` Nullable(String), `role_name` Nullable(String), `granted_role_name` String, + `granted_role_id` UUID, `granted_role_is_default` UInt8, `with_admin_option` UInt8 ) From 8e3bd222f00324667a1d41fb62f50adc91045b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:42:32 +0200 Subject: [PATCH 009/249] Add SQLTest --- .github/workflows/pull_request.yml | 36 +++++++ docker/images.json | 7 +- docker/test/sqltest/Dockerfile | 29 ++++++ docker/test/sqltest/run.sh | 41 ++++++++ docker/test/sqltest/test.py | 118 ++++++++++++++++++++++ tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/sqltest.py | 157 +++++++++++++++++++++++++++++ tests/ci/tests/docker_images.json | 4 + 8 files changed, 391 insertions(+), 3 deletions(-) create mode 100644 docker/test/sqltest/Dockerfile create mode 100755 docker/test/sqltest/run.sh create mode 100644 docker/test/sqltest/test.py create mode 100644 tests/ci/sqltest.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index dd834959578..b3283c3afbe 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5182,3 +5182,39 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### SQL TEST ############################################### +############################################################################################## + SQLTest: + needs: [BuilderDebRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/sqltest + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME="SQLTest" + REPO_COPY=${{runner.temp}}/sqltest/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: SQLTest + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 sqltest.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" diff --git a/docker/images.json b/docker/images.json index e8fc329a640..663fa21a6c9 100644 --- a/docker/images.json +++ b/docker/images.json @@ -155,11 +155,14 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [ - ] + "dependent": [] }, "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile new file mode 100644 index 00000000000..c58907a98a8 --- /dev/null +++ b/docker/test/sqltest/Dockerfile @@ -0,0 +1,29 @@ +# docker build -t clickhouse/sqltest . +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +RUN apt-get update --yes \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get install --yes --no-install-recommends \ + wget \ + git \ + python3 \ + python3-dev \ + python3-pip \ + sudo \ + && apt-get clean + +RUN pip3 install \ + pyyaml \ + clickhouse-driver + +ARG sqltest_repo="https://github.com/elliotchance/sqltest/" + +RUN git clone ${sqltest_repo} + +ENV TZ=UTC +ENV MAX_RUN_TIME=900 +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY run.sh / +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh new file mode 100755 index 00000000000..ef9ecaec33b --- /dev/null +++ b/docker/test/sqltest/run.sh @@ -0,0 +1,41 @@ +#!/bin/bash + +set -x +set -e +set -u +set -o pipefail + +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-16_debug_none_unsplitted_disable_False_binary"} +BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} + +function wget_with_retry +{ + for _ in 1 2 3 4; do + if wget -nv -nd -c "$1";then + return 0 + else + sleep 0.5 + fi + done + return 1 +} + +wget_with_retry "$BINARY_URL_TO_DOWNLOAD" +chmod +x clickhouse +./clickhouse install --noninteractive +clickhouse start + +# Wait for start +for _ in {1..100} +do + clickhouse-client --query "SELECT 1" && break ||: + sleep 1 +done + +# Run the test +pushd sqltest/standards/2016/ +./test.py +popd + +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py new file mode 100644 index 00000000000..7b38fdc52ad --- /dev/null +++ b/docker/test/sqltest/test.py @@ -0,0 +1,118 @@ +#!/usr/bin/env python3 + +import os +import yaml +import random +import string +from clickhouse_driver import Client + + +client = Client(host = 'localhost', port = 9000) +settings = {'default_table_engine': 'Memory', 'union_default_mode': 'DISTINCT', 'calculate_text_stack_trace': 0} + +database_name = 'sqltest_' + ''.join(random.choice(string.ascii_lowercase) for _ in range(10)) + +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings = settings) +client.execute(f"CREATE DATABASE {database_name}", settings = settings) + +client = Client(host = 'localhost', port = 9000, database = database_name) + +summary = {'success': 0, 'total': 0, 'results': {}} + +log_file = open("test.log", "w") +report_html_file = open("report.html", "w") + +with open('features.yml', 'r') as file: + yaml_content = yaml.safe_load(file) + + for category in yaml_content: + log_file.write(category.capitalize() + " features:") + summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + + for test in yaml_content[category]: + log_file.write(test + ": " + yaml_content[category][test]) + summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} + + test_path = test[0] + "/" + test + ".tests.yml" + if os.path.exists(test_path): + with open(test_path, 'r') as test_file: + test_yaml_content = yaml.load_all(test_file, Loader = yaml.FullLoader) + + for test_case in test_yaml_content: + + queries = test_case['sql']; + if not isinstance(queries, list): + queries = [queries] + + for query in queries: + # Example: E011-01 + test_group = '' + if '-' in test: + test_group = test.split("-", 1)[0] + summary['results'][category]['results'][test_group]['total'] += 1 + summary['results'][category]['results'][test]['total'] += 1 + summary['results'][category]['total'] += 1 + summary['total'] += 1 + + log_file.write(query) + + try: + result = client.execute(query, settings = settings) + log_file.write(result) + + if test_group: + summary['results'][category]['results'][test_group]['success'] += 1 + summary['results'][category]['results'][test]['success'] += 1 + summary['results'][category]['success'] += 1 + summary['success'] += 1 + + except Exception as e: + log_file.write(f"Error occurred: {str(e)}") + +client.execute(f"DROP DATABASE {database_name}", settings = settings) + +def enable_color(ratio): + if ratio == 0: + return "" + elif ratio < 0.5: + return "" + elif ratio < 1: + return "" + else: + return "" + +reset_color = "" + +def print_ratio(indent, name, success, total, description): + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + ' ' * indent, + name, + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - {description}" if description else '')) + + +report_html_file.write("

") + +print_ratio(0, 'Total', summary['success'], summary['total'], '') + +for category in summary['results']: + cat_summary = summary['results'][category] + + if cat_summary['total'] == 0: + continue + + print_ratio(2, category, cat_summary['success'], cat_summary['total'], '') + + for test in summary['results'][category]['results']: + test_summary = summary['results'][category]['results'][test] + + if test_summary['total'] == 0: + continue + + print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description']) + +report_html_file.write("

") diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 514aaf7e2ac..74b875c23be 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -79,7 +79,7 @@ def main(): build_url = url break else: - raise Exception("Cannot binary clickhouse among build results") + raise Exception("Cannot find the clickhouse binary among build results") logging.info("Got build url %s", build_url) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py new file mode 100644 index 00000000000..5c20cc4849b --- /dev/null +++ b/tests/ci/sqltest.py @@ -0,0 +1,157 @@ +#!/usr/bin/env python3 + +import logging +import subprocess +import os +import sys + +from github import Github + +from build_download_helper import get_build_name_for_check, read_build_urls +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from commit_status_helper import ( + RerunHelper, + format_description, + get_commit, + post_commit_status, +) +from docker_pull_helper import get_image_with_version +from env_helper import ( + GITHUB_RUN_URL, + REPORTS_PATH, + TEMP_PATH, +) +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from report import TestResult +from s3_helper import S3Helper +from stopwatch import Stopwatch + +IMAGE_NAME = "clickhouse/sqltest" + + +def get_run_command(pr_number, sha, download_url, workspace_path, image): + return ( + f"docker run " + # For sysctl + "--privileged " + "--network=host " + f"--volume={workspace_path}:/workspace " + "--cap-add syslog --cap-add sys_admin --cap-add=SYS_PTRACE " + f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD="{download_url}" ' + f"{image}" + ) + + +def main(): + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + temp_path = TEMP_PATH + reports_path = REPORTS_PATH + + check_name = sys.argv[1] + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + pr_info = PRInfo() + + gh = Github(get_best_robot_token(), per_page=100) + commit = get_commit(gh, pr_info.sha) + + rerun_helper = RerunHelper(commit, check_name) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + + docker_image = get_image_with_version(reports_path, IMAGE_NAME) + + build_name = get_build_name_for_check(check_name) + print(build_name) + urls = read_build_urls(build_name, reports_path) + if not urls: + raise Exception("No build URLs found") + + for url in urls: + if url.endswith("/clickhouse"): + build_url = url + break + else: + raise Exception("Cannot find the clickhouse binary among build results") + + logging.info("Got build url %s", build_url) + + workspace_path = os.path.join(temp_path, "workspace") + if not os.path.exists(workspace_path): + os.makedirs(workspace_path) + + run_command = get_run_command( + pr_info.number, pr_info.sha, build_url, workspace_path, docker_image + ) + logging.info("Going to run %s", run_command) + + run_log_path = os.path.join(temp_path, "run.log") + with open(run_log_path, "w", encoding="utf-8") as log: + with subprocess.Popen( + run_command, shell=True, stderr=log, stdout=log + ) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + + check_name_lower = ( + check_name.lower().replace("(", "").replace(")", "").replace(" ", "") + ) + s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" + paths = { + "run.log": run_log_path, + "main.log": os.path.join(workspace_path, "main.log"), + "server.log.zst": os.path.join(workspace_path, "server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), + "report.html": os.path.join(workspace_path, "report.html"), + "test.log": os.path.join(workspace_path, "test.log"), + } + + s3_helper = S3Helper() + for f in paths: + try: + paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f) + except Exception as ex: + logging.info("Exception uploading file %s text %s", f, ex) + paths[f] = "" + + report_url = GITHUB_RUN_URL + if paths["report.html"]: + report_url = paths["report.html"] + + status = "success" + description = "See the report" + test_result = TestResult(description, "OK") + + ch_helper = ClickHouseHelper() + + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [test_result], + status, + stopwatch.duration_seconds, + stopwatch.start_time_str, + report_url, + check_name, + ) + + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + logging.info("Result: '%s', '%s', '%s'", status, description, report_url) + print(f"::notice ::Report url: {report_url}") + post_commit_status(commit, status, report_url, description, check_name, pr_info) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0d40d43c33f..0637058e184 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -153,5 +153,9 @@ "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } From df5abcf685a141d163e4aefc213521fc64b613ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:45:43 +0200 Subject: [PATCH 010/249] Update Docker --- docker/test/sqltest/Dockerfile | 1 + docker/test/sqltest/test.py | 0 2 files changed, 1 insertion(+) mode change 100644 => 100755 docker/test/sqltest/test.py diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile index c58907a98a8..437677f4fd1 100644 --- a/docker/test/sqltest/Dockerfile +++ b/docker/test/sqltest/Dockerfile @@ -26,4 +26,5 @@ ENV MAX_RUN_TIME=900 RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone COPY run.sh / +COPY test.py / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py old mode 100644 new mode 100755 From 174166b23382488f1a6fc12cbf121b1a1da7f49b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:47:41 +0200 Subject: [PATCH 011/249] Fix error --- docker/test/sqltest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index ef9ecaec33b..b26d4d89f8b 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -34,7 +34,7 @@ done # Run the test pushd sqltest/standards/2016/ -./test.py +/test.py popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log From fcf4a6ea887f957b8d911b1611932ecc59260b7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:48:48 +0200 Subject: [PATCH 012/249] Fix error --- docker/test/sqltest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index b26d4d89f8b..24032874403 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -35,6 +35,7 @@ done # Run the test pushd sqltest/standards/2016/ /test.py +mv report.html test.log / popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log From 2255307a477ab86dc7740537a31e4791f3e769a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:49:47 +0200 Subject: [PATCH 013/249] Improvement --- docker/test/sqltest/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 7b38fdc52ad..945446358c3 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -95,7 +95,7 @@ def print_ratio(indent, name, success, total, description): f" - {description}" if description else '')) -report_html_file.write("

") +report_html_file.write("

")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +115,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("

") +report_html_file.write("
") From 50dac21483b9003ad546da6d87340124bf108d69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:54:09 +0200 Subject: [PATCH 014/249] Improvement --- docker/test/sqltest/test.py | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 945446358c3..ae2d6406c78 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -2,6 +2,7 @@ import os import yaml +import html import random import string from clickhouse_driver import Client @@ -26,11 +27,11 @@ with open('features.yml', 'r') as file: yaml_content = yaml.safe_load(file) for category in yaml_content: - log_file.write(category.capitalize() + " features:") + log_file.write(category.capitalize() + " features:\n") summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} for test in yaml_content[category]: - log_file.write(test + ": " + yaml_content[category][test]) + log_file.write(test + ": " + yaml_content[category][test] + "\n") summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} test_path = test[0] + "/" + test + ".tests.yml" @@ -54,11 +55,11 @@ with open('features.yml', 'r') as file: summary['results'][category]['total'] += 1 summary['total'] += 1 - log_file.write(query) + log_file.write(query + "\n") try: result = client.execute(query, settings = settings) - log_file.write(result) + log_file.write(result + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 @@ -67,7 +68,7 @@ with open('features.yml', 'r') as file: summary['success'] += 1 except Exception as e: - log_file.write(f"Error occurred: {str(e)}") + log_file.write(f"Error occurred: {str(e)}\n") client.execute(f"DROP DATABASE {database_name}", settings = settings) @@ -84,7 +85,7 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( ' ' * indent, name, enable_color(success / total), @@ -92,10 +93,10 @@ def print_ratio(indent, name, success, total, description): total, success / total, reset_color, - f" - {description}" if description else '')) + f" - " + html.escape(description) if description else '')) -report_html_file.write("
")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +116,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("
") +report_html_file.write("
\n") From 76677e009b8cf9a501fde1f7273fa5d546f4ffa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:00:50 +0200 Subject: [PATCH 015/249] Improvement --- docker/test/sqltest/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ae2d6406c78..cf2f976e05a 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -59,7 +59,7 @@ with open('features.yml', 'r') as file: try: result = client.execute(query, settings = settings) - log_file.write(result + "\n") + log_file.write(str(result) + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 From fe5972953703edb8a7e13c6066e5af369359b82c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:01:00 +0200 Subject: [PATCH 016/249] Improvement --- docker/test/sqltest/run.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 24032874403..03678971f60 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -23,6 +23,12 @@ function wget_with_retry wget_with_retry "$BINARY_URL_TO_DOWNLOAD" chmod +x clickhouse ./clickhouse install --noninteractive + +echo " +users: + default: + access_management: 1" > /etc/clickhouse-server/users.d/access_management.yaml + clickhouse start # Wait for start From 9b10e0a5bc2b7e3f84614d1aa3fa84c60de76929 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:09:19 +0200 Subject: [PATCH 017/249] Improvement --- docker/test/sqltest/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index cf2f976e05a..ff321d3e5df 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -85,9 +85,9 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( ' ' * indent, - name, + name.capitalize(), enable_color(success / total), success, total, @@ -96,7 +96,7 @@ def print_ratio(indent, name, success, total, description): f" - " + html.escape(description) if description else '')) -report_html_file.write("
\n")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -114,6 +114,6 @@ for category in summary['results']:
         if test_summary['total'] == 0:
             continue
 
-        print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
 report_html_file.write("
\n") From 891c01e6ab30c2d4377a118cb4c449134a56e17a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 02:37:02 +0000 Subject: [PATCH 018/249] Automatic style fix --- docker/test/sqltest/test.py | 119 ++++++++++++++++++++++-------------- 1 file changed, 74 insertions(+), 45 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ff321d3e5df..5807ca79b02 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -8,69 +8,85 @@ import string from clickhouse_driver import Client -client = Client(host = 'localhost', port = 9000) -settings = {'default_table_engine': 'Memory', 'union_default_mode': 'DISTINCT', 'calculate_text_stack_trace': 0} +client = Client(host="localhost", port=9000) +settings = { + "default_table_engine": "Memory", + "union_default_mode": "DISTINCT", + "calculate_text_stack_trace": 0, +} -database_name = 'sqltest_' + ''.join(random.choice(string.ascii_lowercase) for _ in range(10)) +database_name = "sqltest_" + "".join( + random.choice(string.ascii_lowercase) for _ in range(10) +) -client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings = settings) -client.execute(f"CREATE DATABASE {database_name}", settings = settings) +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings=settings) +client.execute(f"CREATE DATABASE {database_name}", settings=settings) -client = Client(host = 'localhost', port = 9000, database = database_name) +client = Client(host="localhost", port=9000, database=database_name) -summary = {'success': 0, 'total': 0, 'results': {}} +summary = {"success": 0, "total": 0, "results": {}} log_file = open("test.log", "w") report_html_file = open("report.html", "w") -with open('features.yml', 'r') as file: +with open("features.yml", "r") as file: yaml_content = yaml.safe_load(file) for category in yaml_content: log_file.write(category.capitalize() + " features:\n") - summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + summary["results"][category] = {"success": 0, "total": 0, "results": {}} for test in yaml_content[category]: log_file.write(test + ": " + yaml_content[category][test] + "\n") - summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} + summary["results"][category]["results"][test] = { + "success": 0, + "total": 0, + "description": yaml_content[category][test], + } test_path = test[0] + "/" + test + ".tests.yml" if os.path.exists(test_path): - with open(test_path, 'r') as test_file: - test_yaml_content = yaml.load_all(test_file, Loader = yaml.FullLoader) + with open(test_path, "r") as test_file: + test_yaml_content = yaml.load_all(test_file, Loader=yaml.FullLoader) for test_case in test_yaml_content: - - queries = test_case['sql']; + queries = test_case["sql"] if not isinstance(queries, list): queries = [queries] for query in queries: # Example: E011-01 - test_group = '' - if '-' in test: + test_group = "" + if "-" in test: test_group = test.split("-", 1)[0] - summary['results'][category]['results'][test_group]['total'] += 1 - summary['results'][category]['results'][test]['total'] += 1 - summary['results'][category]['total'] += 1 - summary['total'] += 1 + summary["results"][category]["results"][test_group][ + "total" + ] += 1 + summary["results"][category]["results"][test]["total"] += 1 + summary["results"][category]["total"] += 1 + summary["total"] += 1 log_file.write(query + "\n") try: - result = client.execute(query, settings = settings) + result = client.execute(query, settings=settings) log_file.write(str(result) + "\n") if test_group: - summary['results'][category]['results'][test_group]['success'] += 1 - summary['results'][category]['results'][test]['success'] += 1 - summary['results'][category]['success'] += 1 - summary['success'] += 1 + summary["results"][category]["results"][test_group][ + "success" + ] += 1 + summary["results"][category]["results"][test][ + "success" + ] += 1 + summary["results"][category]["success"] += 1 + summary["success"] += 1 except Exception as e: log_file.write(f"Error occurred: {str(e)}\n") -client.execute(f"DROP DATABASE {database_name}", settings = settings) +client.execute(f"DROP DATABASE {database_name}", settings=settings) + def enable_color(ratio): if ratio == 0: @@ -82,38 +98,51 @@ def enable_color(ratio): else: return "" + reset_color = "" + def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( - ' ' * indent, - name.capitalize(), - enable_color(success / total), - success, - total, - success / total, - reset_color, - f" - " + html.escape(description) if description else '')) + report_html_file.write( + "{}{}: {}{} / {} ({:.1%}){}{}\n".format( + " " * indent, + name.capitalize(), + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - " + html.escape(description) if description else "", + ) + ) -report_html_file.write("
\n")
+report_html_file.write(
+    "
\n"
+)
 
-print_ratio(0, 'Total', summary['success'], summary['total'], '')
+print_ratio(0, "Total", summary["success"], summary["total"], "")
 
-for category in summary['results']:
-    cat_summary = summary['results'][category]
+for category in summary["results"]:
+    cat_summary = summary["results"][category]
 
-    if cat_summary['total'] == 0:
+    if cat_summary["total"] == 0:
         continue
 
-    print_ratio(2, category, cat_summary['success'], cat_summary['total'], '')
+    print_ratio(2, category, cat_summary["success"], cat_summary["total"], "")
 
-    for test in summary['results'][category]['results']:
-        test_summary = summary['results'][category]['results'][test]
+    for test in summary["results"][category]["results"]:
+        test_summary = summary["results"][category]["results"][test]
 
-        if test_summary['total'] == 0:
+        if test_summary["total"] == 0:
             continue
 
-        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(
+            6 if "-" in test else 4,
+            test,
+            test_summary["success"],
+            test_summary["total"],
+            test_summary["description"],
+        )
 
 report_html_file.write("
\n") From 29094a22cf4cc787cc50f5d494b29dcf511cc765 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 20:00:00 +0200 Subject: [PATCH 019/249] Fix Docker --- docker/images.json | 1 + tests/ci/tests/docker_images.json | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 663fa21a6c9..8e19401ba72 100644 --- a/docker/images.json +++ b/docker/images.json @@ -125,6 +125,7 @@ "docker/test/keeper-jepsen", "docker/test/server-jepsen", "docker/test/sqllogic", + "docker/test/sqltest", "docker/test/stateless" ] }, diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0637058e184..70db8760561 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -119,7 +119,8 @@ "docker/test/stateless", "docker/test/integration/base", "docker/test/fuzzer", - "docker/test/keeper-jepsen" + "docker/test/keeper-jepsen", + "docker/test/sqltest" ] }, "docker/test/integration/kerberized_hadoop": { From f458108c44f2d9f0c92725f05d3959658c97ee6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 22:15:23 +0200 Subject: [PATCH 020/249] Fix style --- docker/test/sqllogic/run.sh | 2 ++ tests/ci/sqltest.py | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 444252837a3..3b900c097e2 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,4 +1,6 @@ #!/bin/bash +# shellcheck disable=SC2015 + set -exu trap "exit" INT TERM diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 5c20cc4849b..b752d4e4aee 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -11,7 +11,6 @@ from build_download_helper import get_build_name_for_check, read_build_urls from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from commit_status_helper import ( RerunHelper, - format_description, get_commit, post_commit_status, ) From fa44f84377bc097cbe0c25852603de9be536f75e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jul 2023 04:50:12 +0200 Subject: [PATCH 021/249] Fix style --- docker/test/sqllogic/run.sh | 1 - docker/test/sqltest/run.sh | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 3b900c097e2..5be44fc148c 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,5 +1,4 @@ #!/bin/bash -# shellcheck disable=SC2015 set -exu trap "exit" INT TERM diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 03678971f60..42aeef9df15 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -1,4 +1,5 @@ #!/bin/bash +# shellcheck disable=SC2015 set -x set -e From 2a00e2aa6ae7c417fdd5aafd63225ef44460de6f Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 20 Jul 2023 22:20:36 -0400 Subject: [PATCH 022/249] Implement `MOVE` query for the access entities. --- src/Access/AccessControl.cpp | 4 +- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 4 +- src/Access/DiskAccessStorage.h | 2 +- src/Access/IAccessStorage.cpp | 46 +++++++-- src/Access/IAccessStorage.h | 21 +++- src/Access/MemoryAccessStorage.cpp | 4 +- src/Access/MemoryAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 40 +++++++- src/Access/MultipleAccessStorage.h | 9 +- src/Access/ReplicatedAccessStorage.cpp | 4 +- src/Access/ReplicatedAccessStorage.h | 2 +- .../Access/InterpreterCreateQuotaQuery.cpp | 36 +++++-- .../Access/InterpreterCreateRoleQuery.cpp | 8 +- .../InterpreterCreateRowPolicyQuery.cpp | 38 ++++++-- .../InterpreterCreateSettingsProfileQuery.cpp | 37 +++++-- .../Access/InterpreterCreateUserQuery.cpp | 33 +++++-- .../InterpreterMoveAccessEntityQuery.cpp | 96 +++++++++++++++++++ .../Access/InterpreterMoveAccessEntityQuery.h | 24 +++++ src/Interpreters/InterpreterFactory.cpp | 6 ++ src/Parsers/Access/ASTCreateQuotaQuery.cpp | 6 ++ src/Parsers/Access/ASTCreateQuotaQuery.h | 1 + src/Parsers/Access/ASTCreateRoleQuery.cpp | 2 +- .../Access/ASTCreateRowPolicyQuery.cpp | 5 + src/Parsers/Access/ASTCreateRowPolicyQuery.h | 1 + .../Access/ASTCreateSettingsProfileQuery.cpp | 6 ++ .../Access/ASTCreateSettingsProfileQuery.h | 1 + src/Parsers/Access/ASTCreateUserQuery.cpp | 5 + src/Parsers/Access/ASTCreateUserQuery.h | 1 + .../Access/ASTDropAccessEntityQuery.cpp | 4 +- .../Access/ASTMoveAccessEntityQuery.cpp | 64 +++++++++++++ src/Parsers/Access/ASTMoveAccessEntityQuery.h | 32 +++++++ src/Parsers/Access/ParserCreateQuotaQuery.cpp | 6 ++ src/Parsers/Access/ParserCreateRoleQuery.cpp | 6 +- .../Access/ParserCreateRowPolicyQuery.cpp | 6 ++ .../ParserCreateSettingsProfileQuery.cpp | 6 ++ src/Parsers/Access/ParserCreateUserQuery.cpp | 5 + .../Access/ParserMoveAccessEntityQuery.cpp | 93 ++++++++++++++++++ .../Access/ParserMoveAccessEntityQuery.h | 17 ++++ src/Parsers/Access/parseUserName.h | 3 +- src/Parsers/IAST.h | 1 + src/Parsers/ParserQuery.cpp | 3 + .../integration/test_grant_and_revoke/test.py | 4 +- .../configs/memory.xml | 5 + .../configs/roles.xml | 4 - .../test.py | 67 ++++++++++++- tests/integration/test_role/test.py | 2 +- .../integration/test_settings_profile/test.py | 14 +-- .../integration/test_user_directories/test.py | 46 ++++----- 49 files changed, 731 insertions(+), 103 deletions(-) create mode 100644 src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp create mode 100644 src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h create mode 100644 src/Parsers/Access/ASTMoveAccessEntityQuery.cpp create mode 100644 src/Parsers/Access/ASTMoveAccessEntityQuery.h create mode 100644 src/Parsers/Access/ParserMoveAccessEntityQuery.cpp create mode 100644 src/Parsers/Access/ParserMoveAccessEntityQuery.h create mode 100644 tests/integration/test_multi_access_storage_role_management/configs/memory.xml diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 0c61a9e3e1b..22b00d5dde7 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -525,9 +525,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists); + auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id); if (id) changes_notifier->sendNotifications(); return id; diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 2a8293a49e7..85feaa70462 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 938881fafff..acbd7b12569 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -498,9 +498,9 @@ std::optional> DiskAccessStorage::readNameWi } -std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - UUID id = generateRandomID(); + UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true)) return id; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index e3bd32c0292..3541c3228e3 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ private: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 9468e8d220a..11bc97f6cc3 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -93,6 +93,17 @@ String IAccessStorage::readName(const UUID & id) const } +bool IAccessStorage::exists(const std::vector & ids) const +{ + for (const auto & id : ids) + { + if (!exists(id)) + return false; + } + + return true; +} + std::optional IAccessStorage::readName(const UUID & id, bool throw_if_not_exists) const { if (auto name_and_type = readNameWithType(id, throw_if_not_exists)) @@ -167,15 +178,26 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true); } - std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insertImpl(entity, replace_if_exists, throw_if_exists); + return *insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); } +std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +{ + return insertImpl(entity, replace_if_exists, throw_if_exists, set_id); +} std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists) { + return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists); +} + +std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists) +{ + if (!ids.empty()) + assert(multiple_entities.size() == ids.size()); + if (multiple_entities.empty()) return {}; @@ -189,16 +211,24 @@ std::vector IAccessStorage::insert(const std::vector & mu std::vector successfully_inserted; try { - std::vector ids; - for (const auto & entity : multiple_entities) + std::vector new_ids; + for (size_t i = 0; i < multiple_entities.size(); ++i) { - if (auto id = insertImpl(entity, replace_if_exists, throw_if_exists)) + const auto & entity = multiple_entities[i]; + + std::optional id; + if (!ids.empty()) + id = ids[i]; + + auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id); + + if (new_id) { successfully_inserted.push_back(entity); - ids.push_back(*id); + new_ids.push_back(*new_id); } } - return ids; + return new_ids; } catch (Exception & e) { @@ -244,7 +274,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool) +std::optional IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional) { if (isReadOnly()) throwReadonlyCannotInsert(entity->getType(), entity->getName()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index aa3947201e7..894220e95ad 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -92,6 +92,7 @@ public: /// Returns whether there is an entity with such identifier in the storage. virtual bool exists(const UUID & id) const = 0; + bool exists(const std::vector & ids) const; /// Reads an entity. Throws an exception if not found. template @@ -100,6 +101,9 @@ public: template std::shared_ptr read(const String & name, bool throw_if_not_exists = true) const; + template + std::vector read(const std::vector & ids, bool throw_if_not_exists = true) const; + /// Reads an entity. Returns nullptr if not found. template std::shared_ptr tryRead(const UUID & id) const; @@ -128,7 +132,9 @@ public: /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); + std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); /// Inserts an entity to the storage. Returns ID of a new entry in the storage. std::optional tryInsert(const AccessEntityPtr & entity); @@ -179,7 +185,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; @@ -240,6 +246,19 @@ std::shared_ptr IAccessStorage::read(const String & name, bo } +template +std::vector IAccessStorage::read(const std::vector & ids, bool throw_if_not_exists) const +{ + std::vector result; + result.reserve(ids.size()); + + for (const auto & id : ids) + result.push_back(read(id, throw_if_not_exists)); + + return result; +} + + template std::shared_ptr IAccessStorage::tryRead(const UUID & id) const { diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index c4192ee4552..16f3b3793a0 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -63,9 +63,9 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - UUID id = generateRandomID(); + UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) return id; diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index b63132147da..7c19ab9c4ab 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -44,7 +44,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36beb3f80b..169dc889509 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -213,6 +213,42 @@ ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & stora return const_cast(this)->getStorageByName(storage_name); } +StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, const DB::String & name, DB::MultipleAccessStorage::StoragePtr exclude) const +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + if (storage == exclude) + continue; + + if (storage->find(type, name)) + return storage; + } + + return nullptr; +} + +void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name) +{ + auto source_storage = findStorageByName(source_storage_name); + auto destination_storage = findStorageByName(destination_storage_name); + + auto to_move = source_storage->read(ids); + source_storage->remove(ids); + + try + { + destination_storage->insert(to_move, ids); + } + catch (Exception & e) + { + e.addMessage("while moving access entities"); + + source_storage->insert(to_move, ids); + throw; + } +} + AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { if (auto storage = findStorage(id)) @@ -280,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { std::shared_ptr storage_for_insertion; @@ -303,7 +339,7 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en getStorageName()); } - auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists); + auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id); if (id) { std::lock_guard lock{mutex}; diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 1898c6ba5d3..1d95296b796 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -46,6 +46,11 @@ public: ConstStoragePtr getStorageByName(const String & storage_name) const; StoragePtr getStorageByName(const String & storage_name); + /// Search for an access entity storage, excluding one. Returns nullptr if not found. + StoragePtr findExcludingStorage(AccessEntityType type, const String & name, StoragePtr exclude) const; + + void moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name); + bool exists(const UUID & id) const override; bool isBackupAllowed() const override; @@ -58,7 +63,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; @@ -70,6 +75,8 @@ private: std::shared_ptr nested_storages TSA_GUARDED_BY(mutex); mutable CacheBase ids_cache TSA_GUARDED_BY(mutex); mutable std::mutex mutex; + + mutable std::mutex move_mutex; }; } diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 56d68be9268..efa9dbbdfaa 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -108,9 +108,9 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - const UUID id = generateRandomID(); + const UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) return id; diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 555d58e6b04..0a0c53d3bc4 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,7 +46,7 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 83a620d11c6..e271497ff5c 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -14,6 +14,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateQuotaFromQueryImpl( @@ -90,6 +96,15 @@ BlockIO InterpreterCreateQuotaQuery::execute() if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -100,11 +115,11 @@ BlockIO InterpreterCreateQuotaQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -116,12 +131,21 @@ BlockIO InterpreterCreateQuotaQuery::execute() new_quotas.emplace_back(std::move(new_quota)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::QUOTA, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Quota {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_quotas); + storage->tryInsert(new_quotas); else if (query.or_replace) - access_control.insertOrReplace(new_quotas); + storage->insertOrReplace(new_quotas); else - access_control.insert(new_quotas); + storage->insert(new_quotas); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 87abe002d3d..f6c3103a7a9 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -97,13 +97,7 @@ BlockIO InterpreterCreateRoleQuery::execute() { for (const auto & name : query.names) { - auto id = access_control.find(name); - - if (!id) - continue; - - auto another_storage_ptr = access_control.findStorage(*id); - if (another_storage_ptr != storage_ptr) + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::ROLE, name, storage_ptr)) throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Role {} already exists in storage {}", name, another_storage_ptr->getStorageName()); } } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 87dc9039c17..a938d7afc16 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -14,6 +14,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateRowPolicyFromQueryImpl( @@ -66,6 +72,16 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + + Strings names = query.names->toStrings(); if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -74,14 +90,13 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() updateRowPolicyFromQueryImpl(*updated_policy, query, {}, roles_from_query); return updated_policy; }; - Strings names = query.names->toStrings(); if (query.if_exists) { - auto ids = access_control.find(names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(names), update_func); + storage->update(storage->getIDs(names), update_func); } else { @@ -93,12 +108,21 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() new_policies.emplace_back(std::move(new_policy)); } + if (!query.storage_name.empty()) + { + for (const auto & name : names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::ROW_POLICY, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Row policy {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_policies); + storage->tryInsert(new_policies); else if (query.or_replace) - access_control.insertOrReplace(new_policies); + storage->insertOrReplace(new_policies); else - access_control.insert(new_policies); + storage->insert(new_policies); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 0727b6f2182..bac0146e821 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -10,6 +10,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateSettingsProfileFromQueryImpl( @@ -67,6 +73,16 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() if (query.to_roles) roles_from_query = RolesOrUsersSet{*query.to_roles, access_control, getContext()->getUserID()}; + + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -77,11 +93,11 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -93,12 +109,21 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() new_profiles.emplace_back(std::move(new_profile)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::SETTINGS_PROFILE, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Settings profile {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_profiles); + storage->tryInsert(new_profiles); else if (query.or_replace) - access_control.insertOrReplace(new_profiles); + storage->insertOrReplace(new_profiles); else - access_control.insert(new_profiles); + storage->insert(new_profiles); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 165937560cc..f1d921a9eb2 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int ACCESS_ENTITY_ALREADY_EXISTS; } namespace { @@ -139,6 +140,16 @@ BlockIO InterpreterCreateUserQuery::execute() if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext()); + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + + Strings names = query.names->toStrings(); if (query.alter) { std::optional grantees_from_query; @@ -152,14 +163,13 @@ BlockIO InterpreterCreateUserQuery::execute() return updated_user; }; - Strings names = query.names->toStrings(); if (query.if_exists) { - auto ids = access_control.find(names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(names), update_func); + storage->update(storage->getIDs(names), update_func); } else { @@ -171,13 +181,22 @@ BlockIO InterpreterCreateUserQuery::execute() new_users.emplace_back(std::move(new_user)); } + if (!query.storage_name.empty()) + { + for (const auto & name : names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::USER, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "User {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + std::vector ids; if (query.if_not_exists) - ids = access_control.tryInsert(new_users); + ids = storage->tryInsert(new_users); else if (query.or_replace) - ids = access_control.insertOrReplace(new_users); + ids = storage->insertOrReplace(new_users); else - ids = access_control.insert(new_users); + ids = storage->insert(new_users); if (query.grantees) { diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..ac7ff4efcb6 --- /dev/null +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -0,0 +1,96 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int ACCESS_ENTITY_NOT_FOUND; +} + + +BlockIO InterpreterMoveAccessEntityQuery::execute() +{ + auto & query = query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); + getContext()->checkAccess(getRequiredAccess()); + + if (!query.cluster.empty()) + return executeDDLQueryOnCluster(query_ptr, getContext()); + + query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); + + std::vector ids; + if (query.type == AccessEntityType::ROW_POLICY) + ids = access_control.find(query.type, query.row_policy_names->toStrings()); + else + ids = access_control.find(query.type, query.names); + + if (ids.empty()) + return {}; + + /// Validate that all entities are from the same storage. + const auto source_storage = access_control.findStorage(ids.front()); + if (!source_storage->exists(ids)) + throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "All access entities must be from the same storage in order to be moved"); + + access_control.moveAccessEntities(ids, source_storage->getStorageName(), query.storage_name); + return {}; +} + + +AccessRightsElements InterpreterMoveAccessEntityQuery::getRequiredAccess() const +{ + const auto & query = query_ptr->as(); + AccessRightsElements res; + switch (query.type) + { + case AccessEntityType::USER: + { + res.emplace_back(AccessType::DROP_USER); + res.emplace_back(AccessType::CREATE_USER); + return res; + } + case AccessEntityType::ROLE: + { + res.emplace_back(AccessType::DROP_ROLE); + res.emplace_back(AccessType::CREATE_ROLE); + return res; + } + case AccessEntityType::SETTINGS_PROFILE: + { + res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); + res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE); + return res; + } + case AccessEntityType::ROW_POLICY: + { + if (query.row_policy_names) + { + for (const auto & row_policy_name : query.row_policy_names->full_names) + { + res.emplace_back(AccessType::DROP_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + res.emplace_back(AccessType::CREATE_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + } + } + return res; + } + case AccessEntityType::QUOTA: + { + res.emplace_back(AccessType::DROP_QUOTA); + res.emplace_back(AccessType::CREATE_QUOTA); + return res; + } + case AccessEntityType::MAX: + break; + } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); +} + +} diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h new file mode 100644 index 00000000000..1a70bf7c065 --- /dev/null +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class AccessRightsElements; + +class InterpreterMoveAccessEntityQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterMoveAccessEntityQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + AccessRightsElements getRequiredAccess() const; + + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 9cd1f2a251c..27c94119750 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,7 @@ #include #include #include +#include #include #include #include @@ -314,6 +316,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.cpp b/src/Parsers/Access/ASTCreateQuotaQuery.cpp index 56abedf5235..091b62b0a9f 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ASTCreateQuotaQuery.cpp @@ -170,6 +170,12 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); formatNames(names, settings); + + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " IN " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h index f5eb59800ec..aecbbb00f9a 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -38,6 +38,7 @@ public: Strings names; String new_name; std::optional key_type; + String storage_name; struct Limits { diff --git a/src/Parsers/Access/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp index 14946f2d3cd..eeeb34c97e4 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -74,7 +74,7 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState & if (!storage_name.empty()) format.ostr << (format.hilite ? IAST::hilite_keyword : "") - << " AT " << (format.hilite ? IAST::hilite_none : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); formatOnCluster(format); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index ca888be2cfe..a423581e300 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -168,6 +168,11 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format settings.ostr << " "; names->format(settings); + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " IN " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); assert(names->cluster.empty()); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h index 32d98fab822..3f2418e7287 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -35,6 +35,7 @@ public: bool if_exists = false; bool if_not_exists = false; bool or_replace = false; + String storage_name; std::shared_ptr names; String new_short_name; diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp index 56ddef433ef..3379486d789 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp @@ -81,6 +81,12 @@ void ASTCreateSettingsProfileQuery::formatImpl(const FormatSettings & format, Fo format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); formatNames(names, format); + + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h index c1a64998f29..be01aae1e26 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h @@ -28,6 +28,7 @@ public: bool if_exists = false; bool if_not_exists = false; bool or_replace = false; + String storage_name; Strings names; String new_name; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 0611545adf0..dbb4c1c6d57 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -208,6 +208,11 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << " "; names->format(format); + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (new_name) diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 62ddbfd0040..06940e67d91 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -45,6 +45,7 @@ public: std::shared_ptr names; std::optional new_name; + String storage_name; std::shared_ptr auth_data; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 02b29f3fd2c..bcd7105d0e9 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -54,8 +54,8 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma formatNames(names, settings); if (!storage_name.empty()) - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") - << " FROM " << (settings.hilite ? IAST::hilite_none : "") + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(storage_name); formatOnCluster(settings); diff --git a/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp b/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..285f07854c0 --- /dev/null +++ b/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp @@ -0,0 +1,64 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + void formatNames(const Strings & names, const IAST::FormatSettings & settings) + { + bool need_comma = false; + for (const auto & name : names) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + settings.ostr << ' ' << backQuoteIfNeed(name); + } + } +} + +String ASTMoveAccessEntityQuery::getID(char) const +{ + return String("MOVE ") + toString(type) + " query"; +} + +ASTPtr ASTMoveAccessEntityQuery::clone() const +{ + auto res = std::make_shared(*this); + + if (row_policy_names) + res->row_policy_names = std::static_pointer_cast(row_policy_names->clone()); + + return res; +} + +void ASTMoveAccessEntityQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "MOVE " << AccessEntityTypeInfo::get(type).name + << (settings.hilite ? hilite_none : ""); + + if (type == AccessEntityType::ROW_POLICY) + { + settings.ostr << " "; + row_policy_names->format(settings); + } + else + formatNames(names, settings); + + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " TO " << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(storage_name); + + formatOnCluster(settings); +} + +void ASTMoveAccessEntityQuery::replaceEmptyDatabase(const String & current_database) const +{ + if (row_policy_names) + row_policy_names->replaceEmptyDatabase(current_database); +} +} diff --git a/src/Parsers/Access/ASTMoveAccessEntityQuery.h b/src/Parsers/Access/ASTMoveAccessEntityQuery.h new file mode 100644 index 00000000000..aa2b3b0f98c --- /dev/null +++ b/src/Parsers/Access/ASTMoveAccessEntityQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +class ASTRowPolicyNames; + +/** MOVE {USER | ROLE | QUOTA | [ROW] POLICY | [SETTINGS] PROFILE} [IF EXISTS] name [,...] [ON [database.]table [,...]] TO storage_name + */ +class ASTMoveAccessEntityQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + AccessEntityType type; + Strings names; + std::shared_ptr row_policy_names; + + String storage_name; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + void replaceEmptyDatabase(const String & current_database) const; + + QueryKind getQueryKind() const override { return QueryKind::Move; } +}; +} diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 2330ba7771c..05c2820b012 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -288,6 +289,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe std::optional key_type; std::vector all_limits; String cluster; + String storage_name; while (true) { @@ -310,6 +312,9 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -332,6 +337,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->key_type = key_type; query->all_limits = std::move(all_limits); query->roles = std::move(roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index de9e5baed18..b548c07ec70 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -93,9 +93,6 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String cluster; String storage_name; - if (ParserKeyword{"AT"}.ignore(pos, expected)) - parseStorageName(pos, expected, storage_name); - while (true) { if (alter && new_name.empty() && (names.size() == 1) && parseRenameTo(pos, expected, new_name)) @@ -114,6 +111,9 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index 2c25fc14e7d..b2177ca4ade 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -245,6 +246,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String new_short_name; std::optional is_restrictive; std::vector> filters; + String storage_name; while (true) { @@ -271,6 +273,9 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -294,6 +299,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->is_restrictive = is_restrictive; query->filters = std::move(filters); query->roles = std::move(roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index c58a3035dc6..634ff68427a 100644 --- a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -111,6 +112,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec String new_name; std::shared_ptr settings; String cluster; + String storage_name; while (true) { @@ -130,6 +132,9 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -152,6 +157,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec query->new_name = std::move(new_name); query->settings = std::move(settings); query->to_roles = std::move(to_roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 0344fb99c04..d95d41296dc 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -414,6 +414,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec std::shared_ptr grantees; std::shared_ptr default_database; String cluster; + String storage_name; while (true) { @@ -480,6 +481,9 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -514,6 +518,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->settings = std::move(settings); query->grantees = std::move(grantees); query->default_database = std::move(default_database); + query->storage_name = std::move(storage_name); if (query->auth_data) query->children.push_back(query->auth_data); diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..6ac41b77b84 --- /dev/null +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool parseEntityType(IParserBase::Pos & pos, Expected & expected, AccessEntityType & type) + { + for (auto i : collections::range(AccessEntityType::MAX)) + { + const auto & type_info = AccessEntityTypeInfo::get(i); + if (ParserKeyword{type_info.name}.ignore(pos, expected) + || (!type_info.alias.empty() && ParserKeyword{type_info.alias}.ignore(pos, expected))) + { + type = i; + return true; + } + } + return false; + } + + + bool parseOnCluster(IParserBase::Pos & pos, Expected & expected, String & cluster) + { + return IParserBase::wrapParseImpl(pos, [&] + { + return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected); + }); + } +} + + +bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"MOVE"}.ignore(pos, expected)) + return false; + + AccessEntityType type; + if (!parseEntityType(pos, expected, type)) + return false; + + Strings names; + std::shared_ptr row_policy_names; + String storage_name; + String cluster; + + if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) + { + if (!parseUserNames(pos, expected, names)) + return false; + } + else if (type == AccessEntityType::ROW_POLICY) + { + ParserRowPolicyNames parser; + ASTPtr ast; + parser.allowOnCluster(); + if (!parser.parse(pos, ast, expected)) + return false; + row_policy_names = typeid_cast>(ast); + cluster = std::exchange(row_policy_names->cluster, ""); + } + else + { + if (!parseIdentifiersOrStringLiterals(pos, expected, names)) + return false; + } + + if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name)) + return false; + + if (cluster.empty()) + parseOnCluster(pos, expected, cluster); + + auto query = std::make_shared(); + node = query; + + query->type = type; + query->cluster = std::move(cluster); + query->names = std::move(names); + query->row_policy_names = std::move(row_policy_names); + query->storage_name = std::move(storage_name); + + return true; +} +} diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.h b/src/Parsers/Access/ParserMoveAccessEntityQuery.h new file mode 100644 index 00000000000..3accec883dd --- /dev/null +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * MOVE {USER | ROLE | QUOTA | [ROW] POLICY | [SETTINGS] PROFILE} [IF EXISTS] name [,...] [ON [database.]table [,...]] TO storage_name + */ +class ParserMoveAccessEntityQuery : public IParserBase +{ +protected: + const char * getName() const override { return "MOVE access entity query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index 717911bf2f1..64fed2ce21c 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -36,7 +37,7 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) { - return parseUserName(pos, expected, storage_name); + return parseIdentifierOrStringLiteral(pos, expected, storage_name); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index aa5302a15b9..92fb8ed275b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -290,6 +290,7 @@ public: Alter, Grant, Revoke, + Move, System, Set, Use, diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 77e7b58e6b1..7ed69940bed 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include @@ -54,6 +55,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateIndexQuery create_index_p; ParserDropIndexQuery drop_index_p; ParserDropAccessEntityQuery drop_access_entity_p; + ParserMoveAccessEntityQuery move_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; ParserExternalDDLQuery external_ddl_p; @@ -80,6 +82,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_index_p.parse(pos, node, expected) || drop_index_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) + || move_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected) || transaction_control_p.parse(pos, node, expected) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index ee5d4b5df93..8e040a18a88 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -471,7 +471,7 @@ def test_introspection(): [ [ "A", - "local directory", + "local_directory", "no_password", "{}", "['::/0']", @@ -484,7 +484,7 @@ def test_introspection(): ], [ "B", - "local directory", + "local_directory", "no_password", "{}", "['::/0']", diff --git a/tests/integration/test_multi_access_storage_role_management/configs/memory.xml b/tests/integration/test_multi_access_storage_role_management/configs/memory.xml new file mode 100644 index 00000000000..c2f60c8bbc1 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/configs/memory.xml @@ -0,0 +1,5 @@ + + + + + diff --git a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml index 672c7e24728..faaf1ee27ae 100644 --- a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml +++ b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml @@ -1,8 +1,4 @@ - - - - diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index d9a4a73c56b..fb4bbd6b718 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -8,6 +8,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", stay_alive=True, + main_configs=["configs/memory.xml"] ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -28,6 +29,68 @@ def started_cluster(): cluster.shutdown() +def execute_test_for_access_type(access_type: str, system_table_name: str): + node.query(f"CREATE {access_type} test1 IN local_directory") + node.query(f"CREATE {access_type} test2 IN local_directory") + node.query(f"CREATE {access_type} test3 IN local_directory") + + node.query(f"CREATE {access_type} test4 IN memory") + node.query(f"CREATE {access_type} test5 IN memory") + node.query(f"CREATE {access_type} test6 IN memory") + + # Already exists + with pytest.raises(QueryRuntimeException): + node.query(f"CREATE {access_type} test1 IN memory") + + node.query(f"MOVE {access_type} test1 TO memory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'") == TSV(["memory"]) + + node.query(f"MOVE {access_type} test2 TO local_directory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["local_directory"]) + + node.query(f"MOVE {access_type} test2,test3 TO memory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["memory"]) + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'") == TSV(["memory"]) + + node.query(f"MOVE {access_type} test4,test5 TO local_directory") + + # Different storages + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test4,test1 TO memory") + + # Doesn't exist + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test7 TO local_directory") + + # Storage doesn't exist + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test6 TO non_existing_storage") + + # Unwriteable storage + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test6 TO users_xml") + + +def test_roles(): + execute_test_for_access_type("ROLE", "roles") + + +def test_users(): + execute_test_for_access_type("USER", "users") + + +def test_settings_profiles(): + execute_test_for_access_type("SETTINGS PROFILE", "settings_profiles") + + +def test_quotas(): + execute_test_for_access_type("QUOTA", "quotas") + + +def test_row_policies(): + execute_test_for_access_type("ROW POLICY", "row_policies") + + def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") @@ -72,9 +135,9 @@ def test_role_from_different_storages(): # Already exists with pytest.raises(QueryRuntimeException): - node.query("CREATE ROLE default_role AT memory") + node.query("CREATE ROLE default_role IN memory") - node.query("CREATE ROLE other_role AT memory") + node.query("CREATE ROLE other_role IN memory") assert node.query( "SELECT storage FROM system.roles WHERE name = 'other_role'" diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 9a352f43eab..43a402d9fda 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -287,7 +287,7 @@ def test_introspection(): assert instance.query( "SELECT name, storage from system.roles WHERE name IN ('R1', 'R2') ORDER BY name" - ) == TSV([["R1", "local directory"], ["R2", "local directory"]]) + ) == TSV([["R1", "local_directory"], ["R2", "local_directory"]]) assert instance.query( "SELECT * from system.grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, access_type, database, table, column, is_partial_revoke, grant_option" diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 13acc3f1042..8770cc82064 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -88,7 +88,7 @@ def test_smoke(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "['robin']", "[]"] + ["xyz", "local_directory", 1, 0, "['robin']", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -120,7 +120,7 @@ def test_smoke(): instance.query("SET max_memory_usage = 80000000", user="robin") instance.query("SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "[]", "[]"] + ["xyz", "local_directory", 1, 0, "[]", "[]"] ] assert system_settings_profile_elements(user_name="robin") == [] @@ -201,7 +201,7 @@ def test_settings_from_granted_role(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "[]", "[]"] + ["xyz", "local_directory", 2, 0, "[]", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -276,7 +276,7 @@ def test_settings_from_granted_role(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "['worker']", "[]"] + ["xyz", "local_directory", 2, 0, "['worker']", "[]"] ] instance.query("ALTER SETTINGS PROFILE xyz TO NONE") @@ -293,7 +293,7 @@ def test_settings_from_granted_role(): ) instance.query("SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "[]", "[]"] + ["xyz", "local_directory", 2, 0, "[]", "[]"] ] @@ -323,7 +323,7 @@ def test_inheritance(): ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "[]", "[]"] + ["xyz", "local_directory", 1, 0, "[]", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -340,7 +340,7 @@ def test_inheritance(): ] ] assert system_settings_profile("alpha") == [ - ["alpha", "local directory", 1, 0, "['robin']", "[]"] + ["alpha", "local_directory", 1, 0, "['robin']", "[]"] ] assert system_settings_profile_elements(profile_name="alpha") == [ ["alpha", "\\N", "\\N", 0, "\\N", "\\N", "\\N", "\\N", "\\N", "xyz"] diff --git a/tests/integration/test_user_directories/test.py b/tests/integration/test_user_directories/test.py index 45afb86f464..704fb30b2fd 100644 --- a/tests/integration/test_user_directories/test.py +++ b/tests/integration/test_user_directories/test.py @@ -38,14 +38,14 @@ def test_old_style(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users2.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access2\\\\/"}', 2, ], @@ -62,20 +62,20 @@ def test_local_directories(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users3.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access3\\\\/"}', 2, ], [ "local directory (ro)", - "local directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access3-ro\\\\/","readonly":true}', 3, ], @@ -92,8 +92,8 @@ def test_relative_path(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users4.xml"}', 1, ] @@ -110,8 +110,8 @@ def test_memory(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users5.xml"}', 1, ], @@ -129,20 +129,20 @@ def test_mixed_style(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users6.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access6\\\\/"}', 2, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access6a\\\\/"}', 3, ], @@ -160,14 +160,14 @@ def test_duplicates(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users7.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access7\\\\/"}', 2, ], From a8c08b34a6ca179d2b89da87d920916978bf4129 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jul 2023 02:39:21 +0000 Subject: [PATCH 023/249] Automatic style fix --- .../test.py | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index fb4bbd6b718..4c2696158ec 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -6,9 +6,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", - stay_alive=True, - main_configs=["configs/memory.xml"] + "node", stay_alive=True, main_configs=["configs/memory.xml"] ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -43,14 +41,22 @@ def execute_test_for_access_type(access_type: str, system_table_name: str): node.query(f"CREATE {access_type} test1 IN memory") node.query(f"MOVE {access_type} test1 TO memory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'") == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'" + ) == TSV(["memory"]) node.query(f"MOVE {access_type} test2 TO local_directory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["local_directory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'" + ) == TSV(["local_directory"]) node.query(f"MOVE {access_type} test2,test3 TO memory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["memory"]) - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'") == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'" + ) == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'" + ) == TSV(["memory"]) node.query(f"MOVE {access_type} test4,test5 TO local_directory") From 3398355e4649beaa2b0a3180aa687c364c245e81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:04:30 +0200 Subject: [PATCH 024/249] Add something with unclear purpose --- docker/README.md | 2 +- tests/ci/docker_test.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..76fdbf0f3f0 100644 --- a/docker/README.md +++ b/docker/README.md @@ -2,4 +2,4 @@ This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. -Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. +Also, there is a bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index d5d27f73694..61319041b9f 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,6 +40,7 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), + di.DockerImage("docker/test/sqltest", "clickhouse/sqltest", True), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From c3c8ea9c27669a548e2404e2ed6c2e8f3c734dc9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 17:30:55 +0200 Subject: [PATCH 025/249] Do something --- tests/ci/docker_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 61319041b9f..8542de412d2 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,7 +40,12 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), - di.DockerImage("docker/test/sqltest", "clickhouse/sqltest", True), + di.DockerImage( + "docker/test/sqltest", + "clickhouse/sqltest", + True, + "clickhouse/test-base", # type: ignore + ), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From fbe9f8d0f653cedd30640c42968df1ac8cf78dff Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 14:12:01 +0000 Subject: [PATCH 026/249] fix recalculation of skip indexes and projcetion in ALTER DELETE --- src/Interpreters/MutationsInterpreter.cpp | 89 +++++++++++------ src/Interpreters/MutationsInterpreter.h | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + src/Storages/MergeTree/MutateTask.cpp | 98 +++++++++++++++++-- src/Storages/StorageInMemoryMetadata.cpp | 6 +- src/Storages/StorageInMemoryMetadata.h | 4 +- ...alter_delete_indexes_projections.reference | 6 ++ ...02832_alter_delete_indexes_projections.sql | 26 +++++ 9 files changed, 201 insertions(+), 40 deletions(-) create mode 100644 tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference create mode 100644 tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 25c52ad8925..1a7cbb45999 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -113,13 +113,14 @@ QueryTreeNodePtr prepareQueryAffectedQueryTree(const std::vector & has_index_or_projection) + const StorageInMemoryMetadata::HasDependencyCallback & has_dependency) { NameSet new_updated_columns = updated_columns; ColumnDependencies dependencies; + while (!new_updated_columns.empty()) { - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_dependency); new_updated_columns.clear(); for (const auto & dependency : new_dependencies) { @@ -292,9 +293,14 @@ bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const return data && data->getSettings()->materialize_ttl_recalculate_only; } -bool MutationsInterpreter::Source::hasIndexOrProjection(const String & file_name) const +bool MutationsInterpreter::Source::hasSecondaryIndex(const String & name) const { - return part && part->checksums.has(file_name); + return part && part->hasSecondaryIndex(name); +} + +bool MutationsInterpreter::Source::hasProjection(const String & name) const +{ + return part && part->hasProjection(name); } static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) @@ -533,13 +539,24 @@ void MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } - std::function has_index_or_projection - = [&](const String & file_name) { return source.hasIndexOrProjection(file_name); }; + StorageInMemoryMetadata::HasDependencyCallback has_dependency = + [&](const String & name, ColumnDependency::Kind kind) + { + if (kind == ColumnDependency::PROJECTION) + return source.hasProjection(name); + + if (kind == ColumnDependency::SKIP_INDEX) + return source.hasSecondaryIndex(name); + + return true; + }; if (settings.recalculate_dependencies_of_updated_columns) - dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_index_or_projection); + dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_dependency); + bool has_alter_delete = false; std::vector read_columns; + /// First, break a sequence of commands into stages. for (auto & command : commands) { @@ -558,6 +575,7 @@ void MutationsInterpreter::prepare(bool dry_run) predicate = makeASTFunction("isZeroOrNull", predicate); stages.back().filters.push_back(predicate); + has_alter_delete = true; } else if (command.type == MutationCommand::UPDATE) { @@ -692,8 +710,7 @@ void MutationsInterpreter::prepare(bool dry_run) if (it == std::cend(indices_desc)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown index: {}", command.index_name); - if (!source.hasIndexOrProjection("skp_idx_" + it->name + ".idx") - && !source.hasIndexOrProjection("skp_idx_" + it->name + ".idx2")) + if (!source.hasSecondaryIndex(it->name)) { auto query = (*it).expression_list_ast->clone(); auto syntax_result = TreeRewriter(context).analyze(query, all_columns); @@ -707,7 +724,7 @@ void MutationsInterpreter::prepare(bool dry_run) { mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); - if (!source.hasIndexOrProjection(projection.getDirectoryName())) + if (!source.hasProjection(projection.name)) { for (const auto & column : projection.required_columns) dependencies.emplace(column, ColumnDependency::PROJECTION); @@ -732,7 +749,7 @@ void MutationsInterpreter::prepare(bool dry_run) // just recalculate ttl_infos without remove expired data auto all_columns_vec = all_columns.getNames(); auto new_dependencies = metadata_snapshot->getColumnDependencies( - NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_index_or_projection); + NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_dependency); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::TTL_EXPRESSION) @@ -758,7 +775,7 @@ void MutationsInterpreter::prepare(bool dry_run) auto all_columns_vec = all_columns.getNames(); auto all_dependencies = getAllColumnDependencies( - metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_index_or_projection); + metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_dependency); for (const auto & dependency : all_dependencies) { @@ -767,7 +784,7 @@ void MutationsInterpreter::prepare(bool dry_run) } /// Recalc only skip indices and projections of columns which could be updated by TTL. - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_dependency); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) @@ -861,30 +878,44 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) + if (!source.hasSecondaryIndex(index.name)) + continue; + + if (has_alter_delete) { - const auto & index_cols = index.expression->getRequiredColumns(); - bool changed = std::any_of( - index_cols.begin(), - index_cols.end(), - [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); - if (changed) - materialized_indices.insert(index.name); + materialized_indices.insert(index.name); + continue; } + + const auto & index_cols = index.expression->getRequiredColumns(); + bool changed = std::any_of( + index_cols.begin(), + index_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + + if (changed) + materialized_indices.insert(index.name); } for (const auto & projection : metadata_snapshot->getProjections()) { - if (source.hasIndexOrProjection(projection.getDirectoryName())) + if (!source.hasProjection(projection.name)) + continue; + + if (has_alter_delete) { - const auto & projection_cols = projection.required_columns; - bool changed = std::any_of( - projection_cols.begin(), - projection_cols.end(), - [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); - if (changed) - materialized_projections.insert(projection.name); + materialized_projections.insert(projection.name); + continue; } + + const auto & projection_cols = projection.required_columns; + bool changed = std::any_of( + projection_cols.begin(), + projection_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + + if (changed) + materialized_projections.insert(projection.name); } /// Stages might be empty when we materialize skip indices or projections which don't add any diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index d783b503531..9b4caaae231 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -120,7 +120,8 @@ public: bool supportsLightweightDelete() const; bool hasLightweightDeleteMask() const; bool materializeTTLRecalculateOnly() const; - bool hasIndexOrProjection(const String & file_name) const; + bool hasSecondaryIndex(const String & name) const; + bool hasProjection(const String & name) const; void read( Stage & first_stage, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7050a98a4bc..1ab8dc7fb05 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1983,6 +1983,12 @@ IndexSize IMergeTreeDataPart::getSecondaryIndexSize(const String & secondary_ind return ColumnSize{}; } +bool IMergeTreeDataPart::hasSecondaryIndex(const String & index_name) const +{ + auto file_name = INDEX_FILE_PREFIX + index_name; + return checksums.has(file_name + ".idx") || checksums.has(file_name + ".idx2"); +} + void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const { for (const auto & [column_name, size] : columns_sizes) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index af6906e004d..bfb472ca50d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -122,6 +122,9 @@ public: /// Otherwise return information about secondary index size on disk. IndexSize getSecondaryIndexSize(const String & secondary_index_name) const; + /// Returns true if there is materialized index with specified name in part. + bool hasSecondaryIndex(const String & index_name) const; + /// Return information about column size on disk for all columns in part ColumnSize getTotalColumnsSize() const { return total_columns_size; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 491c36433ca..fa5ad9858e8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -69,6 +69,7 @@ static void splitAndModifyMutationCommands( { NameSet mutated_columns; NameSet dropped_columns; + NameSet extra_columns_for_indices_and_projections; for (const auto & command : commands) { @@ -85,6 +86,41 @@ static void splitAndModifyMutationCommands( if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN) mutated_columns.emplace(command.column_name); + + if (command.type == MutationCommand::Type::MATERIALIZE_INDEX) + { + const auto & all_indices = metadata_snapshot->getSecondaryIndices(); + for (const auto & index : all_indices) + { + if (index.name == command.index_name) + { + auto required_columns = index.expression->getRequiredColumns(); + for (const auto & column : required_columns) + { + if (!part_columns.has(column)) + extra_columns_for_indices_and_projections.insert(column); + } + break; + } + } + } + + if (command.type == MutationCommand::Type::MATERIALIZE_PROJECTION) + { + const auto & all_projections = metadata_snapshot->getProjections(); + for (const auto & projection : all_projections) + { + if (projection.name == command.projection_name) + { + for (const auto & column : projection.required_columns) + { + if (!part_columns.has(column)) + extra_columns_for_indices_and_projections.insert(column); + } + break; + } + } + } } else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION) { @@ -187,6 +223,25 @@ static void splitAndModifyMutationCommands( }); } } + + for (const auto & column_name : extra_columns_for_indices_and_projections) + { + if (mutated_columns.contains(column_name)) + continue; + + auto data_type = metadata_snapshot->getColumns().getColumn( + GetColumnsOptions::AllPhysical, + column_name).type; + + for_interpreter.push_back( + MutationCommand + { + .type = MutationCommand::Type::READ_COLUMN, + .column_name = column_name, + .data_type = std::move(data_type), + } + ); + } } else { @@ -453,6 +508,7 @@ static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snaps /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream static std::set getIndicesToRecalculate( + const MergeTreeDataPartPtr & source_part, QueryPipelineBuilder & builder, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, @@ -463,10 +519,15 @@ static std::set getIndicesToRecalculate( std::set indices_to_recalc; ASTPtr indices_recalc_expr_list = std::make_shared(); const auto & indices = metadata_snapshot->getSecondaryIndices(); + bool is_full_part_storage = isFullPartStorage(source_part->getDataPartStorage()); for (const auto & index : indices) { - if (materialized_indices.contains(index.name)) + bool need_recalculate = + materialized_indices.contains(index.name) + || (!is_full_part_storage && source_part->hasSecondaryIndex(index.name)); + + if (need_recalculate) { if (indices_to_recalc.insert(index_factory.get(index)).second) { @@ -496,15 +557,23 @@ static std::set getIndicesToRecalculate( } static std::set getProjectionsToRecalculate( + const MergeTreeDataPartPtr & source_part, const StorageMetadataPtr & metadata_snapshot, const NameSet & materialized_projections) { std::set projections_to_recalc; + bool is_full_part_storage = isFullPartStorage(source_part->getDataPartStorage()); + for (const auto & projection : metadata_snapshot->getProjections()) { - if (materialized_projections.contains(projection.name)) + bool need_recalculate = + materialized_projections.contains(projection.name) + || (!is_full_part_storage && source_part->hasProjection(projection.name)); + + if (need_recalculate) projections_to_recalc.insert(&projection); } + return projections_to_recalc; } @@ -1279,14 +1348,20 @@ private: removed_indices.insert(command.column_name); } + bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage()); const auto & indices = ctx->metadata_snapshot->getSecondaryIndices(); + MergeTreeIndices skip_indices; for (const auto & idx : indices) { if (removed_indices.contains(idx.name)) continue; - if (ctx->materialized_indices.contains(idx.name)) + bool need_recalculate = + ctx->materialized_indices.contains(idx.name) + || (!is_full_part_storage && ctx->source_part->hasSecondaryIndex(idx.name)); + + if (need_recalculate) { skip_indices.push_back(MergeTreeIndexFactory::instance().get(idx)); } @@ -1319,7 +1394,11 @@ private: if (removed_projections.contains(projection.name)) continue; - if (ctx->materialized_projections.contains(projection.name)) + bool need_recalculate = + ctx->materialized_projections.contains(projection.name) + || (!is_full_part_storage && ctx->source_part->hasProjection(projection.name)); + + if (need_recalculate) { ctx->projections_to_build.push_back(&projection); } @@ -1920,9 +1999,16 @@ bool MutateTask::prepare() else /// TODO: check that we modify only non-key columns in this case. { ctx->indices_to_recalc = MutationHelpers::getIndicesToRecalculate( - ctx->mutating_pipeline_builder, ctx->metadata_snapshot, ctx->context, ctx->materialized_indices); + ctx->source_part, + ctx->mutating_pipeline_builder, + ctx->metadata_snapshot, + ctx->context, + ctx->materialized_indices); - ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate(ctx->metadata_snapshot, ctx->materialized_projections); + ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( + ctx->source_part, + ctx->metadata_snapshot, + ctx->materialized_projections); ctx->files_to_skip = MutationHelpers::collectFilesToSkip( ctx->source_part, diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index afe75349864..b1da509c9b0 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -239,7 +239,7 @@ bool StorageInMemoryMetadata::hasAnyGroupByTTL() const ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( const NameSet & updated_columns, bool include_ttl_target, - const std::function & has_indice_or_projection) const + const HasDependencyCallback & has_dependency) const { if (updated_columns.empty()) return {}; @@ -268,13 +268,13 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( for (const auto & index : getSecondaryIndices()) { - if (has_indice_or_projection("skp_idx_" + index.name + ".idx") || has_indice_or_projection("skp_idx_" + index.name + ".idx2")) + if (has_dependency(index.name, ColumnDependency::SKIP_INDEX)) add_dependent_columns(index.expression, indices_columns); } for (const auto & projection : getProjections()) { - if (has_indice_or_projection(projection.getDirectoryName())) + if (has_dependency(projection.getDirectoryName(), ColumnDependency::PROJECTION)) add_dependent_columns(&projection, projections_columns); } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 4ed7eb8bf29..30b2b303492 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -147,12 +147,14 @@ struct StorageInMemoryMetadata TTLDescriptions getGroupByTTLs() const; bool hasAnyGroupByTTL() const; + using HasDependencyCallback = std::function; + /// Returns columns, which will be needed to calculate dependencies (skip indices, projections, /// TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies( const NameSet & updated_columns, bool include_ttl_target, - const std::function & has_indice_or_projection) const; + const HasDependencyCallback & has_dependency) const; /// Block with ordinary + materialized columns. Block getSampleBlock() const; diff --git a/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference new file mode 100644 index 00000000000..f14acdf9e6d --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference @@ -0,0 +1,6 @@ +2 +0 +3355402240 +3355402240 +3321851904 +3321851904 diff --git a/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql new file mode 100644 index 00000000000..b87230e57d1 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql @@ -0,0 +1,26 @@ +set mutations_sync = 2; + +drop table if exists t_delete_skip_index; + +create table t_delete_skip_index (x UInt32, y String, index i y type minmax granularity 3) engine = MergeTree order by tuple(); +insert into t_delete_skip_index select number, toString(number) from numbers(8192 * 10); + +select count() from t_delete_skip_index where y in (4, 5); +alter table t_delete_skip_index delete where x < 8192; +select count() from t_delete_skip_index where y in (4, 5); + +drop table if exists t_delete_skip_index; +drop table if exists t_delete_projection; + +create table t_delete_projection (x UInt32, y UInt64, projection p (select sum(y))) engine = MergeTree order by tuple(); +insert into t_delete_projection select number, toString(number) from numbers(8192 * 10); + +select sum(y) from t_delete_projection settings optimize_use_projections = 0; +select sum(y) from t_delete_projection settings optimize_use_projections = 0, force_optimize_projection = 1; + +alter table t_delete_projection delete where x < 8192; + +select sum(y) from t_delete_projection settings optimize_use_projections = 0; +select sum(y) from t_delete_projection settings optimize_use_projections = 0, force_optimize_projection = 1; + +drop table if exists t_delete_projection; From 393481181d5fb7c878af2f1e0bbbd5d9c8d56807 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 24 Jul 2023 13:35:07 -0400 Subject: [PATCH 027/249] Remove Trailing whitespaces --- src/Parsers/Access/ASTCreateRowPolicyQuery.cpp | 2 +- src/Parsers/Access/ASTCreateUserQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index a423581e300..e95959703ee 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -172,7 +172,7 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IN " << (settings.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); - + formatOnCluster(settings); assert(names->cluster.empty()); diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index dbb4c1c6d57..b7290c16013 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -212,7 +212,7 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " IN " << (format.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); - + formatOnCluster(format); if (new_name) From a5d7391fbf953d6395ecc5adba48a59e1c248811 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 15:34:38 +0800 Subject: [PATCH 028/249] add new function startsWithUTF8 and endsWithUTF8 --- .../functions/string-functions.md | 28 +++++++ src/Functions/FunctionStartsEndsWith.h | 77 ++++++++++++++++--- src/Functions/endsWithUTF8.cpp | 16 ++++ src/Functions/startsWithUTF8.cpp | 16 ++++ .../02833_starts_ends_with_utf8.reference | 29 +++++++ .../02833_starts_ends_with_utf8.sql | 19 +++++ 6 files changed, 175 insertions(+), 10 deletions(-) create mode 100644 src/Functions/endsWithUTF8.cpp create mode 100644 src/Functions/startsWithUTF8.cpp create mode 100644 tests/queries/0_stateless/02833_starts_ends_with_utf8.reference create mode 100644 tests/queries/0_stateless/02833_starts_ends_with_utf8.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 9890d257e84..a398a7c0993 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -693,6 +693,22 @@ Returns whether string `str` ends with `suffix`. endsWith(str, suffix) ``` +## endsWithUTF8 + +Returns whether string `str` ends with `suffix`, assuming that both strings contain svalid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +endsWithUTF8(str, suffix) +``` + +**Example** + +``` sql +SELECT endsWithUTF8('中国', '国'); +``` + ## startsWith Returns whether string `str` starts with `prefix`. @@ -709,6 +725,18 @@ startsWith(str, prefix) SELECT startsWith('Spider-Man', 'Spi'); ``` +## startsWithUTF8 + +Returns whether string `str` starts with `prefix`, assuming that both string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + + +**Example** + +``` sql +SELECT startsWithUTF8('中国', '中'); +``` + + ## trim Removes the specified characters from the start or end of a string. If not specified otherwise, the function removes whitespace (ASCII-character 32). diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index b15bab47ae0..3565af326fe 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -28,10 +28,23 @@ namespace ErrorCodes struct NameStartsWith { static constexpr auto name = "startsWith"; + static constexpr auto is_utf8 = false; }; struct NameEndsWith { static constexpr auto name = "endsWith"; + static constexpr auto is_utf8 = false; +}; + +struct NameStartsWithUTF8 +{ + static constexpr auto name = "startsWithUTF8"; + static constexpr auto is_utf8 = true; +}; +struct NameEndsWithUTF8 +{ + static constexpr auto name = "endsWithUTF8"; + static constexpr auto is_utf8 = true; }; DECLARE_MULTITARGET_CODE( @@ -41,6 +54,7 @@ class FunctionStartsEndsWith : public IFunction { public: static constexpr auto name = Name::name; + static constexpr auto is_utf8 = Name::is_utf8; String getName() const override { @@ -64,7 +78,8 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1])) + if (!is_utf8 && isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1]) + || isString(arguments[0]) && isString(arguments[1])) return std::make_shared(); if (isArray(arguments[0]) && isArray(arguments[1])) @@ -78,8 +93,11 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { auto data_type = arguments[0].type; - if (isStringOrFixedString(*data_type)) + + if (!is_utf8 && isStringOrFixedString(*data_type)) return executeImplString(arguments, {}, input_rows_count); + if (is_utf8 && isString(*data_type)) + return executeImplStringUTF8(arguments, {}, input_rows_count); if (isArray(data_type)) return executeImplArray(arguments, {}, input_rows_count); return {}; @@ -131,7 +149,6 @@ private: typename ColumnVector::Container & vec_res = col_res->getData(); vec_res.resize(input_rows_count); - if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) dispatch(StringSource(*haystack), needle_column, vec_res); else if (const ColumnFixedString * haystack_fixed = checkAndGetColumn(haystack_column)) @@ -146,6 +163,26 @@ private: return col_res; } + ColumnPtr executeImplStringUTF8(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const + { + const IColumn * haystack_column = arguments[0].column.get(); + const IColumn * needle_column = arguments[1].column.get(); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + + vec_res.resize(input_rows_count); + if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) + dispatchUTF8(UTF8StringSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack_const = checkAndGetColumnConst(haystack_column)) + dispatchUTF8>(ConstSource(*haystack_const), needle_column, vec_res); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); + + return col_res; + } + + template void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const { @@ -161,6 +198,17 @@ private: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); } + template + void dispatchUTF8(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const + { + if (const ColumnString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, UTF8StringSource(*needle), res_data); + else if (const ColumnConst * needle_const = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle_const), res_data); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); + } + template static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) { @@ -172,18 +220,27 @@ private: auto needle = needle_source.getWhole(); if (needle.size > haystack.size) - { res_data[row_num] = false; - } else { - if constexpr (std::is_same_v) - { + if constexpr (std::is_same_v) /// startsWith res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); - } - else /// endsWith - { + else if constexpr (std::is_same_v) /// endsWith res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); + else /// startsWithUTF8 or endsWithUTF8 + { + auto length = UTF8::countCodePoints(needle.data, needle.size); + + if constexpr (std::is_same_v) + { + auto slice = haystack_source.getSliceFromLeft(0, length); + res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size); + } + else + { + auto slice = haystack_source.getSliceFromRight(length); + res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size); + } } } diff --git a/src/Functions/endsWithUTF8.cpp b/src/Functions/endsWithUTF8.cpp new file mode 100644 index 00000000000..377683ce47c --- /dev/null +++ b/src/Functions/endsWithUTF8.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionEndsWithUTF8 = FunctionStartsEndsWith; + +REGISTER_FUNCTION(EndsWithUTF8) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/startsWithUTF8.cpp b/src/Functions/startsWithUTF8.cpp new file mode 100644 index 00000000000..e623a55f4dc --- /dev/null +++ b/src/Functions/startsWithUTF8.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionStartsWithUTF8 = FunctionStartsEndsWith; + +REGISTER_FUNCTION(StartsWithUTF8) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference b/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference new file mode 100644 index 00000000000..ca2a5bc50f8 --- /dev/null +++ b/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference @@ -0,0 +1,29 @@ +-- { echoOn } +select startsWithUTF8('富强民主文明和谐', '富强'); +1 +select startsWithUTF8('富强民主文明和谐', '\xe5'); +0 +select startsWithUTF8('富强民主文明和谐', ''); +1 +SELECT startsWithUTF8('123', '123'); +1 +SELECT startsWithUTF8('123', '12'); +1 +SELECT startsWithUTF8('123', '1234'); +0 +SELECT startsWithUTF8('123', ''); +1 +select endsWithUTF8('富强民主文明和谐', '和谐'); +1 +select endsWithUTF8('富强民主文明和谐', '\x90'); +0 +select endsWithUTF8('富强民主文明和谐', ''); +1 +SELECT endsWithUTF8('123', '3'); +1 +SELECT endsWithUTF8('123', '23'); +1 +SELECT endsWithUTF8('123', '32'); +0 +SELECT endsWithUTF8('123', ''); +1 diff --git a/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql b/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql new file mode 100644 index 00000000000..3a783dc280e --- /dev/null +++ b/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql @@ -0,0 +1,19 @@ +-- { echoOn } +select startsWithUTF8('富强民主文明和谐', '富强'); +select startsWithUTF8('富强民主文明和谐', '\xe5'); +select startsWithUTF8('富强民主文明和谐', ''); + +SELECT startsWithUTF8('123', '123'); +SELECT startsWithUTF8('123', '12'); +SELECT startsWithUTF8('123', '1234'); +SELECT startsWithUTF8('123', ''); + +select endsWithUTF8('富强民主文明和谐', '和谐'); +select endsWithUTF8('富强民主文明和谐', '\x90'); +select endsWithUTF8('富强民主文明和谐', ''); + +SELECT endsWithUTF8('123', '3'); +SELECT endsWithUTF8('123', '23'); +SELECT endsWithUTF8('123', '32'); +SELECT endsWithUTF8('123', ''); +-- { echoOff } From acb33c1d810982291e7f208340dfba50d7f6c327 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 15:39:49 +0800 Subject: [PATCH 029/249] fix code style --- src/Functions/FunctionStartsEndsWith.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 3565af326fe..7671129fcfc 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -41,6 +41,7 @@ struct NameStartsWithUTF8 static constexpr auto name = "startsWithUTF8"; static constexpr auto is_utf8 = true; }; + struct NameEndsWithUTF8 { static constexpr auto name = "endsWithUTF8"; From 23897b823d7a98c122e385ed9452f8e930132825 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 20:41:16 +0800 Subject: [PATCH 030/249] fix code style --- docs/en/sql-reference/functions/string-functions.md | 2 +- .../02415_all_new_functions_must_be_documented.reference | 2 ++ utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a398a7c0993..f36de0dc0d8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -695,7 +695,7 @@ endsWith(str, suffix) ## endsWithUTF8 -Returns whether string `str` ends with `suffix`, assuming that both strings contain svalid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` ends with `suffix`, assuming that both strings contain valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. **Syntax** diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 247df64f741..d70214e9930 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -263,6 +263,7 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith +endsWithUTF8 equals erf erfc @@ -665,6 +666,7 @@ splitByString splitByWhitespace sqrt startsWith +startsWithUTF8 subBitmap substring substringUTF8 diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 65271830555..3cc25108eb1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1394,6 +1394,7 @@ encodings encryptions endian endsWith +endsWithUTF8 enum enum's enums @@ -2200,6 +2201,7 @@ src stacktrace stacktraces startsWith +startsWithUTF8 statbox stateful stddev From 27f3c5cfa27f9aef5df563a9fd90089963afe26c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Jul 2023 10:53:07 +0800 Subject: [PATCH 031/249] change as requested --- docs/en/sql-reference/functions/string-functions.md | 4 ++-- src/Functions/endsWithUTF8.cpp | 7 ++++++- src/Functions/startsWithUTF8.cpp | 7 ++++++- .../02415_all_new_functions_must_be_documented.reference | 2 -- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f36de0dc0d8..b1ffb32e7cd 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -695,7 +695,7 @@ endsWith(str, suffix) ## endsWithUTF8 -Returns whether string `str` ends with `suffix`, assuming that both strings contain valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters. **Syntax** @@ -727,7 +727,7 @@ SELECT startsWith('Spider-Man', 'Spi'); ## startsWithUTF8 -Returns whether string `str` starts with `prefix`, assuming that both string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters. **Example** diff --git a/src/Functions/endsWithUTF8.cpp b/src/Functions/endsWithUTF8.cpp index 377683ce47c..1b042452298 100644 --- a/src/Functions/endsWithUTF8.cpp +++ b/src/Functions/endsWithUTF8.cpp @@ -10,7 +10,12 @@ using FunctionEndsWithUTF8 = FunctionStartsEndsWith; REGISTER_FUNCTION(EndsWithUTF8) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters. + )", + .examples{{"endsWithUTF8", "select endsWithUTF8('富强民主文明和谐', '富强');", ""}}, + .categories{"String"}}); } } diff --git a/src/Functions/startsWithUTF8.cpp b/src/Functions/startsWithUTF8.cpp index e623a55f4dc..3f46916d760 100644 --- a/src/Functions/startsWithUTF8.cpp +++ b/src/Functions/startsWithUTF8.cpp @@ -10,7 +10,12 @@ using FunctionStartsWithUTF8 = FunctionStartsEndsWith; REGISTER_FUNCTION(StartsWithUTF8) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters. + )", + .examples{{"startsWithUTF8", "select startsWithUTF8('富强民主文明和谐', '富强');", ""}}, + .categories{"String"}}); } } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index d70214e9930..247df64f741 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -263,7 +263,6 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith -endsWithUTF8 equals erf erfc @@ -666,7 +665,6 @@ splitByString splitByWhitespace sqrt startsWith -startsWithUTF8 subBitmap substring substringUTF8 From 21e12ea0842578043fd4686e16c935052480bb80 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Jul 2023 00:53:45 -0400 Subject: [PATCH 032/249] Fix tests --- src/Access/IAccessStorage.cpp | 2 +- tests/integration/test_role/test.py | 2 +- tests/queries/0_stateless/01292_create_user.reference | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 11bc97f6cc3..db73db900aa 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -180,7 +180,7 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return *insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); + return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); } std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 43a402d9fda..0e1bbea9cea 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -301,7 +301,7 @@ def test_introspection(): ) assert instance.query( - "SELECT * from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name" + "SELECT user_name, role_name, granted_role_name, granted_role_is_default, with_admin_option from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name" ) == TSV([["A", "\\N", "R1", 1, 0], ["B", "\\N", "R2", 1, 1]]) assert instance.query( diff --git a/tests/queries/0_stateless/01292_create_user.reference b/tests/queries/0_stateless/01292_create_user.reference index eb89a5ed38c..b7c30b304bf 100644 --- a/tests/queries/0_stateless/01292_create_user.reference +++ b/tests/queries/0_stateless/01292_create_user.reference @@ -106,10 +106,10 @@ CREATE USER u2_01292 DEFAULT ROLE r1_01292, r2_01292 SETTINGS readonly = 1 CREATE USER u3_01292 HOST LIKE \'%.%.myhost.com\' DEFAULT ROLE r1_01292, r2_01292 CREATE USER u4_01292 HOST LIKE \'%.%.myhost.com\' DEFAULT ROLE r1_01292, r2_01292 -- system.users -u1_01292 local directory plaintext_password {} [] ['localhost'] [] [] 1 [] [] -u2_01292 local directory no_password {} [] [] [] ['%.%.myhost.com'] 0 [] [] -u3_01292 local directory sha256_password {} ['192.169.1.1','192.168.0.0/16'] ['localhost'] [] [] 0 ['r1_01292'] [] -u4_01292 local directory double_sha1_password {} ['::/0'] [] [] [] 1 [] ['r1_01292'] +u1_01292 local_directory plaintext_password {} [] ['localhost'] [] [] 1 [] [] +u2_01292 local_directory no_password {} [] [] [] ['%.%.myhost.com'] 0 [] [] +u3_01292 local_directory sha256_password {} ['192.169.1.1','192.168.0.0/16'] ['localhost'] [] [] 0 ['r1_01292'] [] +u4_01292 local_directory double_sha1_password {} ['::/0'] [] [] [] 1 [] ['r1_01292'] -- system.settings_profile_elements \N u1_01292 \N 0 readonly 1 \N \N \N \N \N u2_01292 \N 0 \N \N \N \N \N default From d83eabcf772d942c18c8d8b2dccb43aaad1bf235 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Jul 2023 17:39:45 +0800 Subject: [PATCH 033/249] fix spell --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3cc25108eb1..240238cab12 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1394,7 +1394,7 @@ encodings encryptions endian endsWith -endsWithUTF8 +endsWithUTF enum enum's enums @@ -2201,7 +2201,7 @@ src stacktrace stacktraces startsWith -startsWithUTF8 +startsWithUTF statbox stateful stddev From 13d9952227162b63ae5e4ae98b0a40677c169b97 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 00:38:51 -0400 Subject: [PATCH 034/249] add id to insertImpl --- src/Access/AccessControl.cpp | 10 +++++--- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 16 +++--------- src/Access/DiskAccessStorage.h | 3 +-- src/Access/IAccessStorage.cpp | 25 ++++++++++++------- src/Access/IAccessStorage.h | 11 ++++++-- src/Access/MemoryAccessStorage.cpp | 14 ++--------- src/Access/MemoryAccessStorage.h | 7 +----- src/Access/MultipleAccessStorage.cpp | 11 ++++---- src/Access/MultipleAccessStorage.h | 2 +- src/Access/ReplicatedAccessStorage.cpp | 16 +++--------- src/Access/ReplicatedAccessStorage.h | 3 +-- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 4 +-- src/Parsers/Access/ParserCreateRoleQuery.cpp | 3 ++- .../Access/ParserCreateRowPolicyQuery.cpp | 4 +-- .../ParserCreateSettingsProfileQuery.cpp | 4 +-- src/Parsers/Access/ParserCreateUserQuery.cpp | 3 ++- .../Access/ParserDropAccessEntityQuery.cpp | 3 ++- .../Access/ParserMoveAccessEntityQuery.cpp | 3 ++- src/Parsers/Access/parseUserName.h | 5 ---- 20 files changed, 64 insertions(+), 85 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 22b00d5dde7..3759b6c8919 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -525,12 +525,14 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id); - if (id) + if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists)) + { changes_notifier->sendNotifications(); - return id; + return true; + } + return false; } bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists) diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 85feaa70462..60b40775a56 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index acbd7b12569..190c7567b85 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -498,20 +498,10 @@ std::optional> DiskAccessStorage::readNameWi } -std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true)) - return id; - - return std::nullopt; -} - - -bool DiskAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) +bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, write_on_disk); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true); } @@ -745,7 +735,7 @@ void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 3541c3228e3..5d94008b34f 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ private: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; @@ -53,7 +53,6 @@ private: void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS; void stopListsWritingThread(); - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk); bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index db73db900aa..468c93c1116 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -180,14 +180,21 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); + auto id = generateRandomID(); + + if (insert(id, entity, replace_if_exists, throw_if_exists)) + return id; + + return std::nullopt; } -std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) + +bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insertImpl(entity, replace_if_exists, throw_if_exists, set_id); + return insertImpl(id, entity, replace_if_exists, throw_if_exists); } + std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists) { return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists); @@ -216,16 +223,16 @@ std::vector IAccessStorage::insert(const std::vector & mu { const auto & entity = multiple_entities[i]; - std::optional id; + UUID id; if (!ids.empty()) id = ids[i]; + else + id = generateRandomID(); - auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id); - - if (new_id) + if (insertImpl(id, entity, replace_if_exists, throw_if_exists)) { successfully_inserted.push_back(entity); - new_ids.push_back(*new_id); + new_ids.push_back(id); } } return new_ids; @@ -274,7 +281,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional) +bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool) { if (isReadOnly()) throwReadonlyCannotInsert(entity->getType(), entity->getName()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 894220e95ad..b82537df639 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -132,7 +134,7 @@ public: /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); - std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); + bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); @@ -185,7 +187,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); + virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; @@ -284,4 +286,9 @@ std::vector>> IAccessStorage return entities; } +inline bool parseAccessStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) +{ + return parseIdentifierOrStringLiteral(pos, expected, storage_name); +} + } diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index 16f3b3793a0..999e2c41d0b 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -63,17 +63,7 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) - return id; - - return std::nullopt; -} - - -bool MemoryAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { std::lock_guard lock{mutex}; return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists); @@ -300,7 +290,7 @@ void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index 7c19ab9c4ab..f877860041a 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -22,11 +22,6 @@ public: const char * getStorageType() const override { return STORAGE_TYPE; } - /// Inserts an entity with a specified ID. - /// If `replace_if_exists == true` it can replace an existing entry with such ID and also remove an existing entry - /// with such name & type. - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists); - /// Removes all entities except the specified list `ids_to_keep`. /// The function skips IDs not contained in the storage. void removeAllExcept(const std::vector & ids_to_keep); @@ -44,7 +39,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 169dc889509..e6665349ae6 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -316,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { std::shared_ptr storage_for_insertion; @@ -339,13 +339,14 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en getStorageName()); } - auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id); - if (id) + if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists)) { std::lock_guard lock{mutex}; - ids_cache.set(*id, storage_for_insertion); + ids_cache.set(id, storage_for_insertion); + return true; } - return id; + + return false; } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 1d95296b796..069d414f601 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -63,7 +63,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index efa9dbbdfaa..275672da115 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -108,17 +108,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - const UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) - return id; - - return std::nullopt; -} - - -bool ReplicatedAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); const String & name = new_entity->getName(); @@ -619,7 +609,7 @@ AccessEntityPtr ReplicatedAccessStorage::tryReadEntityFromZooKeeper(const zkutil void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity) { LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName()); - memory_storage.insertWithID(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false); + memory_storage.insert(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false); } @@ -711,7 +701,7 @@ void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 0a0c53d3bc4..cddb20860f7 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,11 +46,10 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists); bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists); bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 05c2820b012..718498ed1d1 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -1,11 +1,11 @@ #include +#include #include #include #include #include #include #include -#include #include #include #include @@ -312,7 +312,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index b548c07ec70..99a97f6901d 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -111,7 +112,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index b2177ca4ade..f9e9466e35d 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -1,10 +1,10 @@ +#include #include #include #include #include #include #include -#include #include #include #include @@ -273,7 +273,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index 634ff68427a..fcb6ebd8806 100644 --- a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -1,10 +1,10 @@ +#include #include #include #include #include #include #include -#include #include #include #include @@ -132,7 +132,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index d95d41296dc..577b6772ab2 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -481,7 +482,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d5ecdf27449..7fd34a6f8e4 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -78,7 +79,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & } if (ParserKeyword{"FROM"}.ignore(pos, expected)) - parseStorageName(pos, expected, storage_name); + parseAccessStorageName(pos, expected, storage_name); if (cluster.empty()) parseOnCluster(pos, expected, cluster); diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp index 6ac41b77b84..798e200bfb3 100644 --- a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -73,7 +74,7 @@ bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } - if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name)) + if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseAccessStorageName(pos, expected, storage_name)) return false; if (cluster.empty()) diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index 64fed2ce21c..10c548d8826 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -35,9 +35,4 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro return parseUserNames(pos, expected, role_names); } -inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) -{ - return parseIdentifierOrStringLiteral(pos, expected, storage_name); -} - } From 8c14c4b5e8925c6793c04cced5ebe64668ba3944 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 14:59:28 +0000 Subject: [PATCH 035/249] Remove unneeded readBinary() specializations + update docs The default instantiation template requires is_arithmetic_v inline void readBinary(T & x, ReadBuffer & buf) { readPODBinary(x, buf); } covers Int32, Int128, Int256, UInt32, UInt128 and UInt256 cases already. --- docs/en/sql-reference/data-types/uuid.md | 24 +++++++++++------------- src/IO/ReadHelpers.h | 6 ------ 2 files changed, 11 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/data-types/uuid.md b/docs/en/sql-reference/data-types/uuid.md index b0f19f0d8be..40f756b9588 100644 --- a/docs/en/sql-reference/data-types/uuid.md +++ b/docs/en/sql-reference/data-types/uuid.md @@ -6,42 +6,42 @@ sidebar_label: UUID # UUID -A universally unique identifier (UUID) is a 16-byte number used to identify records. For detailed information about the UUID, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). +A Universally Unique Identifier (UUID) is a 16-byte value used to identify records. For detailed information about UUIDs, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). -The example of UUID type value is represented below: +While different UUID variants exist (see [here](https://datatracker.ietf.org/doc/html/draft-ietf-uuidrev-rfc4122bis)), ClickHouse does not validate that inserted UUIDs conform to a particular variant. UUIDs are internally treated as a sequence of 16 random bytes with [8-4-4-4-12 representation](https://en.wikipedia.org/wiki/Universally_unique_identifier#Textual_representation) at SQL level. + +Example UUID value: ``` text 61f0c404-5cb3-11e7-907b-a6006ad3dba0 ``` -If you do not specify the UUID column value when inserting a new record, the UUID value is filled with zero: +The default UUID is all-zero. It is used, for example, when a new record is inserted but no value for a UUID column is specified: ``` text 00000000-0000-0000-0000-000000000000 ``` -## How to Generate +## Generating UUIDs -To generate the UUID value, ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function. +ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function to generate random UUID version 4 values. ## Usage Example **Example 1** -This example demonstrates creating a table with the UUID type column and inserting a value into the table. +This example demonstrates the creation of a table with a UUID column and the insertion of a value into the table. ``` sql CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog -``` -``` sql INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1' -``` -``` sql SELECT * FROM t_uuid ``` +Result: + ``` text ┌────────────────────────────────────x─┬─y─────────┐ │ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │ @@ -50,13 +50,11 @@ SELECT * FROM t_uuid **Example 2** -In this example, the UUID column value is not specified when inserting a new record. +In this example, no UUID column value is specified when the record is inserted, i.e. the default UUID value is inserted: ``` sql INSERT INTO t_uuid (y) VALUES ('Example 2') -``` -``` sql SELECT * FROM t_uuid ``` diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 2636898c1b3..1fe08a738c2 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1096,12 +1096,6 @@ inline void readBinary(bool & x, ReadBuffer & buf) } inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); } -inline void readBinary(Int32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(Int128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } From 07147ef88805ca09b5cd7120ecf60d71c1d94d55 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 15:24:28 +0000 Subject: [PATCH 036/249] Remove remainders of legacy setting 'allow_experimental_query_cache' --- src/Core/Settings.h | 1 - ...726_set_allow_experimental_query_cache_as_obsolete.reference | 0 .../02726_set_allow_experimental_query_cache_as_obsolete.sql | 2 -- 3 files changed, 3 deletions(-) delete mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference delete mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..e7fd28476d9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -808,7 +808,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_pool_size, 16) \ diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql deleted file mode 100644 index 244ba4e959a..00000000000 --- a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql +++ /dev/null @@ -1,2 +0,0 @@ -SET allow_experimental_query_cache = 0; -SET allow_experimental_query_cache = 1; From b98cb7fa145d1a92c2c78421be1eeb8fe8353d53 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 17:50:39 +0000 Subject: [PATCH 037/249] Fix build --- src/IO/ReadHelpers.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 1fe08a738c2..bbb1c517d7c 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1101,6 +1101,9 @@ inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(UUID & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } From 9dbb106dc14a085013140427401a235ae2115e47 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 23:37:09 -0400 Subject: [PATCH 038/249] Fix tests, docs --- .../sql-reference/statements/create/quota.md | 1 + .../sql-reference/statements/create/role.md | 1 + .../statements/create/row-policy.md | 1 + .../statements/create/settings-profile.md | 1 + .../sql-reference/statements/create/user.md | 1 + docs/en/sql-reference/statements/drop.md | 10 +++--- docs/en/sql-reference/statements/move.md | 32 +++++++++++++++++ src/Access/IAccessStorage.cpp | 15 +++++--- src/Access/MultipleAccessStorage.cpp | 4 +-- .../InterpreterMoveAccessEntityQuery.cpp | 7 ++-- .../test.py | 11 +++--- tests/integration/test_quota/test.py | 36 +++++++++---------- 12 files changed, 81 insertions(+), 39 deletions(-) create mode 100644 docs/en/sql-reference/statements/move.md diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index c69285171ab..a6ced870c18 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -11,6 +11,7 @@ Syntax: ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] + [IN access_storage_type] [KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED] [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} {MAX { {queries | query_selects | query_inserts | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] | diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md index 9b14e220e1f..4b6fffe4f60 100644 --- a/docs/en/sql-reference/statements/create/role.md +++ b/docs/en/sql-reference/statements/create/role.md @@ -11,6 +11,7 @@ Syntax: ``` sql CREATE ROLE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] + [IN access_storage_type] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | PROFILE 'profile_name'] [,...] ``` diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 83bb2e6bb9a..cd7718793bd 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -16,6 +16,7 @@ Syntax: ``` sql CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluster_name1] ON [db1.]table1|db1.* [, policy_name2 [ON CLUSTER cluster_name2] ON [db2.]table2|db2.* ...] + [IN access_storage_type] [FOR SELECT] USING condition [AS {PERMISSIVE | RESTRICTIVE}] [TO {role1 [, role2 ...] | ALL | ALL EXCEPT role1 [, role2 ...]}] diff --git a/docs/en/sql-reference/statements/create/settings-profile.md b/docs/en/sql-reference/statements/create/settings-profile.md index 8e221a4d82f..d8afce9d6de 100644 --- a/docs/en/sql-reference/statements/create/settings-profile.md +++ b/docs/en/sql-reference/statements/create/settings-profile.md @@ -12,6 +12,7 @@ Syntax: ``` sql CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] + [IN access_storage_type] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | INHERIT 'profile_name'] [,...] ``` diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index d168be63c36..11d4eae7bc8 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -14,6 +14,7 @@ CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + [IN access_storage_type] [DEFAULT ROLE role [,...]] [DEFAULT DATABASE database | NONE] [GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]] diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index b6208c2fd52..245fd22e57c 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -49,7 +49,7 @@ Deletes a user. Syntax: ``` sql -DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP ROLE @@ -59,7 +59,7 @@ Deletes a role. The deleted role is revoked from all the entities where it was a Syntax: ``` sql -DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP ROW POLICY @@ -69,7 +69,7 @@ Deletes a row policy. Deleted row policy is revoked from all the entities where Syntax: ``` sql -DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] +DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP QUOTA @@ -79,7 +79,7 @@ Deletes a quota. The deleted quota is revoked from all the entities where it was Syntax: ``` sql -DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP SETTINGS PROFILE @@ -89,7 +89,7 @@ Deletes a settings profile. The deleted settings profile is revoked from all the Syntax: ``` sql -DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP VIEW diff --git a/docs/en/sql-reference/statements/move.md b/docs/en/sql-reference/statements/move.md new file mode 100644 index 00000000000..fac738ff711 --- /dev/null +++ b/docs/en/sql-reference/statements/move.md @@ -0,0 +1,32 @@ +--- +slug: /en/sql-reference/statements/move +sidebar_position: 54 +sidebar_label: MOVE +--- + +# MOVE access entity statement + +This statement allows to move an access entity from one access storage to another. + +Syntax: + +```sql +MOVE {USER, ROLE, QUOTA, SETTINGS PROFILE, ROW POLICY} name1 [, name2, ...] TO access_storage_type +``` + +Currently, there are five access storages in ClickHouse: + - `local_directory` + - `memory` + - `replicated` + - `users_xml` (ro) + - `ldap` (ro) + +Examples: + +```sql +MOVE USER test TO local_directory +``` + +```sql +MOVE ROLE test TO memory +``` diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 468c93c1116..ec25fd505f0 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -202,16 +202,21 @@ std::vector IAccessStorage::insert(const std::vector & mu std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists) { - if (!ids.empty()) - assert(multiple_entities.size() == ids.size()); + assert(ids.empty() || (multiple_entities.size() == ids.size())); if (multiple_entities.empty()) return {}; if (multiple_entities.size() == 1) { - if (auto id = insert(multiple_entities[0], replace_if_exists, throw_if_exists)) - return {*id}; + UUID id; + if (!ids.empty()) + id = ids[0]; + else + id = generateRandomID(); + + if (insert(id, multiple_entities[0], replace_if_exists, throw_if_exists)) + return {id}; return {}; } @@ -229,7 +234,7 @@ std::vector IAccessStorage::insert(const std::vector & mu else id = generateRandomID(); - if (insertImpl(id, entity, replace_if_exists, throw_if_exists)) + if (insert(id, entity, replace_if_exists, throw_if_exists)) { successfully_inserted.push_back(entity); new_ids.push_back(id); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index e6665349ae6..c6c9ede9e1c 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -230,8 +230,8 @@ StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, co void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name) { - auto source_storage = findStorageByName(source_storage_name); - auto destination_storage = findStorageByName(destination_storage_name); + auto source_storage = getStorageByName(source_storage_name); + auto destination_storage = getStorageByName(destination_storage_name); auto to_move = source_storage->read(ids); source_storage->remove(ids); diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp index ac7ff4efcb6..49e90783a59 100644 --- a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -28,12 +28,9 @@ BlockIO InterpreterMoveAccessEntityQuery::execute() std::vector ids; if (query.type == AccessEntityType::ROW_POLICY) - ids = access_control.find(query.type, query.row_policy_names->toStrings()); + ids = access_control.getIDs(query.type, query.row_policy_names->toStrings()); else - ids = access_control.find(query.type, query.names); - - if (ids.empty()) - return {}; + ids = access_control.getIDs(query.type, query.names); /// Validate that all entities are from the same storage. const auto source_storage = access_control.findStorage(ids.front()); diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index 4c2696158ec..77aea411394 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -76,6 +76,13 @@ def execute_test_for_access_type(access_type: str, system_table_name: str): with pytest.raises(QueryRuntimeException): node.query(f"MOVE {access_type} test6 TO users_xml") + node.query(f"DROP {access_type} test1") + node.query(f"DROP {access_type} test2") + node.query(f"DROP {access_type} test3") + node.query(f"DROP {access_type} test4") + node.query(f"DROP {access_type} test5") + node.query(f"DROP {access_type} test6") + def test_roles(): execute_test_for_access_type("ROLE", "roles") @@ -93,10 +100,6 @@ def test_quotas(): execute_test_for_access_type("QUOTA", "quotas") -def test_row_policies(): - execute_test_for_access_type("ROW POLICY", "row_policies") - - def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 4f98b9a0d0d..cec14b0af73 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -105,7 +105,7 @@ def test_quota_from_users_xml(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -252,7 +252,7 @@ def test_simpliest_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[]", 0, @@ -326,7 +326,7 @@ def test_tracking_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -446,7 +446,7 @@ def test_exceed_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -523,7 +523,7 @@ def test_exceed_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -613,7 +613,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -675,7 +675,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952,63113904]", 0, @@ -824,7 +824,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -914,7 +914,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[]", 0, @@ -986,7 +986,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -1048,7 +1048,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -1111,7 +1111,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1121,7 +1121,7 @@ def test_add_remove_quota(): [ "myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", - "users.xml", + "users_xml", "['client_key','user_name']", "[3600,2629746]", 0, @@ -1214,7 +1214,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1283,7 +1283,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1346,7 +1346,7 @@ def test_reload_users_xml_by_timer(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1385,7 +1385,7 @@ def test_reload_users_xml_by_timer(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", ["user_name"], "[31556952]", 0, @@ -1554,7 +1554,7 @@ def test_query_inserts(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, From f3a31c20fe8fd4a0ba57586bd36828af8581cf5d Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 10:53:02 -0400 Subject: [PATCH 039/249] Update test_row_policy.py --- .../test_row_policy.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py index 509b4de1a37..b620e88e7eb 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py +++ b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py @@ -76,7 +76,7 @@ def test_introspection(): "mydb", "filtered_table1", "6068883a-0e9d-f802-7e22-0144f8e66d3c", - "users.xml", + "users_xml", "1", 0, 0, @@ -89,7 +89,7 @@ def test_introspection(): "mydb", "filtered_table2", "c019e957-c60b-d54e-cc52-7c90dac5fb01", - "users.xml", + "users_xml", "1", 0, 0, @@ -102,7 +102,7 @@ def test_introspection(): "mydb", "filtered_table3", "4cb080d0-44e8-dbef-6026-346655143628", - "users.xml", + "users_xml", "1", 0, 0, @@ -115,7 +115,7 @@ def test_introspection(): "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -128,7 +128,7 @@ def test_introspection(): "mydb", "filtered_table1", "9e8a8f62-4965-2b5e-8599-57c7b99b3549", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -141,7 +141,7 @@ def test_introspection(): "mydb", "filtered_table2", "cffae79d-b9bf-a2ef-b798-019c18470b25", - "users.xml", + "users_xml", "a + b < 1 or c - d > 5", 0, 0, @@ -154,7 +154,7 @@ def test_introspection(): "mydb", "filtered_table3", "12fc5cef-e3da-3940-ec79-d8be3911f42b", - "users.xml", + "users_xml", "c = 1", 0, 0, @@ -167,7 +167,7 @@ def test_introspection(): "mydb", "local", "cdacaeb5-1d97-f99d-2bb0-4574f290629c", - "users.xml", + "users_xml", "1", 0, 0, From f42cfddee7966496836da44e26d27fe208d43ac9 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 20:28:56 -0400 Subject: [PATCH 040/249] Update test.py --- tests/integration/test_row_policy/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 1933823f5d2..acf544c98aa 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -420,7 +420,7 @@ def test_introspection(): "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -433,7 +433,7 @@ def test_introspection(): "mydb", "filtered_table1", "9e8a8f62-4965-2b5e-8599-57c7b99b3549", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -446,7 +446,7 @@ def test_introspection(): "mydb", "filtered_table2", "cffae79d-b9bf-a2ef-b798-019c18470b25", - "users.xml", + "users_xml", "a + b < 1 or c - d > 5", 0, 0, @@ -459,7 +459,7 @@ def test_introspection(): "mydb", "filtered_table3", "12fc5cef-e3da-3940-ec79-d8be3911f42b", - "users.xml", + "users_xml", "c = 1", 0, 0, From 1ee1ae120e921a355a752b332aae25b18b1cf89d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 02:03:21 +0000 Subject: [PATCH 041/249] add modulo, intDiv, intDivOrZero for tuple --- src/Functions/FunctionBinaryArithmetic.h | 24 ++++++++++++++++--- src/Functions/IsOperation.h | 2 +- src/Functions/vectorFunctions.cpp | 21 ++++++++++++++++ .../0_stateless/02841_tuple_modulo.reference | 4 ++++ .../0_stateless/02841_tuple_modulo.sql | 4 ++++ 5 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02841_tuple_modulo.reference create mode 100644 tests/queries/0_stateless/02841_tuple_modulo.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index bf3b33d13ff..408a16236e8 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -742,6 +742,9 @@ class FunctionBinaryArithmetic : public IFunction static constexpr bool is_multiply = IsOperation::multiply; static constexpr bool is_division = IsOperation::division; static constexpr bool is_bit_hamming_distance = IsOperation::bit_hamming_distance; + static constexpr bool is_modulo = IsOperation::modulo; + static constexpr bool is_div_int = IsOperation::div_int; + static constexpr bool is_div_int_or_zero = IsOperation::div_int_or_zero; ContextPtr context; bool check_decimal_overflow = true; @@ -951,13 +954,28 @@ class FunctionBinaryArithmetic : public IFunction "argument of numeric type cannot be first", name); std::string function_name; - if (is_multiply) + if constexpr (is_multiply) { function_name = "tupleMultiplyByNumber"; } - else + else // is_division { - function_name = "tupleDivideByNumber"; + if constexpr (is_modulo) + { + function_name = "tupleModuloByNumber"; + } + else if constexpr (is_div_int) + { + function_name = "tupleIntDivByNumber"; + } + else if constexpr (is_div_int_or_zero) + { + function_name = "tupleIntDivOrZeroByNumber"; + } + else + { + function_name = "tupleDivideByNumber"; + } } return FunctionFactory::instance().get(function_name, context); diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 0c54901579e..8ea53c865ce 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -60,7 +60,7 @@ struct IsOperation static constexpr bool bit_hamming_distance = IsSameOperation::value; - static constexpr bool division = div_floating || div_int || div_int_or_zero; + static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index d53d39e2f3b..35ba49e4545 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -23,6 +23,9 @@ struct PlusName { static constexpr auto name = "plus"; }; struct MinusName { static constexpr auto name = "minus"; }; struct MultiplyName { static constexpr auto name = "multiply"; }; struct DivideName { static constexpr auto name = "divide"; }; +struct ModuloName { static constexpr auto name = "modulo"; }; +struct IntDivName { static constexpr auto name = "intDiv"; }; +struct IntDivOrZeroName { static constexpr auto name = "intDivOrZero"; }; struct L1Label { static constexpr auto name = "1"; }; struct L2Label { static constexpr auto name = "2"; }; @@ -141,6 +144,12 @@ using FunctionTupleMultiply = FunctionTupleOperator; using FunctionTupleDivide = FunctionTupleOperator; +using FunctionTupleModulo = FunctionTupleOperator; + +using FunctionTupleIntDiv = FunctionTupleOperator; + +using FunctionTupleIntDivOrZero = FunctionTupleOperator; + class FunctionTupleNegate : public ITupleFunction { public: @@ -297,6 +306,12 @@ using FunctionTupleMultiplyByNumber = FunctionTupleOperatorByNumber; +using FunctionTupleModuloByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivOrZeroByNumber = FunctionTupleOperatorByNumber; + class FunctionDotProduct : public ITupleFunction { public: @@ -1563,6 +1578,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerAlias("vectorDifference", FunctionTupleMinus::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(FunctionDocumentation @@ -1626,6 +1644,9 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); diff --git a/tests/queries/0_stateless/02841_tuple_modulo.reference b/tests/queries/0_stateless/02841_tuple_modulo.reference new file mode 100644 index 00000000000..6e6f07d0683 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.reference @@ -0,0 +1,4 @@ +(1,0) +(2,2) +(2,2) +(0,0) diff --git a/tests/queries/0_stateless/02841_tuple_modulo.sql b/tests/queries/0_stateless/02841_tuple_modulo.sql new file mode 100644 index 00000000000..56bacf87967 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.sql @@ -0,0 +1,4 @@ +SELECT (5,4) % 2; +SELECT intDiv((5,4), 2); +SELECT intDivOrZero((5,4), 2); +SELECT intDivOrZero((5,4), 0); From 6f2404b79209bf332e54e83b86eff2f677bbd2bc Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 22:18:53 -0400 Subject: [PATCH 042/249] Additional info in exception --- src/Access/MultipleAccessStorage.cpp | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index c6c9ede9e1c..0550c140c17 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -234,17 +234,32 @@ void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, co auto destination_storage = getStorageByName(destination_storage_name); auto to_move = source_storage->read(ids); - source_storage->remove(ids); + bool need_rollback = false; try { + source_storage->remove(ids); + need_rollback = true; destination_storage->insert(to_move, ids); } catch (Exception & e) { - e.addMessage("while moving access entities"); + String message; + + bool need_comma = false; + for (const auto & entity : to_move) + { + if (std::exchange(need_comma, true)) + message += ", "; + + message += entity->formatTypeWithName(); + } + + e.addMessage("while moving {} from {} to {}", message, source_storage_name, destination_storage_name); + + if (need_rollback) + source_storage->insert(to_move, ids); - source_storage->insert(to_move, ids); throw; } } From a64d6b11f29c3eac5eedf4e48fc118510d4e2a01 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 04:55:19 +0000 Subject: [PATCH 043/249] add functions to undocumented --- .../02415_all_new_functions_must_be_documented.reference | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..b57372ffa8d 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -885,7 +885,13 @@ tupleDivide tupleDivideByNumber tupleElement tupleHammingDistance +tupleIntDiv +tupleIntDivByNumber +tupleIntDivOrZero +tupleIntDivOrZeroByNumber tupleMinus +tupleModulo +tupleModuloByNumber tupleMultiply tupleMultiplyByNumber tupleNegate From 12069b2cff6c4ae77cc674f02ce4e04987c9515b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Jul 2023 23:25:44 +0200 Subject: [PATCH 044/249] Allow yaml configs in clickhouse-client --- src/Common/Config/configReadClient.cpp | 51 ++++++++++++++++++-------- 1 file changed, 36 insertions(+), 15 deletions(-) diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 44d338c07af..fea055618bb 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -6,36 +6,57 @@ #include #include + namespace fs = std::filesystem; namespace DB { -bool safeFsExists(const String & path) -{ - std::error_code ec; - return fs::exists(path, ec); -} - bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path) { std::string config_path; - if (config.has("config-file")) - config_path = config.getString("config-file"); - else if (safeFsExists("./clickhouse-client.xml")) - config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && safeFsExists(home_path + "/.clickhouse-client/config.xml")) - config_path = home_path + "/.clickhouse-client/config.xml"; - else if (safeFsExists("/etc/clickhouse-client/config.xml")) - config_path = "/etc/clickhouse-client/config.xml"; - if (!config_path.empty()) + bool found = false; + if (config.has("config-file")) + { + found = true; + config_path = config.getString("config-file"); + } + else + { + std::vector names; + names.emplace_back("./clickhouse-client"); + if (!home_path.empty()) + names.emplace_back(home_path + "/.clickhouse-client/config"); + names.emplace_back("/etc/clickhouse-client/config"); + + for (const auto & name : names) + { + for (const auto & extension : {".xml", ".yaml", ".yml"}) + { + config_path = name + extension; + + std::error_code ec; + if (fs::exists(config_path, ec)) + { + found = true; + break; + } + } + if (found) + break; + } + } + + if (found) { ConfigProcessor config_processor(config_path); auto loaded_config = config_processor.loadConfig(); config.add(loaded_config.configuration); return true; } + return false; } + } From fdfe64f1b0330522180811b83e8db8436839c829 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 31 Jul 2023 12:07:50 +0800 Subject: [PATCH 045/249] change as request --- .../functions/string-functions.md | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index b1ffb32e7cd..fff37257917 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -706,7 +706,15 @@ endsWithUTF8(str, suffix) **Example** ``` sql -SELECT endsWithUTF8('中国', '国'); +SELECT endsWithUTF8('中国', '\xbd'), endsWith('中国', '\xbd') +``` + +Result: + +```result +┌─endsWithUTF8('中国', '½')─┬─endsWith('中国', '½')─┐ +│ 0 │ 1 │ +└──────────────────────────┴──────────────────────┘ ``` ## startsWith @@ -733,9 +741,16 @@ Returns whether string `str` starts with `prefix`, the difference between `start **Example** ``` sql -SELECT startsWithUTF8('中国', '中'); +SELECT startsWithUTF8('中国', '\xe4'), startsWith('中国', '\xe4') ``` +Result: + +```result +┌─startsWithUTF8('中国', '⥩─┬─startsWith('中国', '⥩─┐ +│ 0 │ 1 │ +└────────────────────────────┴────────────────────────┘ +``` ## trim From fb8502ba7671f75cb420ab5937cced224f5be7c6 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 31 Jul 2023 12:20:27 +0000 Subject: [PATCH 046/249] do not test upper bounds for throttlers --- tests/integration/test_throttling/test.py | 45 ++++++++++++----------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 62640394a85..526c34ac916 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -117,7 +117,8 @@ def assert_took(took, should_took): # we need to decrease the lower limit because the server limits could # be enforced by throttling some server background IO instead of query IO # and we have no control over it - assert took >= should_took[0] * 0.85 and took < should_took[1] + # Note that throttler does not apply any restrictions on upper bound, so we can only tell how much time required "at least", not "at most" + assert took >= should_took * 0.85 @pytest.mark.parametrize( @@ -132,7 +133,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -142,7 +143,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -152,7 +153,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -164,7 +165,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -184,7 +185,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_to_local_throttling", ), # @@ -196,7 +197,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -206,7 +207,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (0, 3), + 0, id="user_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -216,7 +217,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (0, 3), + 0, id="server_remote_to_remote_throttling", ), # @@ -233,7 +234,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_to_remote_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds, but for S3Client it is 2x more @@ -243,7 +244,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7 * 3, 7 * 4 - 1), + 7 * 3, id="user_local_to_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds, but for S3Client it is 2x more @@ -253,7 +254,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3 * 3, 3 * 5), + 3 * 3, id="server_local_to_remote_throttling", ), ], @@ -306,7 +307,7 @@ def test_backup_throttling_override(): "user", "max_local_read_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -315,7 +316,7 @@ def test_backup_throttling_override(): "server", "max_local_read_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -328,7 +329,7 @@ def test_backup_throttling_override(): "user", "max_remote_read_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -337,7 +338,7 @@ def test_backup_throttling_override(): "server", "max_remote_read_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], @@ -368,7 +369,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): "user", "max_local_write_bandwidth", "1M", - (7, 14), + 7, id="local_user_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -377,29 +378,29 @@ def test_read_throttling(policy, mode, setting, value, should_took): "server", "max_local_write_bandwidth_for_server", "2M", - (3, 7), + 3, id="local_server_throttling", ), # # Remote # pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), - # writeing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", "user", "max_remote_write_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), - # writeing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # writing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds pytest.param( "s3", "server", "max_remote_write_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], From a0ff04e0e7cc7bd911d80a01aee34846ec26bb3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 08:03:09 +0200 Subject: [PATCH 047/249] Add a test --- .../02834_client_yaml_configs.reference | 3 +++ .../0_stateless/02834_client_yaml_configs.sh | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02834_client_yaml_configs.reference create mode 100755 tests/queries/0_stateless/02834_client_yaml_configs.sh diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference new file mode 100644 index 00000000000..302360f2570 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -0,0 +1,3 @@ +31337 +31337 +31337 diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh new file mode 100755 index 00000000000..f17186328b4 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +pushd "${CLICKHOUSE_TMP}" > /dev/null + +echo "max_block_size: 31337" > clickhouse-client.yaml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yaml + +echo "max_block_size: 31337" > clickhouse-client.yml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yml + +echo "31337" > clickhouse-client.xml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.xml + +popd > /dev/null From 628786e7eb6a55d5f5b2881a0fc34c3edcf7d0e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 10:52:06 +0300 Subject: [PATCH 048/249] Update 02834_client_yaml_configs.sh --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index f17186328b4..5a6d26808a3 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8b37abfa598b80bf9a1281d519f09d953e18523e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 11:16:00 +0300 Subject: [PATCH 049/249] Update 02834_client_yaml_configs.sh --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index 5a6d26808a3..dbb40d33e0a 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -pushd "${CLICKHOUSE_TMP}" > /dev/null +pushd "${CLICKHOUSE_TMP}" || exit > /dev/null echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" @@ -19,4 +19,4 @@ echo "31337" > clickho ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml -popd > /dev/null +popd || exit > /dev/null From 513d3fd3c1c1484cc908987f135f153f59d36405 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Aug 2023 09:15:35 +0000 Subject: [PATCH 050/249] fix more places --- tests/integration/test_throttling/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 526c34ac916..91b472513c7 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -175,7 +175,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_remote_to_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -291,7 +291,7 @@ def test_backup_throttling_override(): }, ) # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds - assert_took(took, (15, 20)) + assert_took(took, 15) @pytest.mark.parametrize( From 135790c0d6caed1075399cab69a5036c71860cd8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Aug 2023 09:44:08 +0000 Subject: [PATCH 051/249] more fixes --- tests/integration/test_throttling/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 91b472513c7..a27bb472ea8 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -300,7 +300,7 @@ def test_backup_throttling_override(): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", @@ -322,7 +322,7 @@ def test_backup_throttling_override(): # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", @@ -362,7 +362,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", @@ -384,7 +384,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", From 8646eefc9cfeedc840c14bc7571c0c6490679a4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:05:24 +0300 Subject: [PATCH 052/249] Update docker_test.py --- tests/ci/docker_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 8542de412d2..bdbee92d2d3 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -43,7 +43,7 @@ class TestDockerImageCheck(unittest.TestCase): di.DockerImage( "docker/test/sqltest", "clickhouse/sqltest", - True, + False, "clickhouse/test-base", # type: ignore ), di.DockerImage( From 07790ab485012f844a7efafa1729ec901df79c0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:40:55 +0300 Subject: [PATCH 053/249] Update tests/queries/0_stateless/02834_client_yaml_configs.sh MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index dbb40d33e0a..d1c5a40c04f 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -11,11 +11,11 @@ echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.yaml -echo "max_block_size: 31337" > clickhouse-client.yml +echo "max_block_size: 31338" > clickhouse-client.yml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.yml -echo "31337" > clickhouse-client.xml +echo "31339" > clickhouse-client.xml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml From 93d766ba74f22f7de2dbbc3f196be06af91614d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:41:00 +0300 Subject: [PATCH 054/249] Update tests/queries/0_stateless/02834_client_yaml_configs.reference MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02834_client_yaml_configs.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference index 302360f2570..b2eddb19e52 100644 --- a/tests/queries/0_stateless/02834_client_yaml_configs.reference +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -1,3 +1,3 @@ 31337 -31337 -31337 +31338 +31339 From 17c4abce10c03bafc5316cde3f80a163fa56a944 Mon Sep 17 00:00:00 2001 From: Kenji Noguchi Date: Tue, 1 Aug 2023 10:44:43 -0700 Subject: [PATCH 055/249] CVE-2016-2183: disable 3DES --- base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h | 6 +++--- base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h | 2 +- base/poco/NetSSL_OpenSSL/src/Context.cpp | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h index 65917ac9dd4..c19eecf5c73 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h @@ -146,7 +146,7 @@ namespace Net std::string cipherList; /// Specifies the supported ciphers in OpenSSL notation. - /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH". + /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH". std::string dhParamsFile; /// Specifies a file containing Diffie-Hellman parameters. @@ -172,7 +172,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. @@ -200,7 +200,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index 21a1ed685e5..e4037c87927 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -76,7 +76,7 @@ namespace Net /// none|relaxed|strict|once /// 1..9 /// true|false - /// ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH + /// ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH /// true|false /// /// KeyFileHandler diff --git a/base/poco/NetSSL_OpenSSL/src/Context.cpp b/base/poco/NetSSL_OpenSSL/src/Context.cpp index ca220c40a33..d0bab902b89 100644 --- a/base/poco/NetSSL_OpenSSL/src/Context.cpp +++ b/base/poco/NetSSL_OpenSSL/src/Context.cpp @@ -41,7 +41,7 @@ Context::Params::Params(): verificationMode(VERIFY_RELAXED), verificationDepth(9), loadDefaultCAs(false), - cipherList("ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH") + cipherList("ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH") { } diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5187ccce789..a7637082496 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1640,7 +1640,7 @@ Keys for server/client settings: - verificationMode (default: relaxed) – The method for checking the node’s certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`. - verificationDepth (default: 9) – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile (default: true) – Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`). -- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions. +- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`) - Supported OpenSSL encryptions. - cacheSessions (default: false) – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`. - sessionIdContext (default: `${application.name}`) – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`. - sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) – The maximum number of sessions that the server caches. A value of 0 means unlimited sessions. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 81a696bcfc1..7b026244624 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1106,7 +1106,7 @@ ClickHouse использует потоки из глобального пул - verificationMode - Способ проверки сертификатов узла. Подробности находятся в описании класса [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h). Допустимые значения: `none`, `relaxed`, `strict`, `once`. - verificationDepth - Максимальная длина верификационной цепи. Верификация завершится ошибкой, если длина цепи сертификатов превысит установленное значение. - loadDefaultCAFile - Признак того, что будут использоваться встроенные CA-сертификаты для OpenSSL. Допустимые значения: `true`, `false`. \| -- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions - Включение/выключение кеширования сессии. Использовать обязательно вместе с `sessionIdContext`. Допустимые значения: `true`, `false`. - sessionIdContext - Уникальный набор произвольных символов, которые сервер добавляет к каждому сгенерированному идентификатору. Длина строки не должна превышать `SSL_MAX_SSL_SESSION_ID_LENGTH`. Рекомендуется к использованию всегда, поскольку позволяет избежать проблем как в случае, если сервер кеширует сессию, так и если клиент затребовал кеширование. По умолчанию `${application.name}`. - sessionCacheSize - Максимальное количество сессий, которые кэширует сервер. По умолчанию - 1024\*20. 0 - неограниченное количество сессий. diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index f6106d8734e..8e2cb389f04 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -455,7 +455,7 @@ SSL客户端/服务器配置。 - verificationMode – The method for checking the node’s certificates. Details are in the description of the [A.背景](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) 同学们 可能的值: `none`, `relaxed`, `strict`, `once`. - verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. \| -- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. 可接受的值: `true`, `false`. - sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. 始终建议使用此参数,因为如果服务器缓存会话,以及客户端请求缓存,它有助于避免出现问题。 默认值: `${application.name}`. - sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. From 0641dfd7e640bfd08d8533d41cb8ada524dd7b3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 22:33:04 +0200 Subject: [PATCH 056/249] Add missing modification --- tests/ci/ci_config.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 517e40fd2d6..be8364f8874 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -402,6 +402,9 @@ CI_CONFIG = { "Sqllogic test (release)": { "required_build": "package_release", }, + "SQLTest": { + "required_build": "package_release", + }, }, } # type: dict From 469dd7f30033b4e7870d6999825f8a16fe106e59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 19:06:31 +0200 Subject: [PATCH 057/249] Add the docs --- docs/en/interfaces/cli.md | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 8779dd1a544..36afb94433a 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -323,9 +323,9 @@ clickhouse-client clickhouse://192.168.1.15,192.168.1.25 `clickhouse-client` uses the first existing file of the following: - Defined in the `--config-file` parameter. -- `./clickhouse-client.xml` -- `~/.clickhouse-client/config.xml` -- `/etc/clickhouse-client/config.xml` +- `./clickhouse-client.xml`, `.yaml`, `.yml` +- `~/.clickhouse-client/config.xml`, `.yaml`, `.yml` +- `/etc/clickhouse-client/config.xml`, `.yaml`, `.yml` Example of a config file: @@ -342,6 +342,17 @@ Example of a config file: ``` +Or the same config in a YAML format: + +```yaml +user: username +password: 'password' +secure: true +openSSL: + client: + caConfig: '/etc/ssl/cert.pem' +``` + ### Query ID Format {#query-id-format} In interactive mode `clickhouse-client` shows query ID for every query. By default, the ID is formatted like this: From b4f0d0bc2edb5865128e16dd90db00d25df8747e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 21:05:39 +0300 Subject: [PATCH 058/249] Update pull_request.yml --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1d515633f61..6ec8c003491 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5194,7 +5194,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/sqltest REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME="SQLTest" + CHECK_NAME=SQLTest REPO_COPY=${{runner.temp}}/sqltest/ClickHouse EOF - name: Download json reports From 5ec6a4695f2911758e8d64ee52e34149c4940112 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:22:15 +0200 Subject: [PATCH 059/249] Fix log paths --- tests/ci/sqltest.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index b752d4e4aee..7385716850f 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,11 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "main.log": os.path.join(workspace_path, "main.log"), - "server.log.zst": os.path.join(workspace_path, "server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), - "report.html": os.path.join(workspace_path, "report.html"), - "test.log": os.path.join(workspace_path, "test.log"), + "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/report.html"), + "test.log": os.path.join(workspace_path, "/test.log"), } s3_helper = S3Helper() From 619af0c2c66255e8c8aa4f8fd62b96c94fe35ee3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 21:46:23 +0000 Subject: [PATCH 060/249] Automatic style fix --- tests/ci/sqltest.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..498448a7928 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,8 +110,12 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "server.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.log.zst" + ), + "server.err.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/report.html"), "test.log": os.path.join(workspace_path, "/test.log"), } From 99c7f7b48ca61af6077fd99431194bde24f761b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:50:19 +0200 Subject: [PATCH 061/249] Fix the test --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index d1c5a40c04f..66d3df8829e 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -pushd "${CLICKHOUSE_TMP}" || exit > /dev/null +pushd "${CLICKHOUSE_TMP}" > /dev/null || exit echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" @@ -19,4 +19,4 @@ echo "31339" > clickho ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml -popd || exit > /dev/null +popd > /dev/null || exit From dadf4678ee9d678459c96cd13b8231e33dc02cc3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 12:50:40 +0000 Subject: [PATCH 062/249] Automatic style fix --- tests/integration/test_dictionary_custom_settings/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 6b5ea643998..eb394da8bb6 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -82,5 +82,9 @@ def test_work(start_cluster): # It is possible that the HTTP server takes long time to start accepting connections - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n") - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n') + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n" + ) + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n' + ) From 65e8ee8fb7d30b0765c8bfd3d58d9def64e002bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 15:28:18 +0200 Subject: [PATCH 063/249] Fix paths --- docker/test/sqltest/run.sh | 4 +++- tests/ci/sqltest.py | 8 ++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 42aeef9df15..cba1c1dab1f 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -42,8 +42,10 @@ done # Run the test pushd sqltest/standards/2016/ /test.py -mv report.html test.log / +mv report.html test.log /workspace popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log + +mv /var/log/clickhouse-server/clickhouse-server.log.zst /var/log/clickhouse-server/clickhouse-server.err.log.zst /workspace diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..d76baf22bb3 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/report.html"), - "test.log": os.path.join(workspace_path, "/test.log"), + "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/workspace/report.html"), + "test.log": os.path.join(workspace_path, "/workspace/test.log"), } s3_helper = S3Helper() From a96b0457e04543876b35ca1a5223ea6007bbf9bb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 13:43:45 +0000 Subject: [PATCH 064/249] Automatic style fix --- tests/ci/sqltest.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..69af21f1761 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -111,7 +111,9 @@ def main(): paths = { "run.log": run_log_path, "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "server.err.log.zst": os.path.join( + workspace_path, "/workspace/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/workspace/report.html"), "test.log": os.path.join(workspace_path, "/workspace/test.log"), } From a0fa3cc73bbbec2879873608607fc18f51a6556a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 3 Aug 2023 18:55:15 +0300 Subject: [PATCH 065/249] EXPLAIN actions for JOIN step --- src/Interpreters/ConcurrentHashJoin.h | 1 + src/Interpreters/DirectJoin.h | 1 + src/Interpreters/FullSortingMergeJoin.h | 1 + src/Interpreters/GraceHashJoin.h | 1 + src/Interpreters/HashJoin.h | 1 + src/Interpreters/IJoin.h | 2 + src/Interpreters/JoinSwitcher.h | 1 + src/Interpreters/MergeJoin.h | 1 + src/Interpreters/TableJoin.h | 2 +- src/Processors/QueryPlan/JoinStep.cpp | 33 +++++ src/Processors/QueryPlan/JoinStep.h | 3 + .../02835_join_step_explain.reference | 116 ++++++++++++++++++ .../0_stateless/02835_join_step_explain.sql | 31 +++++ 13 files changed, 193 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02835_join_step_explain.reference create mode 100644 tests/queries/0_stateless/02835_join_step_explain.sql diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 1283879971d..85e0c5a0ae7 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -36,6 +36,7 @@ public: explicit ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_ = false); ~ConcurrentHashJoin() override = default; + std::string getName() const override { return "ConcurrentHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index e55ac278705..5f664314818 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -30,6 +30,7 @@ public: std::shared_ptr storage_, const Block & right_sample_block_with_storage_column_names_); + std::string getName() const override { return "DirectKeyValueJoin"; } virtual const TableJoin & getTableJoin() const override { return *table_join; } virtual bool addBlockToJoin(const Block &, bool) override; diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a6b53a51c04..3fc9f8920ed 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -28,6 +28,7 @@ public: LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } + std::string getName() const override { return "FullSortingMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & /* block */, bool /* check_limits */) override diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ce519892b0e..44949440467 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -60,6 +60,7 @@ public: ~GraceHashJoin() override; + std::string getName() const override { return "GraceHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } void initialize(const Block & sample_block) override; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 56dea98c1f1..9f55945816c 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -151,6 +151,7 @@ public: ~HashJoin() override; + std::string getName() const override { return "HashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } /** Add block of data from right hand of JOIN to the map. diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 97b119bd795..493a5dd2126 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -48,6 +48,8 @@ class IJoin public: virtual ~IJoin() = default; + virtual std::string getName() const = 0; + virtual const TableJoin & getTableJoin() const = 0; /// Add block of data from right hand of JOIN. diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index fb5066b2d04..1d2ebc6b456 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -18,6 +18,7 @@ class JoinSwitcher : public IJoin public: JoinSwitcher(std::shared_ptr table_join_, const Block & right_sample_block_); + std::string getName() const override { return "JoinSwitcher"; } const TableJoin & getTableJoin() const override { return *table_join; } /// Add block of data from right hand of JOIN into current join object. diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 03a661c5b8a..98fae1d419f 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -22,6 +22,7 @@ class MergeJoin : public IJoin public: MergeJoin(std::shared_ptr table_join_, const Block & right_sample_block); + std::string getName() const override { return "PartialMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 5d14a57759f..75626764bda 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -331,7 +331,7 @@ public: const ColumnsWithTypeAndName & right_sample_columns); void setAsofInequality(ASOFJoinInequality inequality) { asof_inequality = inequality; } - ASOFJoinInequality getAsofInequality() { return asof_inequality; } + ASOFJoinInequality getAsofInequality() const { return asof_inequality; } ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 33fa7955e0d..858a01a437d 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include #include namespace DB @@ -62,6 +65,36 @@ void JoinStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } +void JoinStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + + const auto & table_join = join->getTableJoin(); + settings.out << prefix << "Kind: " << toString(table_join.kind()) << '\n'; + settings.out << prefix << "Strictness: " << toString(table_join.strictness()) << '\n'; + settings.out << prefix << "Type: " << join->getName() << '\n'; + + if (table_join.strictness() == JoinStrictness::Asof) + settings.out << prefix << "ASOF inequality: " << toString(table_join.getAsofInequality()) << '\n'; + + if (!table_join.getClauses().empty()) + settings.out << prefix << "Clauses: " << table_join.formatClauses(table_join.getClauses(), true /*short_format*/) << '\n'; +} + +void JoinStep::describeActions(JSONBuilder::JSONMap & map) const +{ + const auto & table_join = join->getTableJoin(); + map.add("Kind", toString(table_join.kind())); + map.add("Strictness", toString(table_join.strictness())); + map.add("Type", join->getName()); + + if (table_join.strictness() == JoinStrictness::Asof) + map.add("ASOF inequality", toString(table_join.getAsofInequality())); + + if (!table_join.getClauses().empty()) + map.add("Clauses", table_join.formatClauses(table_join.getClauses(), true /*short_format*/)); +} + void JoinStep::updateInputStream(const DataStream & new_input_stream_, size_t idx) { if (idx == 0) diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index e7185f36588..369ee9bec8b 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -27,6 +27,9 @@ public: void describePipeline(FormatSettings & settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + const JoinPtr & getJoin() const { return join; } bool allowPushDownToRight() const; diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference new file mode 100644 index 00000000000..77c91b3d7e9 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -0,0 +1,116 @@ +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT : 2 -> value_1_3 String : 2 + INPUT : 3 -> id_2 UInt64 : 3 + ALIAS id_0 :: 0 -> id UInt64 : 4 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1 + ALIAS id_2 :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_1_3 String + id_2 UInt64 + Kind: INNER + Strictness: ALL + Type: HashJoin + Clauses: [(id_0) = (id_2)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_0 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_2 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 +-- +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT :: 2 -> value_2_4 UInt64 : 2 + INPUT : 3 -> value_1_3 String : 3 + INPUT :: 4 -> value_2_5 UInt64 : 4 + INPUT : 5 -> id_2 UInt64 : 5 + ALIAS id_0 :: 0 -> id UInt64 : 6 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1 + ALIAS id_2 :: 5 -> rhs.id UInt64 : 3 +Positions: 6 0 3 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + value_1_3 String + value_2_5 UInt64 + id_2 UInt64 + Kind: INNER + Strictness: ASOF + Type: HashJoin + ASOF inequality: LESS + Clauses: [(id_0, value_2_4) = (id_2, value_2_5)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_0 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + value_2_5 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_2 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql new file mode 100644 index 00000000000..d0475fa14b6 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -0,0 +1,31 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table_1; +CREATE TABLE test_table_1 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +DROP TABLE IF EXISTS test_table_2; +CREATE TABLE test_table_2 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table_1 VALUES (0, 'Value', 0); +INSERT INTO test_table_2 VALUES (0, 'Value', 0); + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs ASOF JOIN test_table_2 AS rhs ON lhs.id = rhs.id AND lhs.value_2 < rhs.value_2; + +DROP TABLE test_table_1; +DROP TABLE test_table_2; From 388feb953d97b244b162c433e2983cbe4e2d73f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 19:28:59 +0200 Subject: [PATCH 066/249] Fix paths --- tests/ci/sqltest.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..be22a1c9312 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/workspace/report.html"), - "test.log": os.path.join(workspace_path, "/workspace/test.log"), + "server.log.zst": os.path.join(workspace_path, "server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), + "report.html": os.path.join(workspace_path, "report.html"), + "test.log": os.path.join(workspace_path, "test.log"), } s3_helper = S3Helper() From 6dc74fb5ccf0c6ef6538e60835667b57b6d4f060 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 3 Aug 2023 12:38:43 -0700 Subject: [PATCH 067/249] Implement big-endian support for transform --- .../Serializations/SerializationArray.cpp | 4 ++-- .../Serializations/SerializationNumber.cpp | 8 ++++---- src/Functions/transform.cpp | 14 ++++++++------ 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index cedcca870dd..e01c1aea0e9 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -129,7 +129,7 @@ namespace for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; - writeIntBinary(current_offset - prev_offset, ostr); + writeBinaryLittleEndian(current_offset - prev_offset, ostr); prev_offset = current_offset; } } @@ -145,7 +145,7 @@ namespace while (i < initial_size + limit && !istr.eof()) { ColumnArray::Offset current_size = 0; - readIntBinary(current_size, istr); + readBinaryLittleEndian(current_size, istr); if (unlikely(current_size > MAX_ARRAY_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size); diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..0294a1c8a67 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -106,28 +106,28 @@ void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = NearestFieldType(x); } template void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); + writeBinaryLittleEndian(assert_cast &>(column).getData()[row_num], ostr); } template void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index e03701327b1..16326dd5a44 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -764,9 +764,8 @@ namespace } /// Note: Doesn't check the duplicates in the `from` array. - - WhichDataType which(from_type); - if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) + /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 + if (WhichDataType which(from_type); isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; @@ -774,10 +773,13 @@ namespace { if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { - /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 - StringRef ref = cache.from_column->getDataAt(i); UInt64 key = 0; - memcpy(&key, ref.data, ref.size); + auto * dst = reinterpret_cast(&key); + const auto ref = cache.from_column->getDataAt(i); + if constexpr (std::endian::native == std::endian::big) + dst += sizeof(key) - ref.size; + + memcpy(dst, ref.data, ref.size); table[key] = i; } } From 06631f6a86a2e099ef69f807592ec2eb70b5aa35 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 4 Aug 2023 09:27:15 -0700 Subject: [PATCH 068/249] Make hasTokenOrNull return null on empty needle --- src/Functions/HasTokenImpl.h | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index ab6b6399486..661b8ae9753 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -39,9 +39,6 @@ struct HasTokenImpl if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); - if (pattern.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); - if (haystack_offsets.empty()) return; @@ -49,7 +46,7 @@ struct HasTokenImpl const UInt8 * const end = haystack_data.data() + haystack_data.size(); const UInt8 * pos = begin; - if (!std::none_of(pattern.begin(), pattern.end(), isTokenSeparator)) + if (const auto has_separator = std::any_of(pattern.cbegin(), pattern.cend(), isTokenSeparator); has_separator || pattern.empty()) { if (res_null) { @@ -57,8 +54,12 @@ struct HasTokenImpl std::ranges::fill(res_null->getData(), true); return; } - else + else if (has_separator) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); + else if (pattern.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected internal state"); } size_t pattern_size = pattern.size(); From 054562233143594b1ebd83d248567151bfa9adb2 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 4 Aug 2023 13:03:15 -0700 Subject: [PATCH 069/249] Add functional tests for hasTokenOrNull --- tests/queries/0_stateless/02816_has_token_empty.reference | 4 ++++ tests/queries/0_stateless/02816_has_token_empty.sql | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/02816_has_token_empty.reference b/tests/queries/0_stateless/02816_has_token_empty.reference index aa47d0d46d4..8435d77c5fe 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.reference +++ b/tests/queries/0_stateless/02816_has_token_empty.reference @@ -1,2 +1,6 @@ 0 +\N +\N 0 +\N +\N diff --git a/tests/queries/0_stateless/02816_has_token_empty.sql b/tests/queries/0_stateless/02816_has_token_empty.sql index e5d6156debd..3e00959126b 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.sql +++ b/tests/queries/0_stateless/02816_has_token_empty.sql @@ -2,6 +2,10 @@ SELECT hasTokenCaseInsensitive('K(G', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', 'Hello'); +SELECT hasTokenCaseInsensitiveOrNull('Hello', ''); +SELECT hasTokenCaseInsensitiveOrNull('', ''); SELECT hasToken('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasToken('', 'Hello'); SELECT hasToken('', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenOrNull('', ''); +SELECT hasTokenOrNull('Hello', ''); From 264bff8c9fcf25e206b86c88e1c3a6f2a5caf8b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 01:43:54 +0200 Subject: [PATCH 070/249] Fix a comment --- base/poco/Data/include/Poco/Data/TypeHandler.h | 2 +- programs/format/Format.cpp | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/base/poco/Data/include/Poco/Data/TypeHandler.h b/base/poco/Data/include/Poco/Data/TypeHandler.h index 34f88e986f7..e7633de7018 100644 --- a/base/poco/Data/include/Poco/Data/TypeHandler.h +++ b/base/poco/Data/include/Poco/Data/TypeHandler.h @@ -97,7 +97,7 @@ namespace Data /// /// static void extract(std::size_t pos, Person& obj, const Person& defVal, AbstractExtractor::Ptr pExt) /// { - /// // defVal is the default person we should use if we encunter NULL entries, so we take the individual fields + /// // defVal is the default person we should use if we encounter NULL entries, so we take the individual fields /// // as defaults. You can do more complex checking, ie return defVal if only one single entry of the fields is null etc... /// poco_assert_dbg (!pExt.isNull()); /// std::string lastName; diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 43c66a32302..d7d61bbcd3b 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -163,13 +163,15 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { ASTPtr res = parseQueryAndMovePosition( parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth); - /// For insert query with data(INSERT INTO ... VALUES ...), will lead to format fail, - /// should throw exception early and make exception message more readable. + + /// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure, + /// we should throw an exception early, and make exception message more readable. if (const auto * insert_query = res->as(); insert_query && insert_query->data) { throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA, "Can't format ASTInsertQuery with data, since data will be lost"); } + if (!quiet) { if (!backslash) From d479c125784010ed516188e948fa6554e72078c7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 5 Aug 2023 14:11:22 +0300 Subject: [PATCH 071/249] Fixed tests --- tests/queries/0_stateless/01655_plan_optimizations.reference | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index be42a656c66..54ca55d2068 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -168,19 +168,23 @@ Filter 3 > one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 3 3 > filter is pushed down before UNION From 9de95879655cb0ebbd895b0e95198e6ad759f09a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 13:54:36 +0200 Subject: [PATCH 072/249] Something with tests --- tests/integration/test_zookeeper_config/test.py | 7 +++---- tests/queries/0_stateless/02263_format_insert_settings.sh | 2 ++ 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 5e36a97b3ca..34ac2cfa572 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -58,9 +58,7 @@ def test_chroot_with_same_root(started_cluster): node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple", "2\n") - assert_eq_with_retry(node2, "select count() from simple", "2\n") @@ -78,5 +76,6 @@ def test_chroot_with_different_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - assert node1.query("select count() from simple_different").strip() == "1" - assert node3.query("select count() from simple_different").strip() == "1" + # Replication might take time + assert_eq_with_retry(node1, "select count() from simple_different", "1\n") + assert_eq_with_retry(node3, "select count() from simple_different", "1\n") diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index 3d5f780a38c..efb3d39ab6c 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -23,10 +23,12 @@ function run_format_both() # NOTE: that those queries may work slow, due to stack trace obtaining run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") + # compatibility run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") + # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' From 10af8fcca4fedf6347fe724ed68f1e60b53b2b1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 14:05:06 +0200 Subject: [PATCH 073/249] Fix Python --- tests/integration/test_zookeeper_config/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 34ac2cfa572..3b5ae7186fb 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -58,8 +58,8 @@ def test_chroot_with_same_root(started_cluster): node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple", "2\n") - assert_eq_with_retry(node2, "select count() from simple", "2\n") + assert_eq_with_retry(node1, "select count() from simple", "2") + assert_eq_with_retry(node2, "select count() from simple", "2") def test_chroot_with_different_root(started_cluster): @@ -77,5 +77,5 @@ def test_chroot_with_different_root(started_cluster): node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple_different", "1\n") - assert_eq_with_retry(node3, "select count() from simple_different", "1\n") + assert_eq_with_retry(node1, "select count() from simple_different", "1") + assert_eq_with_retry(node3, "select count() from simple_different", "1") From aa57fb6602bd0a815fe512b82df8b18893d268eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 14:07:55 +0200 Subject: [PATCH 074/249] Maybe better, but not sure --- tests/integration/test_zookeeper_config/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 3b5ae7186fb..4008df3f84f 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -57,8 +57,9 @@ def test_chroot_with_same_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - # Replication might take time + node1.query("SYSTEM SYNC REPLICA simple") assert_eq_with_retry(node1, "select count() from simple", "2") + node2.query("SYSTEM SYNC REPLICA simple") assert_eq_with_retry(node2, "select count() from simple", "2") @@ -76,6 +77,7 @@ def test_chroot_with_different_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - # Replication might take time + node1.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node1, "select count() from simple_different", "1") + node2.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node3, "select count() from simple_different", "1") From 0a6d9881a72c9476ffeb6263f924150bf3116f29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 16:52:44 +0200 Subject: [PATCH 075/249] Maybe better, but not sure --- tests/integration/test_zookeeper_config/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 4008df3f84f..0c0f77ec597 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -79,5 +79,5 @@ def test_chroot_with_different_root(started_cluster): node1.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node1, "select count() from simple_different", "1") - node2.query("SYSTEM SYNC REPLICA simple_different") + node3.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node3, "select count() from simple_different", "1") From afb1a55bf3789196e2b9f96aea5318ea0a9f9219 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:56:33 +0200 Subject: [PATCH 076/249] Remove whitespaces --- src/Interpreters/SystemLog.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 437b1b2a6bb..c4c2b0b2499 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -131,8 +131,6 @@ protected: using Base::queue; private: - - /* Saving thread data */ const StorageID table_id; const String storage_def; From 7e90df67a746fb12a9d4ecb6944beeab198d869d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:58:26 +0200 Subject: [PATCH 077/249] Remove whitespaces --- src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index c4c2b0b2499..cf287ad7775 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -111,7 +111,7 @@ public: * and new table get created - as if previous table was not exist. */ SystemLog(ContextPtr context_, - const SystemLogSettings& settings_, + const SystemLogSettings & settings_, std::shared_ptr> queue_ = nullptr); /** Append a record into log. From 06d45cfa81c2e21963a1a82bb7fabc1bcb9b115b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 01:29:31 +0200 Subject: [PATCH 078/249] Allow creating system logs at startup --- programs/server/config.xml | 3 +++ src/Interpreters/SystemLog.cpp | 6 ++++++ src/Interpreters/SystemLog.h | 3 +++ 3 files changed, 12 insertions(+) diff --git a/programs/server/config.xml b/programs/server/config.xml index 14b8954fc39..3d1c92b135c 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1037,6 +1037,9 @@ + + + false +**Example** ``` sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 7ed2deaeda6..88e4ac03fdb 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -559,6 +559,29 @@ Result: └────────────────────────────┘ ``` +## tupleConcat + +Combines tuples passed as arguments. + +``` sql +tupleConcat(tuples) +``` + +**Arguments** + +- `tuples` – Arbitrary number of arguments of [Tuple](../../sql-reference/data-types/tuple.md) type. + +**Example** + +``` sql +SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res +``` + +``` text +┌─res──────────────────┐ +│ (1,2,3,4,true,false) │ +└──────────────────────┘ +``` ## Distance functions diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 8288d872f18..9eb222d8c09 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -208,6 +208,10 @@ public: { return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments); } + else if (isTuple(arguments.at(0).type)) + { + return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments); + } else return std::make_unique( FunctionConcat::create(context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); diff --git a/src/Functions/tupleConcat.cpp b/src/Functions/tupleConcat.cpp new file mode 100644 index 00000000000..0556f4181e6 --- /dev/null +++ b/src/Functions/tupleConcat.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; +} + +/// tupleConcat(tup1, ...) - concatenate tuples. +class FunctionTupleConcat : public IFunction +{ +public: + static constexpr auto name = "tupleConcat"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires at least one argument.", + getName()); + + DataTypes tuple_arg_types; + + for (const auto arg_idx : collections::range(0, arguments.size())) + { + const auto * arg = arguments[arg_idx].get(); + if (!isTuple(arg)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arg->getName(), + arg_idx + 1, + getName()); + + const auto * type = checkAndGetDataType(arg); + for (const auto & elem : type->getElements()) + tuple_arg_types.push_back(elem); + } + + return std::make_shared(tuple_arg_types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const size_t num_arguments = arguments.size(); + Columns columns; + + for (size_t i = 0; i < num_arguments; i++) + { + const DataTypeTuple * arg_type = checkAndGetDataType(arguments[i].type.get()); + + if (!arg_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arguments[i].type->getName(), + i + 1, + getName()); + + ColumnPtr arg_col = arguments[i].column->convertToFullColumnIfConst(); + const ColumnTuple * tuple_col = checkAndGetColumn(arg_col.get()); + + if (!tuple_col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}", + arguments[i].column->getName(), + getName()); + + for (const auto & inner_col : tuple_col->getColumns()) + columns.push_back(inner_col); + } + + return ColumnTuple::create(columns); + } +}; + +REGISTER_FUNCTION(TupleConcat) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02833_tuple_concat.reference b/tests/queries/0_stateless/02833_tuple_concat.reference new file mode 100644 index 00000000000..2c865f13ffc --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.reference @@ -0,0 +1,6 @@ +(1,'y',2,'n') +(1,'y',2,'n',3,'n') +(1,2,3,'a','b','c','2020-10-08','2020-11-08') 1 2 3 a b c 2020-10-08 2020-11-08 +(1,2,1,2) 1 2 1 2 +(1,2,3,4) 1 2 3 4 +(3,4,1,2) 3 4 1 2 diff --git a/tests/queries/0_stateless/02833_tuple_concat.sql b/tests/queries/0_stateless/02833_tuple_concat.sql new file mode 100644 index 00000000000..df43e08d595 --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.sql @@ -0,0 +1,23 @@ +SELECT tupleConcat(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleConcat((1, 'y'), 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT tupleConcat((1, 'y'), (2, 'n')); +SELECT tupleConcat((1, 'y'), (2, 'n'), (3, 'n')); + +WITH (1,2,3) || ('a','b','c') || ('2020-10-08'::Date, '2020-11-08'::Date) AS t +SELECT t, t.1, t.2, t.3, t.4, t.5, t.6, t.7, t.8; + +DROP TABLE IF EXISTS t_02833; +CREATE TABLE t_02833 (tup Tuple(a UInt64, b UInt64)) ENGINE=Log; +INSERT INTO t_02833 VALUES ((1, 2)); + +WITH (tup || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH (tup || (3, 4)) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH ((3, 4) || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +DROP TABLE t_02833; From 0498b48c70aa60b49ae5a2f6ee702f564921062d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 21:01:58 +0000 Subject: [PATCH 154/249] different linker --- cmake/linux/toolchain-s390x.cmake | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index c1cbb849bd3..f5e1cf660c9 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -22,6 +22,8 @@ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 481792181e6ac9515db966817d123afb99d9971f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 23:12:39 +0200 Subject: [PATCH 155/249] Remove garbage --- .../test_cluster_copier/test_two_nodes.py | 21 - .../test_cassandra.py | 99 - .../test.py | 68 - .../test_default_reading.py | 1 - .../test_default_string.py | 1 - .../test_dict_get.py | 1 - .../test_dict_get_or_default.py | 1 - .../test_extreme_deduplication/test.py | 70 - tests/integration/test_hive_query/__init__.py | 0 .../test_hive_query/configs/config.xml | 32 - .../test_hive_query/configs/hdfs-site.xml | 6 - .../test_hive_query/data/prepare_hive_data.sh | 15 - tests/integration/test_hive_query/test.py | 498 ----- .../__init__.py | 0 .../configs/host_regexp.xml | 11 - .../configs/listen_host.xml | 5 - .../coredns_config/Corefile | 8 - .../coredns_config/example.com | 1 - .../test.py | 115 - .../test.py | 68 - .../test.py | 78 - .../test_mutations_with_merge_tree/test.py | 194 -- .../test_polymorphic_parts/test.py | 48 - .../__init__.py | 0 .../configs/config.d/storage_conf.xml | 93 - .../test.py | 322 --- .../test_reverse_dns_query/__init__.py | 0 .../test_reverse_dns_query/configs/config.xml | 3 - .../configs/listen_host.xml | 5 - .../configs/reverse_dns_function.xml | 3 - .../coredns_config/Corefile | 4 - .../test_reverse_dns_query/test.py | 56 - .../test_s3_zero_copy_replication/test.py | 90 - .../test_s3_zero_copy_ttl/__init__.py | 0 .../configs/max_delayed_streams.xml | 9 - .../test_s3_zero_copy_ttl/configs/s3.xml | 39 - .../integration/test_s3_zero_copy_ttl/test.py | 94 - .../test_ttl_move_memory_usage.py | 81 - .../test_vertical_merge_memory_usage.py | 46 - .../integration/test_storage_nats/__init__.py | 0 .../clickhouse_path/format_schemas/nats.proto | 6 - .../test_storage_nats/configs/macros.xml | 7 - .../configs/named_collection.xml | 13 - .../test_storage_nats/configs/nats.xml | 6 - .../test_storage_nats/configs/users.xml | 7 - .../test_storage_nats/nats_certs.sh | 13 - .../integration/test_storage_nats/nats_pb2.py | 37 - tests/integration/test_storage_nats/test.py | 1875 ----------------- .../test_nats_json.reference | 50 - .../__init__.py | 0 .../configs/log_conf.xml | 11 - .../test_storage_postgresql_replica/test.py | 780 ------- .../test_postgresql_replica.reference | 50 - .../integration/test_storage_rabbitmq/test.py | 63 - tests/integration/test_ttl_move/test.py | 100 - 55 files changed, 5204 deletions(-) delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py delete mode 100644 tests/integration/test_hive_query/__init__.py delete mode 100644 tests/integration/test_hive_query/configs/config.xml delete mode 100644 tests/integration/test_hive_query/configs/hdfs-site.xml delete mode 100755 tests/integration/test_hive_query/data/prepare_hive_data.sh delete mode 100644 tests/integration/test_hive_query/test.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/__init__.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/test.py delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py delete mode 100644 tests/integration/test_reverse_dns_query/__init__.py delete mode 100644 tests/integration/test_reverse_dns_query/configs/config.xml delete mode 100644 tests/integration/test_reverse_dns_query/configs/listen_host.xml delete mode 100644 tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml delete mode 100644 tests/integration/test_reverse_dns_query/coredns_config/Corefile delete mode 100644 tests/integration/test_reverse_dns_query/test.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/__init__.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml delete mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/s3.xml delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py delete mode 100644 tests/integration/test_storage_nats/__init__.py delete mode 100755 tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto delete mode 100644 tests/integration/test_storage_nats/configs/macros.xml delete mode 100644 tests/integration/test_storage_nats/configs/named_collection.xml delete mode 100644 tests/integration/test_storage_nats/configs/nats.xml delete mode 100644 tests/integration/test_storage_nats/configs/users.xml delete mode 100755 tests/integration/test_storage_nats/nats_certs.sh delete mode 100644 tests/integration/test_storage_nats/nats_pb2.py delete mode 100644 tests/integration/test_storage_nats/test.py delete mode 100644 tests/integration/test_storage_nats/test_nats_json.reference delete mode 100644 tests/integration/test_storage_postgresql_replica/__init__.py delete mode 100644 tests/integration/test_storage_postgresql_replica/configs/log_conf.xml delete mode 100644 tests/integration/test_storage_postgresql_replica/test.py delete mode 100644 tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 2b6fcf6cac2..1bd3561f24f 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -595,24 +595,3 @@ def execute_task(started_cluster, task, cmd_options): task.check() finally: zk.delete(task.zk_task_path, recursive=True) - - -# Tests -@pytest.mark.skip(reason="Too flaky :(") -def test_different_schema(started_cluster): - execute_task(started_cluster, TaskWithDifferentSchema(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_ttl_columns(started_cluster): - execute_task(started_cluster, TaskTTL(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_skip_index(started_cluster): - execute_task(started_cluster, TaskSkipIndex(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_ttl_move_to_volume(started_cluster): - execute_task(started_cluster, TaskTTLMoveToVolume(started_cluster), []) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py deleted file mode 100644 index 90287e19bd0..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ /dev/null @@ -1,99 +0,0 @@ -import os -import math -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/33006 - -pytestmark = pytest.mark.skip - -from .common import * - -from helpers.cluster import ClickHouseCluster -from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout -from helpers.external_sources import SourceCassandra - -SOURCE = None -cluster = None -node = None -simple_tester = None -complex_tester = None -ranged_tester = None -test_name = "cassandra" - - -def setup_module(module): - global cluster - global node - global simple_tester - global complex_tester - global ranged_tester - - cluster = ClickHouseCluster(__file__) - - SOURCE = SourceCassandra( - "Cassandra", - None, - cluster.cassandra_port, - cluster.cassandra_host, - cluster.cassandra_port, - "", - "", - ) - - simple_tester = SimpleLayoutTester(test_name) - simple_tester.cleanup() - simple_tester.create_dictionaries(SOURCE) - - complex_tester = ComplexLayoutTester(test_name) - complex_tester.create_dictionaries(SOURCE) - - ranged_tester = RangedLayoutTester(test_name) - ranged_tester.create_dictionaries(SOURCE) - # Since that all .xml configs were created - - main_configs = [] - main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) - - dictionaries = simple_tester.list_dictionaries() - - node = cluster.add_instance( - "cass_node", - main_configs=main_configs, - dictionaries=dictionaries, - with_cassandra=True, - ) - - -def teardown_module(module): - simple_tester.cleanup() - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - simple_tester.prepare(cluster) - complex_tester.prepare(cluster) - ranged_tester.prepare(cluster) - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE)) -def test_simple(started_cluster, layout_name): - simple_tester.execute(layout_name, node) - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_COMPLEX)) -def test_complex(started_cluster, layout_name): - complex_tester.execute(layout_name, node) - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_RANGED)) -def test_ranged(started_cluster, layout_name): - ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index ae9cd4e7c91..bca7122a2a8 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -27,71 +27,3 @@ def started_cluster(): yield cluster finally: cluster.shutdown() - - -@pytest.mark.skip(reason="SSD cache test can run on disk only") -@pytest.mark.parametrize("type", ["memory", "ssd"]) -def test_memory_consumption(started_cluster, type): - node = started_cluster.instances[f"node_{type}"] - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "w" * 8 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "x" * 16 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "y" * 32 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "z" * 64 - ) - ) - - # Fill dictionary - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated_first = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - - alloc_array = [] - for i in range(5): - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - alloc_array.append(allocated) - - # size doesn't grow - assert all(allocated_first >= a for a in alloc_array) - - for i in range(5): - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - alloc_array.append(allocated) - - # size doesn't grow - assert all(allocated_first >= a for a in alloc_array) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 85c45d5df3c..921fb4e4154 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_default_reading(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 92d681698bc..6eedf63f95c 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -53,7 +53,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_return_real_values(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 05f638ec337..8e45af44640 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_simple_dict_get(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 1da8fd3325a..8d6242f4711 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_simple_dict_get_or_default(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 03451933171..3632369154a 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -71,73 +71,3 @@ def test_deduplication_window_in_seconds(started_cluster): assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3" node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") - - -# Currently this test just reproduce incorrect behavior that sould be fixed -@pytest.mark.skip(reason="Flapping test") -def test_deduplication_works_in_case_of_intensive_inserts(started_cluster): - inserters = [] - fetchers = [] - - node1.query( - """ - CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id""" - ) - - node1.query("INSERT INTO simple VALUES (0, 0)") - - for node in nodes: - host = node.ip_address - - inserters.append( - CommandRequest( - ["/bin/bash"], - timeout=10, - stdin=""" -set -e -for i in `seq 1000`; do - {} --host {} -q "INSERT INTO simple VALUES (0, 0)" -done -""".format( - cluster.get_client_cmd(), host - ), - ) - ) - - fetchers.append( - CommandRequest( - ["/bin/bash"], - timeout=10, - stdin=""" -set -e -for i in `seq 1000`; do - res=`{} --host {} -q "SELECT count() FROM simple"` - if [[ $? -ne 0 || $res -ne 1 ]]; then - echo "Selected $res elements! Host: {}" 1>&2 - exit -1 - fi; -done -""".format( - cluster.get_client_cmd(), host, node.name - ), - ) - ) - - # There were not errors during INSERTs - for inserter in inserters: - try: - inserter.get_answer() - except QueryTimeoutExceedException: - # Only timeout is accepted - pass - - # There were not errors during SELECTs - for fetcher in fetchers: - try: - fetcher.get_answer() - except QueryTimeoutExceedException: - # Only timeout is accepted - pass - - node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") diff --git a/tests/integration/test_hive_query/__init__.py b/tests/integration/test_hive_query/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml deleted file mode 100644 index 7de1391e56c..00000000000 --- a/tests/integration/test_hive_query/configs/config.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - - localhost - 9000 - - - - - - - true - /tmp/clickhouse_local_cache,/tmp/clickhouse_local_cache1 - 207374182400 - 1048576 - - - - /etc/clickhouse-server/extra_conf.d/hdfs-site.xml - - - - system - metric_log
- 1000 - 1000 -
- -
diff --git a/tests/integration/test_hive_query/configs/hdfs-site.xml b/tests/integration/test_hive_query/configs/hdfs-site.xml deleted file mode 100644 index 82c525ea414..00000000000 --- a/tests/integration/test_hive_query/configs/hdfs-site.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - dfs.replication - 1 - - diff --git a/tests/integration/test_hive_query/data/prepare_hive_data.sh b/tests/integration/test_hive_query/data/prepare_hive_data.sh deleted file mode 100755 index 495ea201870..00000000000 --- a/tests/integration/test_hive_query/data/prepare_hive_data.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/bash -hive -e "create database test" - -hive -e "drop table if exists test.demo; create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; " -hive -e "drop table if exists test.parquet_demo; create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" -hive -e "drop table if exists test.demo_text; create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" - -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" - -hive -e "drop table if exists test.test_hive_types; CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct>) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" - -hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 10, 'y', 'xyz')); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 11, 'y', 'abc'));" - diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py deleted file mode 100644 index 791ae03f9f6..00000000000 --- a/tests/integration/test_hive_query/test.py +++ /dev/null @@ -1,498 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/43541 - -pytestmark = pytest.mark.skip - -import logging -import os - -import time -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "h0_0_0", - main_configs=["configs/config.xml"], - extra_configs=["configs/hdfs-site.xml", "data/prepare_hive_data.sh"], - with_hive=True, - ) - - logging.info("Starting cluster ...") - cluster.start() - cluster.copy_file_to_container( - "roottesthivequery_hdfs1_1", - "/ClickHouse/tests/integration/test_hive_query/data/prepare_hive_data.sh", - "/prepare_hive_data.sh", - ) - cluster.exec_in_container( - "roottesthivequery_hdfs1_1", ["bash", "-c", "bash /prepare_hive_data.sh"] - ) - yield cluster - finally: - cluster.shutdown() - - -def test_create_parquet_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - test_passed = False - for i in range(10): - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query( - """ -DROP TABLE IF EXISTS default.demo_parquet; -CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - assert test_passed - - -def test_create_parquet_table_1(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - for i in range(10): - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query( - """ -DROP TABLE IF EXISTS default.demo_parquet_parts; -CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - assert test_passed - - -def test_create_orc_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - test_passed = False - for i in range(10): - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - - assert test_passed - - -def test_create_text_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_text; - CREATE TABLE default.demo_text (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_text') PARTITION BY (tuple()) - """ - ) - logging.info("create result {}".format(result)) - - assert result.strip() == "" - - -def test_parquet_groupby(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_in_filter(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') - """ - ) - expected_result = """2 -""" - logging.info("query result:{}".format(result)) - assert result == expected_result - - -def test_orc_groupby(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day); - SELECT day, count(*) FROM default.demo_orc group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -@pytest.mark.parametrize( - "table,use_local_cache_for_remote_storage,enable_orc_file_minmax_index,enable_orc_stripe_minmax_index", - [ - pytest.param( - "demo_orc_no_cache_no_index", - "false", - "false", - "false", - id="demo_orc_no_cache_no_index", - ), - pytest.param( - "demo_orc_with_cache_no_index", - "true", - "false", - "false", - id="demo_orc_with_cache_no_index", - ), - pytest.param( - "demo_orc_no_cache_file_index", - "false", - "true", - "false", - id="demo_orc_no_cache_file_index", - ), - pytest.param( - "demo_orc_with_cache_file_index", - "true", - "true", - "false", - id="demo_orc_with_cache_file_index", - ), - pytest.param( - "demo_orc_no_cache_stripe_index", - "false", - "true", - "true", - id="demo_orc_no_cache_stripe_index", - ), - pytest.param( - "demo_orc_with_cache_stripe_index", - "true", - "true", - "true", - id="demo_orc_with_cache_stripe_index", - ), - ], -) -def test_orc_minmax_index( - started_cluster, - table, - use_local_cache_for_remote_storage, - enable_orc_file_minmax_index, - enable_orc_stripe_minmax_index, -): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.{table}; - CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) - SETTINGS enable_orc_file_minmax_index = {enable_orc_file_minmax_index}, enable_orc_stripe_minmax_index = {enable_orc_stripe_minmax_index}; - """.format( - table=table, - enable_orc_file_minmax_index=enable_orc_file_minmax_index, - enable_orc_stripe_minmax_index=enable_orc_stripe_minmax_index, - ) - ) - assert result.strip() == "" - - for i in range(2): - result = node.query( - """ - SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id - SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} - """.format( - table=table, - use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, - ) - ) - - assert ( - result - == """2021-11-05 abd 15 -2021-11-16 aaa 22 -""" - ) - - -@pytest.mark.parametrize( - "table,use_local_cache_for_remote_storage,enable_parquet_rowgroup_minmax_index", - [ - pytest.param( - "demo_parquet_no_cache_no_index", - "false", - "false", - id="demo_parquet_no_cache_no_index", - ), - pytest.param( - "demo_parquet_with_cache_no_index", - "true", - "false", - id="demo_parquet_with_cache_no_index", - ), - pytest.param( - "demo_parquet_no_cache_rowgroup_index", - "false", - "true", - id="demo_parquet_no_cache_rowgroup_index", - ), - pytest.param( - "demo_parquet_with_cache_rowgroup_index", - "true", - "true", - id="demo_parquet_with_cache_rowgroup_index", - ), - ], -) -def test_parquet_minmax_index( - started_cluster, - table, - use_local_cache_for_remote_storage, - enable_parquet_rowgroup_minmax_index, -): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.{table}; - CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - SETTINGS enable_parquet_rowgroup_minmax_index = {enable_parquet_rowgroup_minmax_index} - """.format( - table=table, - enable_parquet_rowgroup_minmax_index=enable_parquet_rowgroup_minmax_index, - ) - ) - assert result.strip() == "" - - for i in range(2): - result = node.query( - """ - SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id - SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} - """.format( - table=table, - use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, - ) - ) - - assert ( - result - == """2021-11-05 abd 15 -2021-11-16 aaa 22 -""" - ) - - -def test_hive_columns_prunning(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' - """ - ) - expected_result = """4 -""" - logging.info("query result:{}".format(result)) - assert result == expected_result - - -def test_text_count(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day); - SELECT day, count(*) FROM default.demo_orc group by day order by day SETTINGS format_csv_delimiter = '\x01' - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_groupby_with_cache(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_groupby_by_hive_function(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM hive('thrift://hivetest:9083', 'test', 'demo', '`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)', 'day') group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_cache_read_bytes(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - test_passed = False - for i in range(10): - result = node.query( - """ - SELECT * FROM default.demo_parquet_1 settings input_format_parquet_allow_missing_columns = true - """ - ) - node.query("system flush logs") - result = node.query( - "select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0" - ) - if result.strip() == "0": - logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") - time.sleep(10) - continue - test_passed = True - break - assert test_passed - - -def test_cache_dir_use(started_cluster): - node = started_cluster.instances["h0_0_0"] - result0 = node.exec_in_container( - ["bash", "-c", "ls /tmp/clickhouse_local_cache | wc -l"] - ) - result1 = node.exec_in_container( - ["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"] - ) - assert result0 != "0" and result1 != "0" - - -def test_hive_struct_type(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32, d Tuple(x Int32, y String)), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day) - """ - ) - result = node.query( - """ - SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ - ) - expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20""" - assert result.strip() == expected_result - - result = node.query( - """ - SELECT day, f_struct.a, f_struct.d.x FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ - ) - expected_result = """2022-02-20 aaa 10""" - - -def test_table_alter_add(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - result = node.query( - """ -ALTER TABLE default.demo_parquet_1 ADD COLUMN id Nullable(String) FIRST - """ - ) - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - - expected_result = "id\tNullable(String)\t\t\t\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)" - assert result.strip() == expected_result - - -def test_table_alter_drop(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - result = node.query( - """ -ALTER TABLE default.demo_parquet_1 DROP COLUMN id - """ - ) - - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - expected_result = """score\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" - assert result.strip() == expected_result - - -def test_table_alter_comment(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - - result = node.query( - """ALTER TABLE default.demo_parquet_1 COMMENT COLUMN id 'Text comment'""" - ) - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" - assert result.strip() == expected_result diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml deleted file mode 100644 index 7a2141e6c7e..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - test1\.example\.com$ - - default - - - \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile deleted file mode 100644 index 3edf37dafa5..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile +++ /dev/null @@ -1,8 +0,0 @@ -. { - hosts /example.com { - reload "20ms" - fallthrough - } - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com deleted file mode 100644 index 9beb415c290..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com +++ /dev/null @@ -1 +0,0 @@ -filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py deleted file mode 100644 index 36f76140a2c..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ /dev/null @@ -1,115 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/51471 - -pytestmark = pytest.mark.skip - -import socket -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=["configs/listen_host.xml"], - user_configs=["configs/host_regexp.xml"], - ipv6_address="2001:3984:3989::1:1111", -) - -client = cluster.add_instance( - "clickhouse-client", - ipv6_address="2001:3984:3989::1:1112", -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def check_ptr_record(ip, hostname): - try: - host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) - if hostname.lower() == host.lower(): - return True - except socket.herror: - pass - return False - - -def setup_dns_server(ip): - domains_string = "test3.example.com test2.example.com test1.example.com" - example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' - run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) - - # DNS server takes time to reload the configuration. - for try_num in range(10): - if all(check_ptr_record(ip, host) for host in domains_string.split()): - break - sleep(1) - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def build_endpoint_v4(ip): - return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" - - -def build_endpoint_v6(ip): - return build_endpoint_v4(f"[{ip}]") - - -def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - random_ip = "9.9.9.9" - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(random_ip) - setup_ch_server(dns_server_ip) - - endpoint = build_endpoint_v4(server_ip) - - assert "1\n" != client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) - - -def test_host_regexp_multiple_ptr_v4(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - client_ip = cluster.get_instance_ip("clickhouse-client") - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(client_ip) - setup_ch_server(dns_server_ip) - - endpoint = build_endpoint_v4(server_ip) - - assert "1\n" == client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) - - -def test_host_regexp_multiple_ptr_v6(started_cluster): - setup_dns_server(client.ipv6_address) - setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host)) - - endpoint = build_endpoint_v6(ch_server.ipv6_address) - - assert "1\n" == client.exec_in_container(["bash", "-c", f"curl -6 {endpoint}"]) diff --git a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py index a5f92002450..b1eb0df2d43 100644 --- a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py +++ b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py @@ -108,74 +108,6 @@ def test_reconnect(started_cluster): assert remote.query("SELECT count(*) FROM local1").strip() == "3" -@pytest.mark.skip(reason="Flapping test") -def test_inserts_batching(started_cluster): - instance = instance_test_inserts_batching - - with PartitionManager() as pm: - pm.partition_instances(instance, remote) - - instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 1)") - # Sleep a bit so that this INSERT forms a batch of its own. - time.sleep(0.2) - - instance.query("INSERT INTO local2_source(x, d) VALUES (2, '2000-01-01')") - - for i in range(3, 7): - instance.query( - "INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i) - ) - - for i in range(7, 9): - instance.query( - "INSERT INTO local2_source(x, d) VALUES ({}, '2000-01-01')".format(i) - ) - - instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 9)") - - # After ALTER the structure of the saved blocks will be different - instance.query("DROP TABLE local2_view") - instance.query("ALTER TABLE distributed ADD COLUMN s String") - - # Memory Engine doesn't support ALTER so we just DROP/CREATE everything - instance.query("DROP TABLE local2_source") - instance.query( - "CREATE TABLE local2_source (d Date, x UInt32, s String) ENGINE = Memory" - ) - instance.query( - "CREATE MATERIALIZED VIEW local2_view to distributed AS SELECT d,x,s FROM local2_source" - ) - - for i in range(10, 13): - instance.query( - "INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i) - ) - - time.sleep(1.0) - - result = remote.query( - "SELECT _part, groupArray(x) FROM local2 GROUP BY _part ORDER BY _part" - ) - - # Explanation: as merges are turned off on remote instance, active parts in local2 table correspond 1-to-1 - # to inserted blocks. - # Batches of max 3 rows are formed as min_insert_block_size_rows = 3. - # Blocks: - # 1. Failed batch that is retried with the same contents. - # 2. Full batch of inserts regardless of the order of columns thanks to the view. - # 3. Full batch of inserts regardless order of columns thanks to the view. - # 4. Full batch of inserts after ALTER (that have different block structure). - # 5. What was left to insert before ALTER. - expected = """\ -20000101_20000101_1_1_0 [1] -20000101_20000101_2_2_0 [2,3,4] -20000101_20000101_3_3_0 [5,6,7] -20000101_20000101_4_4_0 [10,11,12] -20000101_20000101_5_5_0 [8,9] -""" - assert TSV(result) == TSV(expected) - - def test_inserts_local(started_cluster): instance = instance_test_inserts_local_cluster instance.query("INSERT INTO local_source VALUES ('2000-01-01', 1)") diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 1ec44d8a002..b099d0513e1 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -41,81 +41,3 @@ def started_cluster(): def get_fake_zk(node): return ku.get_fake_zk(cluster, node) - - -@pytest.mark.skip(reason="test is flaky because changes are not properly waited for") -def test_reconfig_replace_leader_in_one_command(started_cluster): - """ - Remove leader from a cluster of 3 and add a new node to this cluster in a single command - """ - - zk1 = get_fake_zk(node1) - config = ku.get_config_str(zk1) - - assert len(config.split("\n")) == 3 - assert "node1" in config - assert "node2" in config - assert "node3" in config - assert "node4" not in config - - for i in range(100): - zk1.create(f"/test_four_{i}", b"somedata") - - zk2 = get_fake_zk(node2) - zk2.sync("/test_four_0") - ku.wait_configs_equal(config, zk2) - - zk3 = get_fake_zk(node3) - zk3.sync("/test_four_0") - ku.wait_configs_equal(config, zk3) - - for i in range(100): - assert zk2.exists(f"/test_four_{i}") is not None - assert zk3.exists(f"/test_four_{i}") is not None - - assert ku.is_leader(cluster, node1) - node4.start_clickhouse() - config, _ = zk2.reconfig( - joining="server.4=node4:9234", leaving="1", new_members=None - ) - config = config.decode("utf-8") - - print("After removing 1 and adding 4", config) - assert len(config.split("\n")) == 3 - assert "node1" not in config - assert "node2" in config - assert "node3" in config - assert "node4" in config - - ku.wait_until_connected(cluster, node4) - time.sleep(1) - - zk4 = get_fake_zk(node4) - zk4.sync("/test_four_0") - ku.wait_configs_equal(config, zk4) - - for i in range(100): - assert zk4.exists(f"test_four_{i}") is not None - zk4.create(f"/test_four_{100 + i}", b"somedata") - - with pytest.raises(Exception): - zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) - zk1.sync("/test_four_0") - - zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) - zk2.sync("/test_four_0") - ku.wait_configs_equal(config, zk2) - - zk3.stop() - zk3.close() - zk3 = get_fake_zk(node3) - zk3.sync("/test_four_0") - ku.wait_configs_equal(config, zk3) - - for i in range(200): - assert zk2.exists(f"test_four_{i}") is not None - assert zk3.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_mutations_with_merge_tree/test.py b/tests/integration/test_mutations_with_merge_tree/test.py index e7d7abc8c7c..0b4a750b38e 100644 --- a/tests/integration/test_mutations_with_merge_tree/test.py +++ b/tests/integration/test_mutations_with_merge_tree/test.py @@ -27,111 +27,6 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_in_partition_background(started_cluster): - try: - numbers = 100 - - name = "test_mutations_in_partition" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}""" - ) - - for i in range(1, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i} """ - ) - - def count_and_changed(): - return instance_test_mutations.query( - f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - all_done = False - for wait_times_for_mutation in range( - 100 - ): # wait for replication 80 seconds max - time.sleep(0.8) - - if count_and_changed() == ["66,33"]: - all_done = True - break - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert (count_and_changed(), all_done) == (["66,33"], True) - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == ["67,67"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -@pytest.mark.parametrize("sync", [("last",), ("all",)]) -def test_mutations_in_partition_sync(started_cluster, sync): - try: - numbers = 10 - - name = "test_mutations_in_partition_sync" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}""" - + (" SETTINGS mutations_sync = 1" if sync == "all" else "") - ) - - for reverse_index, i in reversed( - list(enumerate(reversed(range(1, numbers, 3)))) - ): - instance_test_mutations.query( - f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i}""" - + ( - " SETTINGS mutations_sync = 1" - if not reverse_index or sync == "all" - else "" - ) - ) - - def count_and_changed(): - return instance_test_mutations.query( - f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count_and_changed() == ["6,3"] - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == ["7,7"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - def test_mutations_with_merge_background_task(started_cluster): instance_test_mutations.query( """SYSTEM STOP MERGES test_mutations_with_ast_elements""" @@ -190,92 +85,3 @@ def test_mutations_with_truncate_table(started_cluster): ).rstrip() == "0" ) - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_will_not_hang_for_non_existing_parts_sync(started_cluster): - try: - numbers = 100 - - name = "test_mutations_will_not_hang_for_non_existing_parts_sync" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i} SETTINGS mutations_sync = 1""" - ) - - def count(): - return instance_test_mutations.query( - f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count() == [f"{numbers}"] - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == [f"34,34"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_will_not_hang_for_non_existing_parts_async(started_cluster): - try: - numbers = 100 - - name = "test_mutations_will_not_hang_for_non_existing_parts_async" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i}""" - ) - - def count(): - return instance_test_mutations.query( - f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - def count_and_sum_is_done(): - return instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - all_done = False - for wait_times_for_mutation in range( - 100 - ): # wait for replication 80 seconds max - time.sleep(0.8) - - if count_and_sum_is_done() == ["34,34"]: - all_done = True - break - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count() == [f"{numbers}"] - assert count_and_sum_is_done() == ["34,34"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 48bcca7d505..debb509de90 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -414,54 +414,6 @@ def start_cluster_diff_versions(): cluster.shutdown() -@pytest.mark.skip(reason="compatability is temporary broken") -def test_polymorphic_parts_diff_versions(start_cluster_diff_versions): - # Check that replication with Wide parts works between different versions. - - node_old = node7 - node_new = node8 - - insert_random_data("polymorphic_table", node7, 100) - node8.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20) - - assert node8.query("SELECT count() FROM polymorphic_table") == "100\n" - assert ( - node8.query( - "SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' and active" - ) - == "Wide\n" - ) - - -@pytest.mark.skip(reason="compatability is temporary broken") -def test_polymorphic_parts_diff_versions_2(start_cluster_diff_versions): - # Replication doesn't work on old version if part is created in compact format, because - # this version doesn't know anything about it. It's considered to be ok. - - node_old = node7 - node_new = node8 - - insert_random_data("polymorphic_table_2", node_new, 100) - - assert node_new.query("SELECT count() FROM polymorphic_table_2") == "100\n" - assert node_old.query("SELECT count() FROM polymorphic_table_2") == "0\n" - with pytest.raises(Exception): - node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=3) - - node_old.restart_with_latest_version(fix_metadata=True) - - node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=20) - - # Works after update - assert node_old.query("SELECT count() FROM polymorphic_table_2") == "100\n" - assert ( - node_old.query( - "SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table_2' and active" - ) - == "Compact\n" - ) - - def test_polymorphic_parts_non_adaptive(start_cluster): node1.query("SYSTEM STOP MERGES") node2.query("SYSTEM STOP MERGES") diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml deleted file mode 100644 index bb4aba94e0b..00000000000 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml +++ /dev/null @@ -1,93 +0,0 @@ - - - - - hdfs - hdfs://hdfs1:9000/clickhouse1/ - - true - - - hdfs - hdfs://hdfs1:9000/clickhouse1/ - - true - - - hdfs - hdfs://hdfs1:9000/clickhouse2/ - - true - - - - - -
- hdfs1 -
-
-
- - -
- default -
- - hdfs1 - -
- 0.0 -
- - -
- hdfs2 -
- - hdfs1 - -
-
- - -
- hdfs1_again -
- - hdfs1 - -
-
-
-
- - - 1024000 - 1 - true - - - - - - - node1 - 9000 - - - - - node2 - 9000 - - - - - - - test_cluster - 1 - - true -
diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py deleted file mode 100644 index eb3d62eb718..00000000000 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ /dev/null @@ -1,322 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/42561 - -pytestmark = pytest.mark.skip - -import logging -from string import Template -import time - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -from pyhdfs import HdfsClient - -SHARDS = 2 -FILES_OVERHEAD_PER_TABLE = 1 # format_version.txt -FILES_OVERHEAD_PER_PART_COMPACT = 7 - - -def wait_for_hdfs_objects(cluster, fp, expected, num_tries=30): - fs = HdfsClient(hosts=cluster.hdfs_ip) - while num_tries > 0: - num_hdfs_objects = len(fs.listdir(fp)) - if num_hdfs_objects == expected: - break - num_tries -= 1 - time.sleep(1) - assert len(fs.listdir(fp)) == expected - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node1", - main_configs=["configs/config.d/storage_conf.xml"], - macros={"replica": "node1"}, - with_zookeeper=True, - with_hdfs=True, - ) - cluster.add_instance( - "node2", - main_configs=["configs/config.d/storage_conf.xml"], - macros={"replica": "node2"}, - with_zookeeper=True, - with_hdfs=True, - ) - logging.info("Starting cluster...") - cluster.start() - if cluster.instances["node1"].is_debug_build(): - # https://github.com/ClickHouse/ClickHouse/issues/27814 - pytest.skip( - "libhdfs3 calls rand function which does not pass harmful check in debug build" - ) - logging.info("Cluster started") - - fs = HdfsClient(hosts=cluster.hdfs_ip) - fs.mkdirs("/clickhouse1") - fs.mkdirs("/clickhouse2") - logging.info("Created HDFS directory") - - yield cluster - finally: - cluster.shutdown() - - -def test_hdfs_zero_copy_replication_insert(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - """ - CREATE TABLE hdfs_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hdfs_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='hdfs_only' - """ - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", SHARDS * FILES_OVERHEAD_PER_TABLE - ) - - node1.query("INSERT INTO hdfs_test VALUES (now() - INTERVAL 3 DAY, 10)") - node2.query("SYSTEM SYNC REPLICA hdfs_test", timeout=30) - assert node1.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)" - assert node2.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)" - assert ( - node1.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)" - ) - assert ( - node2.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - wait_for_hdfs_objects( - cluster, - "/clickhouse1", - SHARDS * FILES_OVERHEAD_PER_TABLE + FILES_OVERHEAD_PER_PART_COMPACT, - ) - finally: - node1.query("DROP TABLE IF EXISTS hdfs_test SYNC") - node2.query("DROP TABLE IF EXISTS hdfs_test SYNC") - - -@pytest.mark.parametrize( - ("storage_policy", "init_objects"), - [("hybrid", 0), ("tiered", 0), ("tiered_copy", FILES_OVERHEAD_PER_TABLE)], -) -def test_hdfs_zero_copy_replication_single_move(cluster, storage_policy, init_objects): - node1 = cluster.instances["node1"] - try: - node1.query( - Template( - """ - CREATE TABLE single_node_move_test (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/single_node_move_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='$policy',temporary_directories_lifetime=1 - """ - ).substitute(policy=storage_policy) - ) - wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects) - - node1.query( - "INSERT INTO single_node_move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - - node1.query( - "ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='single_node_move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - - node1.query( - "ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'main'" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS single_node_move_test SYNC") - - -@pytest.mark.parametrize( - ("storage_policy", "init_objects"), - [("hybrid", 0), ("tiered", 0), ("tiered_copy", SHARDS * FILES_OVERHEAD_PER_TABLE)], -) -def test_hdfs_zero_copy_replication_move(cluster, storage_policy, init_objects): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - Template( - """ - CREATE TABLE move_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/move_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='$policy' - """ - ).substitute(policy=storage_policy) - ) - wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects) - - node1.query( - "INSERT INTO move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)" - ) - node2.query("SYSTEM SYNC REPLICA move_test", timeout=30) - - assert ( - node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - - node1.query( - "ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - - node2.query( - "ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - finally: - node1.query("DROP TABLE IF EXISTS move_test SYNC") - node2.query("DROP TABLE IF EXISTS move_test SYNC") - - -@pytest.mark.parametrize(("storage_policy"), ["hybrid", "tiered", "tiered_copy"]) -def test_hdfs_zero_copy_with_ttl_move(cluster, storage_policy): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - Template( - """ - CREATE TABLE ttl_move_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_move_test', '{replica}') - ORDER BY (dt, id) - TTL dt + INTERVAL 2 DAY TO VOLUME 'external' - SETTINGS storage_policy='$policy' - """ - ).substitute(policy=storage_policy) - ) - - node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 3 DAY, 10)") - node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 1 DAY, 11)") - - node1.query("OPTIMIZE TABLE ttl_move_test FINAL") - node2.query("SYSTEM SYNC REPLICA ttl_move_test", timeout=30) - - assert_eq_with_retry(node1, "SELECT count() FROM ttl_move_test", "2") - assert_eq_with_retry(node2, "SELECT count() FROM ttl_move_test", "2") - assert ( - node1.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values") - == "(10),(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS ttl_move_test SYNC") - node2.query("DROP TABLE IF EXISTS ttl_move_test SYNC") - - -def test_hdfs_zero_copy_with_ttl_delete(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - """ - CREATE TABLE ttl_delete_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_delete_test', '{replica}') - ORDER BY (dt, id) - TTL dt + INTERVAL 2 DAY - SETTINGS storage_policy='tiered' - """ - ) - - node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 3 DAY, 10)") - node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 1 DAY, 11)") - - node1.query("OPTIMIZE TABLE ttl_delete_test FINAL") - node2.query("SYSTEM SYNC REPLICA ttl_delete_test", timeout=30) - - assert_eq_with_retry(node1, "SELECT count() FROM ttl_delete_test", "1") - assert_eq_with_retry(node2, "SELECT count() FROM ttl_delete_test", "1") - - assert ( - node1.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values") - == "(11)" - ) - assert ( - node2.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values") - == "(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS ttl_delete_test SYNC") - node2.query("DROP TABLE IF EXISTS ttl_delete_test SYNC") diff --git a/tests/integration/test_reverse_dns_query/__init__.py b/tests/integration/test_reverse_dns_query/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_reverse_dns_query/configs/config.xml b/tests/integration/test_reverse_dns_query/configs/config.xml deleted file mode 100644 index 5ce55afa2a7..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/listen_host.xml b/tests/integration/test_reverse_dns_query/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml b/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml deleted file mode 100644 index 35d0a07c6a6..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/coredns_config/Corefile b/tests/integration/test_reverse_dns_query/coredns_config/Corefile deleted file mode 100644 index 84d297f7cdf..00000000000 --- a/tests/integration/test_reverse_dns_query/coredns_config/Corefile +++ /dev/null @@ -1,4 +0,0 @@ -. { - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_reverse_dns_query/test.py b/tests/integration/test_reverse_dns_query/test.py deleted file mode 100644 index 6a39bd7b586..00000000000 --- a/tests/integration/test_reverse_dns_query/test.py +++ /dev/null @@ -1,56 +0,0 @@ -import pytest - -# FIXME This test is broken -# https://github.com/ClickHouse/ClickHouse/issues/53194 - -pytestmark = pytest.mark.skip - -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=[ - "configs/config.xml", - "configs/reverse_dns_function.xml", - "configs/listen_host.xml", - ], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def test_reverse_dns_query(started_cluster): - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_ch_server(dns_server_ip) - - for _ in range(0, 200): - response = ch_server.query("select reverseDNSQuery('2001:4860:4860::8888')") - assert response == "['dns.google']\n" diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 2a4e0eece08..0ca81a27802 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -153,96 +153,6 @@ def test_s3_zero_copy_replication(started_cluster, policy): node2.query("DROP TABLE IF EXISTS s3_test SYNC") -@pytest.mark.skip(reason="Test is flaky (and never was stable)") -def test_s3_zero_copy_on_hybrid_storage(started_cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - - node1.query( - """ - CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}') - ORDER BY id - SETTINGS storage_policy='hybrid',temporary_directories_lifetime=1 - """.format( - "{replica}" - ) - ) - - node1.query("INSERT INTO hybrid_test VALUES (0,'data'),(1,'data')") - node2.query("SYSTEM SYNC REPLICA hybrid_test", timeout=30) - - assert ( - node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - assert ( - node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - - node1.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - - # Total objects in S3 - s3_objects = get_large_objects_count(cluster, size=0) - - node2.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - - # Check that after moving partition on node2 no new obects on s3 - wait_for_large_objects_count(cluster, s3_objects, size=0) - - assert ( - node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - assert ( - node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - node1.query("DROP TABLE IF EXISTS hybrid_test SYNC") - node2.query("DROP TABLE IF EXISTS hybrid_test SYNC") - - def insert_data_time(node, table, number_of_mb, time, start=0): values = ",".join( f"({x},{time})" diff --git a/tests/integration/test_s3_zero_copy_ttl/__init__.py b/tests/integration/test_s3_zero_copy_ttl/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml deleted file mode 100644 index 54f7152690b..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - - 10 - - - diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml deleted file mode 100644 index 7bb7fa875e4..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - - - - - - -
- default -
- - s3_disk - -
-
- - -
- s3_disk -
-
-
-
-
- - - true - 1.0 - - - true -
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py deleted file mode 100644 index 04bff4a44fb..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ /dev/null @@ -1,94 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) -node2 = cluster.add_instance( - "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) -node3 = cluster.add_instance( - "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_ttl_move_and_s3(started_cluster): - for i, node in enumerate([node1, node2, node3]): - node.query( - """ - CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') - ORDER BY id - PARTITION BY id - TTL date TO DISK 's3_disk' - SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 - """.format( - i - ) - ) - - node1.query("SYSTEM STOP MOVES s3_test_with_ttl") - - node2.query("SYSTEM STOP MOVES s3_test_with_ttl") - - for i in range(30): - if i % 2 == 0: - node = node1 - else: - node = node2 - - node.query( - f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" - ) - - node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - - assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - - node1.query("SYSTEM START MOVES s3_test_with_ttl") - node2.query("SYSTEM START MOVES s3_test_with_ttl") - - assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - - for attempt in reversed(range(5)): - time.sleep(5) - - print( - node1.query( - "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" - ) - ) - - minio = cluster.minio_client - objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) - counter = 0 - for obj in objects: - print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") - counter += 1 - - print(f"Total objects: {counter}") - - if counter == 330: - break - - print(f"Attempts remaining: {attempt}") - - assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py deleted file mode 100644 index 5fbe426074f..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py +++ /dev/null @@ -1,81 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/45887 - -pytestmark = pytest.mark.skip - -from helpers.cluster import ClickHouseCluster - - -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", main_configs=["configs/s3.xml"], with_minio=True -) - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_move_and_s3_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'" - ) - - for _ in range(10): - small_node.query( - "insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000" - ) - - # After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition - small_node.query( - "optimize table s3_test_with_ttl final", - settings={ - "send_logs_level": "error", - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - - small_node.query("system flush logs") - # Will take memory usage from metric_log. - # It is easier then specifying total memory limit (insert queries can hit this limit). - small_node.query("truncate table system.metric_log") - - small_node.query( - "alter table s3_test_with_ttl move partition 0 to volume 'external'", - settings={ - "send_logs_level": "error", - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - small_node.query("system flush logs") - max_usage = small_node.query( - """ - select max(m.val - am.val * 4096) from - (select toStartOfMinute(event_time) as time, max(CurrentMetric_MemoryTracking) as val from system.metric_log group by time) as m join - (select toStartOfMinute(event_time) as time, min(value) as val from system.asynchronous_metric_log where metric='jemalloc.arenas.all.pdirty' group by time) as am using time;""" - ) - # 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled. - # Also actual value can be bigger because of memory drift. - # Increase it a little bit if test fails. - assert int(max_usage) < 3e9 - res = small_node.query( - "select * from system.errors where last_error_message like '%Memory limit%' limit 1", - settings={ - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - assert res == "" diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py deleted file mode 100644 index fb9f3eb67b9..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest -from helpers.cluster import ClickHouseCluster - - -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", - main_configs=["configs/s3.xml"], - user_configs=["configs/max_delayed_streams.xml"], - with_minio=True, -) - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_vertical_merge_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" - ) - - small_node.query( - "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" - ) - small_node.query("optimize table tvm2 final") - small_node.query("system flush logs") - - # Should be about 25M - res = small_node.query( - "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" - ) - - assert res == "" diff --git a/tests/integration/test_storage_nats/__init__.py b/tests/integration/test_storage_nats/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto deleted file mode 100755 index 090ed917cdd..00000000000 --- a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto +++ /dev/null @@ -1,6 +0,0 @@ -syntax = "proto3"; - - message ProtoKeyValue { - uint64 key = 1; - string value = 2; - } diff --git a/tests/integration/test_storage_nats/configs/macros.xml b/tests/integration/test_storage_nats/configs/macros.xml deleted file mode 100644 index 4aa547e049e..00000000000 --- a/tests/integration/test_storage_nats/configs/macros.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - nats1:4444 - macro - JSONEachRow - - diff --git a/tests/integration/test_storage_nats/configs/named_collection.xml b/tests/integration/test_storage_nats/configs/named_collection.xml deleted file mode 100644 index 15817f321f0..00000000000 --- a/tests/integration/test_storage_nats/configs/named_collection.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - nats1:4444 - named - JSONEachRow - 111 - 12 - click - house - - - diff --git a/tests/integration/test_storage_nats/configs/nats.xml b/tests/integration/test_storage_nats/configs/nats.xml deleted file mode 100644 index 0a8be9122ad..00000000000 --- a/tests/integration/test_storage_nats/configs/nats.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - click - house - - diff --git a/tests/integration/test_storage_nats/configs/users.xml b/tests/integration/test_storage_nats/configs/users.xml deleted file mode 100644 index 2cef0a6de3c..00000000000 --- a/tests/integration/test_storage_nats/configs/users.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_storage_nats/nats_certs.sh b/tests/integration/test_storage_nats/nats_certs.sh deleted file mode 100755 index 689221c39e4..00000000000 --- a/tests/integration/test_storage_nats/nats_certs.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash -set -euxo pipefail - -mkdir -p "${NATS_CERT_DIR}/ca" -mkdir -p "${NATS_CERT_DIR}/nats" -openssl req -newkey rsa:4096 -x509 -days 365 -nodes -batch -keyout "${NATS_CERT_DIR}/ca/ca-key.pem" -out "${NATS_CERT_DIR}/ca/ca-cert.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca" -openssl req -newkey rsa:4096 -nodes -batch -keyout "${NATS_CERT_DIR}/nats/server-key.pem" -out "${NATS_CERT_DIR}/nats/server-req.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server" -openssl x509 -req -days 365 -in "${NATS_CERT_DIR}/nats/server-req.pem" -CA "${NATS_CERT_DIR}/ca/ca-cert.pem" -CAkey "${NATS_CERT_DIR}/ca/ca-key.pem" -CAcreateserial -out "${NATS_CERT_DIR}/nats/server-cert.pem" -extfile <( -cat <<-EOF -subjectAltName = DNS:localhost, DNS:nats1 -EOF -) -rm -f "${NATS_CERT_DIR}/nats/server-req.pem" diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py deleted file mode 100644 index e9e5cb72363..00000000000 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ /dev/null @@ -1,37 +0,0 @@ -# -*- coding: utf-8 -*- -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: clickhouse_path/format_schemas/nats.proto -"""Generated protocol buffer code.""" -from google.protobuf import descriptor as _descriptor -from google.protobuf import descriptor_pool as _descriptor_pool -from google.protobuf import message as _message -from google.protobuf import reflection as _reflection -from google.protobuf import symbol_database as _symbol_database - -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n)clickhouse_path/format_schemas/nats.proto"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' -) - - -_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name["ProtoKeyValue"] -ProtoKeyValue = _reflection.GeneratedProtocolMessageType( - "ProtoKeyValue", - (_message.Message,), - { - "DESCRIPTOR": _PROTOKEYVALUE, - "__module__": "clickhouse_path.format_schemas.nats_pb2" - # @@protoc_insertion_point(class_scope:ProtoKeyValue) - }, -) -_sym_db.RegisterMessage(ProtoKeyValue) - -if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _PROTOKEYVALUE._serialized_start = 45 - _PROTOKEYVALUE._serialized_end = 88 -# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py deleted file mode 100644 index 4d7e4cf813d..00000000000 --- a/tests/integration/test_storage_nats/test.py +++ /dev/null @@ -1,1875 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/39185 - -pytestmark = pytest.mark.skip - -import json -import os.path as p -import random -import subprocess -import threading -import logging -import time -from random import randrange -import math - -import asyncio -from google.protobuf.internal.encoder import _VarintBytes -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, check_nats_is_available, nats_connect_ssl -from helpers.test_tools import TSV - -from . import nats_pb2 - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - main_configs=[ - "configs/nats.xml", - "configs/macros.xml", - "configs/named_collection.xml", - ], - user_configs=["configs/users.xml"], - with_nats=True, - clickhouse_path_dir="clickhouse_path", -) - - -# Helpers - - -def wait_nats_to_start(nats_port, ssl_ctx=None, timeout=180): - start = time.time() - while time.time() - start < timeout: - try: - if asyncio.run(check_nats_is_available(nats_port, ssl_ctx=ssl_ctx)): - logging.debug("NATS is available") - return - time.sleep(0.5) - except Exception as ex: - logging.debug("Can't connect to NATS " + str(ex)) - time.sleep(0.5) - - -def nats_check_result(result, check=False, ref_file="test_nats_json.reference"): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -def kill_nats(nats_id): - p = subprocess.Popen(("docker", "stop", nats_id), stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def revive_nats(nats_id, nats_port): - p = subprocess.Popen(("docker", "start", nats_id), stdout=subprocess.PIPE) - p.communicate() - wait_nats_to_start(nats_port) - - -# Fixtures - - -@pytest.fixture(scope="module") -def nats_cluster(): - try: - cluster.start() - logging.debug("nats_id is {}".format(instance.cluster.nats_docker_id)) - instance.query("CREATE DATABASE test") - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture(autouse=True) -def nats_setup_teardown(): - print("NATS is available - running test") - yield # run test - instance.query("DROP DATABASE test SYNC") - instance.query("CREATE DATABASE test") - - -# Tests - - -async def nats_produce_messages(cluster_inst, subject, messages=(), bytes=None): - nc = await nats_connect_ssl( - cluster_inst.nats_port, - user="click", - password="house", - ssl_ctx=cluster_inst.nats_ssl_context, - ) - logging.debug("NATS connection status: " + str(nc.is_connected)) - - for message in messages: - await nc.publish(subject, message.encode()) - if bytes is not None: - await nc.publish(subject, bytes) - logging.debug("Finished publising to " + subject) - - await nc.close() - return messages - - -def check_table_is_ready(instance, table_name): - try: - instance.query("SELECT * FROM {}".format(table_name)) - return True - except Exception: - return False - - -def test_nats_select(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'select', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "select", messages)) - - # The order of messages in select * from test.nats is not guaranteed, so sleep to collect everything in one select - time.sleep(1) - - result = "" - while True: - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_select_empty(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'empty', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - - assert int(instance.query("SELECT count() FROM test.nats")) == 0 - - -def test_nats_json_without_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'json', - nats_format = 'JSONEachRow'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = "" - for i in range(25): - messages += json.dumps({"key": i, "value": i}) + "\n" - - all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster, "json", all_messages)) - - messages = "" - for i in range(25, 50): - messages += json.dumps({"key": i, "value": i}) + "\n" - all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster, "json", all_messages)) - - time.sleep(1) - - result = "" - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_csv_with_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'csv', - nats_format = 'CSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append("{i}, {i}".format(i=i)) - - asyncio.run(nats_produce_messages(nats_cluster, "csv", messages)) - - time.sleep(1) - - result = "" - for _ in range(60): - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_tsv_with_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'tsv', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append("{i}\t{i}".format(i=i)) - - asyncio.run(nats_produce_messages(nats_cluster, "tsv", messages)) - - result = "" - for _ in range(60): - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -# - - -def test_nats_macros(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = '{nats_url}', - nats_subjects = '{nats_subjects}', - nats_format = '{nats_format}' - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - message = "" - for i in range(50): - message += json.dumps({"key": i, "value": i}) + "\n" - asyncio.run(nats_produce_messages(nats_cluster, "macro", [message])) - - time.sleep(1) - - result = "" - for _ in range(60): - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_materialized_view(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mv', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.nats group by (key, value); - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view2 ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_materialized_view_with_subquery(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mvsq', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM (SELECT * FROM test.nats); - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mvsq", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_many_materialized_views(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view1; - DROP TABLE IF EXISTS test.view2; - DROP TABLE IF EXISTS test.consumer1; - DROP TABLE IF EXISTS test.consumer2; - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mmv', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view1 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS - SELECT * FROM test.nats; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mmv", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result1 = instance.query("SELECT * FROM test.view1 ORDER BY key") - result2 = instance.query("SELECT * FROM test.view2 ORDER BY key") - if nats_check_result(result1) and nats_check_result(result2): - break - - instance.query( - """ - DROP TABLE test.consumer1; - DROP TABLE test.consumer2; - DROP TABLE test.view1; - DROP TABLE test.view2; - """ - ) - - nats_check_result(result1, True) - nats_check_result(result2, True) - - -def test_nats_protobuf(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value String) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'pb', - nats_format = 'Protobuf', - nats_schema = 'nats.proto:ProtoKeyValue'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - data = b"" - for i in range(0, 20): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - data = b"" - for i in range(20, 21): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - data = b"" - for i in range(21, 50): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - - result = "" - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_big_message(nats_cluster): - # Create batchs of messages of size ~100Kb - nats_messages = 1000 - batch_messages = 1000 - messages = [ - json.dumps({"key": i, "value": "x" * 100}) * batch_messages - for i in range(nats_messages) - ] - - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value String) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'big', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - asyncio.run(nats_produce_messages(nats_cluster, "big", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - if int(result) == batch_messages * nats_messages: - break - - assert ( - int(result) == nats_messages * batch_messages - ), "ClickHouse lost some messages: {}".format(result) - - -def test_nats_mv_combo(nats_cluster): - NUM_MV = 5 - NUM_CONSUMERS = 4 - - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'combo', - nats_num_consumers = {}, - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """.format( - NUM_CONSUMERS - ) - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - for mv_id in range(NUM_MV): - instance.query( - """ - DROP TABLE IF EXISTS test.combo_{0}; - DROP TABLE IF EXISTS test.combo_{0}_mv; - CREATE TABLE test.combo_{0} (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.combo_{0}_mv TO test.combo_{0} AS - SELECT * FROM test.nats; - """.format( - mv_id - ) - ) - - time.sleep(2) - - i = [0] - messages_num = 10000 - - def produce(): - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({"key": i[0], "value": i[0]})) - i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster, "combo", messages)) - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = 0 - for mv_id in range(NUM_MV): - result += int( - instance.query("SELECT count() FROM test.combo_{0}".format(mv_id)) - ) - if int(result) == messages_num * threads_num * NUM_MV: - break - time.sleep(1) - - for thread in threads: - thread.join() - - for mv_id in range(NUM_MV): - instance.query( - """ - DROP TABLE test.combo_{0}_mv; - DROP TABLE test.combo_{0}; - """.format( - mv_id - ) - ) - - assert ( - int(result) == messages_num * threads_num * NUM_MV - ), "ClickHouse lost some messages: {}".format(result) - - -def test_nats_insert(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'insert', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("insert") - await sub.unsubscribe(50) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - while True: - try: - instance.query("INSERT INTO test.nats VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - thread.join() - - result = "\n".join(insert_messages) - nats_check_result(result, True) - - -def test_nats_many_subjects_insert_wrong(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'insert1,insert2.>,insert3.*.foo', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - # no subject specified - instance.query_and_get_error("INSERT INTO test.nats VALUES {}".format(values)) - - # can't insert into wildcard subjects - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert2.>' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.*.foo' VALUES {}".format( - values - ) - ) - - # specified subject is not among engine's subjects - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert4' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.foo.baz' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='foo.insert2' VALUES {}".format( - values - ) - ) - - -def test_nats_many_subjects_insert_right(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'right_insert1,right_insert2', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("right_insert1") - await sub.unsubscribe(50) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - while True: - try: - instance.query( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='right_insert1' VALUES {}".format( - values - ) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - thread.join() - - result = "\n".join(insert_messages) - nats_check_result(result, True) - - -def test_nats_many_inserts(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats_many; - DROP TABLE IF EXISTS test.nats_consume; - DROP TABLE IF EXISTS test.view_many; - DROP TABLE IF EXISTS test.consumer_many; - CREATE TABLE test.nats_many (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_inserts', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.nats_consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_inserts', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view_many (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS - SELECT * FROM test.nats_consume; - """ - ) - while not check_table_is_ready(instance, "test.nats_consume"): - logging.debug("Table test.nats_consume is not yet ready") - time.sleep(0.5) - - messages_num = 10000 - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - def insert(): - while True: - try: - instance.query("INSERT INTO test.nats_many VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 10 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() - - time_limit_sec = 300 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT count() FROM test.view_many") - print(result, messages_num * threads_num) - if int(result) >= messages_num * threads_num: - break - time.sleep(1) - - instance.query( - """ - DROP TABLE test.nats_consume; - DROP TABLE test.nats_many; - DROP TABLE test.consumer_many; - DROP TABLE test.view_many; - """ - ) - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( - result - ) - - -def test_nats_overloaded_insert(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view_overload; - DROP TABLE IF EXISTS test.consumer_overload; - DROP TABLE IF EXISTS test.nats_consume; - CREATE TABLE test.nats_consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'over', - nats_num_consumers = 5, - nats_max_block_size = 10000, - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.nats_overload (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'over', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view_overload (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3, - cleanup_thread_preferred_points_per_iteration=0; - CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS - SELECT * FROM test.nats_consume; - """ - ) - while not check_table_is_ready(instance, "test.nats_consume"): - logging.debug("Table test.nats_consume is not yet ready") - time.sleep(0.5) - - messages_num = 100000 - - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query( - "INSERT INTO test.nats_overload VALUES {}".format(values) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 5 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - time_limit_sec = 300 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT count() FROM test.view_overload") - time.sleep(1) - if int(result) >= messages_num * threads_num: - break - - instance.query( - """ - DROP TABLE test.consumer_overload; - DROP TABLE test.view_overload; - DROP TABLE test.nats_consume; - DROP TABLE test.nats_overload; - """ - ) - - for thread in threads: - thread.join() - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( - result - ) - - -def test_nats_virtual_column(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats_virtuals (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'virtuals', - nats_format = 'JSONEachRow'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _subject FROM test.nats_virtuals; - """ - ) - while not check_table_is_ready(instance, "test.nats_virtuals"): - logging.debug("Table test.nats_virtuals is not yet ready") - time.sleep(0.5) - - message_num = 10 - i = 0 - messages = [] - for _ in range(message_num): - messages.append(json.dumps({"key": i, "value": i})) - i += 1 - - asyncio.run(nats_produce_messages(nats_cluster, "virtuals", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == message_num: - break - - result = instance.query( - """ - SELECT key, value, _subject - FROM test.view ORDER BY key - """ - ) - - expected = """\ -0 0 virtuals -1 1 virtuals -2 2 virtuals -3 3 virtuals -4 4 virtuals -5 5 virtuals -6 6 virtuals -7 7 virtuals -8 8 virtuals -9 9 virtuals -""" - - instance.query( - """ - DROP TABLE test.nats_virtuals; - DROP TABLE test.view; - """ - ) - - assert TSV(result) == TSV(expected) - - -def test_nats_virtual_column_with_materialized_view(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats_virtuals_mv (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'virtuals_mv', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64, subject String) ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _subject as subject - FROM test.nats_virtuals_mv; - """ - ) - while not check_table_is_ready(instance, "test.nats_virtuals_mv"): - logging.debug("Table test.nats_virtuals_mv is not yet ready") - time.sleep(0.5) - - message_num = 10 - i = 0 - messages = [] - for _ in range(message_num): - messages.append(json.dumps({"key": i, "value": i})) - i += 1 - - asyncio.run(nats_produce_messages(nats_cluster, "virtuals_mv", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == message_num: - break - - result = instance.query("SELECT key, value, subject FROM test.view ORDER BY key") - expected = """\ -0 0 virtuals_mv -1 1 virtuals_mv -2 2 virtuals_mv -3 3 virtuals_mv -4 4 virtuals_mv -5 5 virtuals_mv -6 6 virtuals_mv -7 7 virtuals_mv -8 8 virtuals_mv -9 9 virtuals_mv -""" - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - DROP TABLE test.nats_virtuals_mv - """ - ) - - assert TSV(result) == TSV(expected) - - -def test_nats_many_consumers_to_each_queue(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - """ - ) - - num_tables = 4 - for table_id in range(num_tables): - print(("Setting up table {}".format(table_id))) - instance.query( - """ - DROP TABLE IF EXISTS test.many_consumers_{0}; - DROP TABLE IF EXISTS test.many_consumers_{0}_mv; - CREATE TABLE test.many_consumers_{0} (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_consumers', - nats_num_consumers = 2, - nats_queue_group = 'many_consumers', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT key, value FROM test.many_consumers_{0}; - """.format( - table_id - ) - ) - while not check_table_is_ready( - instance, "test.many_consumers_{}".format(table_id) - ): - logging.debug( - "Table test.many_consumers_{} is not yet ready".format(table_id) - ) - time.sleep(0.5) - - i = [0] - messages_num = 1000 - - def produce(): - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({"key": i[0], "value": i[0]})) - i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster, "many_consumers", messages)) - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - result1 = "" - while True: - result1 = instance.query("SELECT count() FROM test.destination") - time.sleep(1) - if int(result1) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - for consumer_id in range(num_tables): - instance.query( - """ - DROP TABLE test.many_consumers_{0}; - DROP TABLE test.many_consumers_{0}_mv; - """.format( - consumer_id - ) - ) - - instance.query( - """ - DROP TABLE test.destination; - """ - ) - - assert ( - int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result1) - - -def test_nats_restore_failed_connection_without_losses_on_write(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.consume; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE TABLE test.consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'producer_reconnect', - nats_format = 'JSONEachRow', - nats_num_consumers = 2, - nats_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.consume; - DROP TABLE IF EXISTS test.producer_reconnect; - CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'producer_reconnect', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.consume"): - logging.debug("Table test.consume is not yet ready") - time.sleep(0.5) - - messages_num = 100000 - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query( - "INSERT INTO test.producer_reconnect VALUES {}".format(values) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - while int(instance.query("SELECT count() FROM test.view")) == 0: - time.sleep(0.1) - - kill_nats(nats_cluster.nats_docker_id) - time.sleep(4) - revive_nats(nats_cluster.nats_docker_id, nats_cluster.nats_port) - - while True: - result = instance.query("SELECT count(DISTINCT key) FROM test.view") - time.sleep(1) - if int(result) == messages_num: - break - - instance.query( - """ - DROP TABLE test.consume; - DROP TABLE test.producer_reconnect; - """ - ) - - assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( - result - ) - - -def test_nats_no_connection_at_startup_1(nats_cluster): - # no connection when table is initialized - nats_cluster.pause_container("nats1") - instance.query_and_get_error( - """ - CREATE TABLE test.cs (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'cs', - nats_format = 'JSONEachRow', - nats_num_consumers = '5', - nats_row_delimiter = '\\n'; - """ - ) - nats_cluster.unpause_container("nats1") - - -def test_nats_no_connection_at_startup_2(nats_cluster): - instance.query( - """ - CREATE TABLE test.cs (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'cs', - nats_format = 'JSONEachRow', - nats_num_consumers = '5', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.cs; - """ - ) - - instance.query("DETACH TABLE test.cs") - nats_cluster.pause_container("nats1") - instance.query("ATTACH TABLE test.cs") - nats_cluster.unpause_container("nats1") - while not check_table_is_ready(instance, "test.cs"): - logging.debug("Table test.cs is not yet ready") - time.sleep(0.5) - - messages_num = 1000 - messages = [] - for i in range(messages_num): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "cs", messages)) - - for _ in range(20): - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == messages_num: - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.cs; - """ - ) - - assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( - result - ) - - -def test_nats_format_factory_settings(nats_cluster): - instance.query( - """ - CREATE TABLE test.format_settings ( - id String, date DateTime - ) ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'format_settings', - nats_format = 'JSONEachRow', - date_time_input_format = 'best_effort'; - """ - ) - while not check_table_is_ready(instance, "test.format_settings"): - logging.debug("Table test.format_settings is not yet ready") - time.sleep(0.5) - - message = json.dumps( - {"id": "format_settings_test", "date": "2021-01-19T14:42:33.1829214Z"} - ) - expected = instance.query( - """SELECT parseDateTimeBestEffort(CAST('2021-01-19T14:42:33.1829214Z', 'String'))""" - ) - - asyncio.run(nats_produce_messages(nats_cluster, "format_settings", [message])) - - while True: - result = instance.query("SELECT date FROM test.format_settings") - if result == expected: - break - - instance.query( - """ - CREATE TABLE test.view ( - id String, date DateTime - ) ENGINE = MergeTree ORDER BY id; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.format_settings; - """ - ) - - asyncio.run(nats_produce_messages(nats_cluster, "format_settings", [message])) - while True: - result = instance.query("SELECT date FROM test.view") - if result == expected: - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.format_settings; - """ - ) - - assert result == expected - - -def test_nats_bad_args(nats_cluster): - instance.query_and_get_error( - """ - CREATE TABLE test.drop (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_secure = true, - nats_format = 'JSONEachRow'; - """ - ) - - -def test_nats_drop_mv(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mv', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(20): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - instance.query("DROP VIEW test.consumer") - messages = [] - for i in range(20, 40): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - instance.query( - """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - messages = [] - for i in range(40, 50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - while True: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - instance.query("DROP VIEW test.consumer") - messages = [] - for i in range(50, 60): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - count = 0 - while True: - count = int(instance.query("SELECT count() FROM test.nats")) - if count: - break - - assert count > 0 - - -def test_nats_predefined_configuration(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS(nats1) """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - asyncio.run( - nats_produce_messages( - nats_cluster, "named", [json.dumps({"key": 1, "value": 2})] - ) - ) - while True: - result = instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if result == "1\t2\n": - break - - -def test_format_with_prefix_and_suffix(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'custom', - nats_format = 'CustomSeparated'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("custom") - await sub.unsubscribe(2) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - "INSERT INTO test.nats select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - thread.join() - - assert ( - "".join(insert_messages) - == "\n0\t0\n\n\n10\t100\n\n" - ) - - -def test_max_rows_per_message(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'custom1', - nats_format = 'CustomSeparated', - nats_max_rows_per_message = 3, - format_custom_result_before_delimiter = '\n', - format_custom_result_after_delimiter = '\n'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - num_rows = 5 - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("custom1") - await sub.unsubscribe(2) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - thread.join() - - assert ( - "".join(insert_messages) - == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" - ) - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" - - -def test_row_based_formats(nats_cluster): - num_rows = 10 - - for format_name in [ - "TSV", - "TSVWithNamesAndTypes", - "TSKV", - "CSV", - "CSVWithNamesAndTypes", - "CustomSeparatedWithNamesAndTypes", - "Values", - "JSON", - "JSONEachRow", - "JSONCompactEachRow", - "JSONCompactEachRowWithNamesAndTypes", - "JSONObjectEachRow", - "Avro", - "RowBinary", - "RowBinaryWithNamesAndTypes", - "MsgPack", - ]: - print(format_name) - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = '{format_name}', - nats_format = '{format_name}'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = 0 - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe(format_name) - await sub.unsubscribe(2) - async for msg in sub.messages: - nonlocal insert_messages - insert_messages += 1 - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows})" - ) - - thread.join() - - assert insert_messages == 2 - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - - result = instance.query("SELECT * FROM test.view") - assert result == expected - - -def test_block_based_formats_1(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'PrettySpace', - nats_format = 'PrettySpace'; - """ - ) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("PrettySpace") - await sub.unsubscribe(3) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - attempt = 0 - while attempt < 100: - try: - instance.query( - "INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0;" - ) - break - except Exception: - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - attempt += 1 - thread.join() - - data = [] - for message in insert_messages: - splitted = message.split("\n") - assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" - assert splitted[1] == "" - assert splitted[-1] == "" - data += [line.split() for line in splitted[2:-1]] - - assert data == [ - ["0", "0"], - ["10", "100"], - ["20", "200"], - ["30", "300"], - ["40", "400"], - ] - - -def test_block_based_formats_2(nats_cluster): - num_rows = 100 - - for format_name in [ - "JSONColumns", - "Native", - "Arrow", - "Parquet", - "ORC", - "JSONCompactColumns", - ]: - print(format_name) - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = '{format_name}', - nats_format = '{format_name}'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = 0 - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe(format_name) - await sub.unsubscribe(9) - async for msg in sub.messages: - nonlocal insert_messages - insert_messages += 1 - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0;" - ) - - thread.join() - - assert insert_messages == 9 - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view ORDER by key") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_nats/test_nats_json.reference b/tests/integration/test_storage_nats/test_nats_json.reference deleted file mode 100644 index 959bb2aad74..00000000000 --- a/tests/integration/test_storage_nats/test_nats_json.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/test_storage_postgresql_replica/__init__.py b/tests/integration/test_storage_postgresql_replica/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml b/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml deleted file mode 100644 index 27c7107ce5e..00000000000 --- a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - trace - /var/log/clickhouse-server/log.log - /var/log/clickhouse-server/log.err.log - 1000M - 10 - /var/log/clickhouse-server/stderr.log - /var/log/clickhouse-server/stdout.log - - diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py deleted file mode 100644 index 66495700102..00000000000 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ /dev/null @@ -1,780 +0,0 @@ -import pytest - -# FIXME Tests with MaterializedPostgresSQL are temporarily disabled -# https://github.com/ClickHouse/ClickHouse/issues/36898 -# https://github.com/ClickHouse/ClickHouse/issues/38677 -# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190 - -pytestmark = pytest.mark.skip - -import time -import psycopg2 -import os.path as p - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry -from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT -from helpers.test_tools import TSV - -import threading - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - main_configs=["configs/log_conf.xml"], - with_postgres=True, - stay_alive=True, -) - -postgres_table_template = """ - CREATE TABLE IF NOT EXISTS {} ( - key Integer NOT NULL, value Integer, PRIMARY KEY(key)) - """ - -queries = [ - "INSERT INTO {} select i, i from generate_series(0, 10000) as t(i);", - "DELETE FROM {} WHERE (value*value) % 3 = 0;", - "UPDATE {} SET value = value + 125 WHERE key % 2 = 0;", - "UPDATE {} SET key=key+20000 WHERE key%2=0", - "INSERT INTO {} select i, i from generate_series(40000, 50000) as t(i);", - "DELETE FROM {} WHERE key % 10 = 0;", - "UPDATE {} SET value = value + 101 WHERE key % 2 = 1;", - "UPDATE {} SET key=key+80000 WHERE key%2=1", - "DELETE FROM {} WHERE value % 2 = 0;", - "UPDATE {} SET value = value + 2000 WHERE key % 5 = 0;", - "INSERT INTO {} select i, i from generate_series(200000, 250000) as t(i);", - "DELETE FROM {} WHERE value % 3 = 0;", - "UPDATE {} SET value = value * 2 WHERE key % 3 = 0;", - "UPDATE {} SET key=key+500000 WHERE key%2=1", - "INSERT INTO {} select i, i from generate_series(1000000, 1050000) as t(i);", - "DELETE FROM {} WHERE value % 9 = 2;", - "UPDATE {} SET key=key+10000000", - "UPDATE {} SET value = value + 2 WHERE key % 3 = 1;", - "DELETE FROM {} WHERE value%5 = 0;", -] - - -@pytest.mark.timeout(30) -def check_tables_are_synchronized( - table_name, order_by="key", postgres_database="postgres_database" -): - while True: - expected = instance.query( - "select * from {}.{} order by {};".format( - postgres_database, table_name, order_by - ) - ) - result = instance.query( - "select * from test.{} order by {};".format(table_name, order_by) - ) - if result == expected: - break - - assert result == expected - - -def get_postgres_conn( - ip, port, database=False, auto_commit=True, database_name="postgres_database" -): - if database == True: - conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format( - ip, port, database_name - ) - else: - conn_string = ( - "host={} port={} user='postgres' password='mysecretpassword'".format( - ip, port - ) - ) - - conn = psycopg2.connect(conn_string) - if auto_commit: - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True - return conn - - -def create_postgres_db(cursor, name): - cursor.execute("CREATE DATABASE {}".format(name)) - - -def create_clickhouse_postgres_db(ip, port, name="postgres_database"): - instance.query( - """ - CREATE DATABASE {} - ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')""".format( - name, ip, port, name - ) - ) - - -def create_materialized_table(ip, port, table_name="postgresql_replica"): - instance.query( - f""" - CREATE TABLE test.{table_name} (key Int64, value Int64) - ENGINE = MaterializedPostgreSQL( - '{ip}:{port}', 'postgres_database', '{table_name}', 'postgres', 'mysecretpassword') - PRIMARY KEY key; """ - ) - - -def create_postgres_table(cursor, table_name, replica_identity_full=False): - cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) - cursor.execute(postgres_table_template.format(table_name)) - if replica_identity_full: - cursor.execute("ALTER TABLE {} REPLICA IDENTITY FULL;".format(table_name)) - - -def postgresql_replica_check_result( - result, check=False, ref_file="test_postgresql_replica.reference" -): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - conn = get_postgres_conn(ip=cluster.postgres_ip, port=cluster.postgres_port) - cursor = conn.cursor() - create_postgres_db(cursor, "postgres_database") - create_clickhouse_postgres_db( - ip=cluster.postgres_ip, port=cluster.postgres_port - ) - - instance.query("CREATE DATABASE test") - yield cluster - - finally: - cluster.shutdown() - - -@pytest.mark.timeout(320) -def test_initial_load_from_snapshot(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.2) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_no_connection_at_startup(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - time.sleep(3) - - instance.query("DETACH TABLE test.postgresql_replica") - started_cluster.pause_container("postgres1") - - instance.query("ATTACH TABLE test.postgresql_replica") - time.sleep(3) - started_cluster.unpause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) == 0: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_detach_attach_is_ok(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) == 0: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - postgresql_replica_check_result(result, True) - - instance.query("DETACH TABLE test.postgresql_replica") - instance.query("ATTACH TABLE test.postgresql_replica") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_replicating_insert_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 10: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)" - ) - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 30: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)" - ) - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_replicating_delete_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.2) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_replicating_update_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("UPDATE postgresql_replica SET value = value - 10;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_resume_from_written_version(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query("DETACH TABLE test.postgresql_replica") - - cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;") - cursor.execute("UPDATE postgresql_replica SET value = value - 10;") - - instance.query("ATTACH TABLE test.postgresql_replica") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_many_replication_messages(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100000: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print("SYNC OK") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 200000: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print("INSERT OK") - - result = instance.query("SELECT key FROM test.postgresql_replica ORDER BY key;") - expected = instance.query("SELECT number from numbers(200000)") - assert result == expected - - cursor.execute("UPDATE postgresql_replica SET value = key + 1 WHERE key < 100000;") - - result = instance.query( - "SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;" - ) - expected = instance.query("SELECT number from numbers(100000)") - - while result != expected: - time.sleep(1) - result = instance.query( - "SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;" - ) - print("UPDATE OK") - - cursor.execute("DELETE FROM postgresql_replica WHERE key % 2 = 1;") - cursor.execute("DELETE FROM postgresql_replica WHERE key != value;") - - result = instance.query( - "SELECT count() FROM (SELECT * FROM test.postgresql_replica);" - ) - while int(result) != 50000: - time.sleep(1) - result = instance.query( - "SELECT count() FROM (SELECT * FROM test.postgresql_replica);" - ) - print("DELETE OK") - - cursor.execute("DROP TABLE postgresql_replica;") - - -@pytest.mark.timeout(320) -def test_connection_loss(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - i = 50 - while i < 100000: - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format( - i - ) - ) - i += 10000 - - started_cluster.pause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print(int(result)) - time.sleep(6) - - started_cluster.unpause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) < 100050: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DROP TABLE postgresql_replica;") - assert int(result) == 100050 - - -@pytest.mark.timeout(320) -def test_clickhouse_restart(started_cluster): - pytest.skip("Temporary disabled (FIXME)") - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - i = 50 - while i < 100000: - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format( - i - ) - ) - i += 10000 - - instance.restart_clickhouse() - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) < 100050: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DROP TABLE postgresql_replica;") - print(result) - assert int(result) == 100050 - - -def test_rename_table(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 25: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed" - ) - assert ( - int(instance.query("SELECT count() FROM test.postgresql_replica_renamed;")) - == 25 - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25, 25)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;") - while int(result) != 50: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;") - - result = instance.query( - "SELECT * FROM test.postgresql_replica_renamed ORDER BY key;" - ) - postgresql_replica_check_result(result, True) - cursor.execute("DROP TABLE postgresql_replica;") - instance.query("DROP TABLE IF EXISTS test.postgresql_replica_renamed") - - -def test_virtual_columns(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)" - ) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 10: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - # just check that it works, no check with `expected` because _version is taken as LSN, which will be different each time. - result = instance.query( - "SELECT key, value, _sign, _version FROM test.postgresql_replica;" - ) - print(result) - cursor.execute("DROP TABLE postgresql_replica;") - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -def test_abrupt_connection_loss_while_heavy_replication(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - table_name = "postgresql_replica" - create_postgres_table(cursor, table_name) - - instance.query(f"DROP TABLE IF EXISTS test.{table_name}") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - for i in range(len(queries)): - query = queries[i].format(table_name) - cursor.execute(query) - print("query {}".format(query.format(table_name))) - - started_cluster.pause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica") - print(result) # Just debug - - started_cluster.unpause_container("postgres1") - - check_tables_are_synchronized("postgresql_replica") - - result = instance.query("SELECT count() FROM test.postgresql_replica") - print(result) # Just debug - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled - if instance.is_built_with_sanitizer(): - pytest.skip("Temporary disabled (FIXME)") - - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - table_name = "postgresql_replica_697" - create_postgres_table(cursor, table_name) - - instance.query(f"INSERT INTO postgres_database.{table_name} SELECT -1, 1") - instance.query(f"DROP TABLE IF EXISTS test.{table_name} SYNC") - create_materialized_table( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - table_name=table_name, - ) - - n = 1 - while int(instance.query(f"select count() from test.{table_name}")) != 1: - sleep(1) - n += 1 - if n > 10: - break - - for query in queries: - cursor.execute(query.format(table_name)) - print("query {}".format(query.format(table_name))) - - instance.restart_clickhouse() - - result = instance.query(f"SELECT count() FROM test.{table_name}") - print(result) # Just debug - - check_tables_are_synchronized(table_name) - - result = instance.query(f"SELECT count() FROM test.{table_name}") - print(result) # Just debug - instance.query(f"DROP TABLE test.{table_name} SYNC") - - -def test_drop_table_immediately(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - instance.query("DROP TABLE test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - check_tables_are_synchronized("postgresql_replica") - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference b/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference deleted file mode 100644 index 959bb2aad74..00000000000 --- a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 751279f5e5a..80d2050b394 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -511,69 +511,6 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): rabbitmq_check_result(result2, True) -@pytest.mark.skip(reason="clichouse_path with rabbitmq.proto fails to be exported") -def test_rabbitmq_protobuf(rabbitmq_cluster): - instance.query( - """ - CREATE TABLE test.rabbitmq (key UInt64, value String) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'pb', - rabbitmq_format = 'Protobuf', - rabbitmq_flush_interval_ms=1000, - rabbitmq_max_block_size=100, - rabbitmq_schema = 'rabbitmq.proto:KeyValueProto'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - """ - ) - - credentials = pika.PlainCredentials("root", "clickhouse") - parameters = pika.ConnectionParameters( - rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials - ) - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - data = "" - for i in range(0, 20): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - data = "" - for i in range(20, 21): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - data = "" - for i in range(21, 50): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - - connection.close() - - result = "" - while True: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if rabbitmq_check_result(result): - break - - rabbitmq_check_result(result, True) - - def test_rabbitmq_big_message(rabbitmq_cluster): # Create batchs of messages of size ~100Kb rabbitmq_messages = 1000 diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index a2f28e21666..c1c076277bb 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1529,106 +1529,6 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): node1.query("DROP TABLE IF EXISTS {name} SYNC".format(name=name)) -@pytest.mark.skip(reason="Flacky test") -@pytest.mark.parametrize( - "name,positive", - [ - pytest.param("test_double_move_while_select_negative", 0, id="negative"), - pytest.param("test_double_move_while_select_positive", 1, id="positive"), - ], -) -def test_double_move_while_select(started_cluster, name, positive): - name = unique_table_name(name) - - try: - node1.query( - """ - CREATE TABLE {name} ( - n Int64, - s String - ) ENGINE = MergeTree - ORDER BY tuple() - PARTITION BY n - SETTINGS storage_policy='small_jbod_with_external',temporary_directories_lifetime=1 - """.format( - name=name - ) - ) - - node1.query( - "INSERT INTO {name} VALUES (1, randomPrintableASCII(10*1024*1024))".format( - name=name - ) - ) - - parts = node1.query( - "SELECT name FROM system.parts WHERE table = '{name}' AND active = 1".format( - name=name - ) - ).splitlines() - assert len(parts) == 1 - - node1.query( - "ALTER TABLE {name} MOVE PART '{part}' TO DISK 'external'".format( - name=name, part=parts[0] - ) - ) - - def long_select(): - if positive: - node1.query( - "SELECT sleep(3), sleep(2), sleep(1), n FROM {name}".format( - name=name - ) - ) - - thread = threading.Thread(target=long_select) - thread.start() - - time.sleep(1) - - node1.query( - "ALTER TABLE {name} MOVE PART '{part}' TO DISK 'jbod1'".format( - name=name, part=parts[0] - ) - ) - - # Fill jbod1 to force ClickHouse to make move of partition 1 to external. - node1.query( - "INSERT INTO {name} VALUES (2, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - node1.query( - "INSERT INTO {name} VALUES (3, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - node1.query( - "INSERT INTO {name} VALUES (4, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - - wait_parts_mover(node1, name, retry_count=40) - - # If SELECT locked old part on external, move shall fail. - assert node1.query( - "SELECT disk_name FROM system.parts WHERE table = '{name}' AND active = 1 AND name = '{part}'".format( - name=name, part=parts[0] - ) - ).splitlines() == ["jbod1" if positive else "external"] - - thread.join() - - assert node1.query( - "SELECT n FROM {name} ORDER BY n".format(name=name) - ).splitlines() == ["1", "2", "3", "4"] - - finally: - node1.query("DROP TABLE IF EXISTS {name} SYNC".format(name=name)) - - @pytest.mark.parametrize( "name,engine,positive", [ From f2c3000a93e70d3eb6994a79bfccc84c5ba275e4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Aug 2023 21:40:06 +0000 Subject: [PATCH 156/249] Improve exception in ALTER query --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ tests/queries/0_stateless/02834_alter_exception.reference | 0 tests/queries/0_stateless/02834_alter_exception.sql | 4 ++++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02834_alter_exception.reference create mode 100644 tests/queries/0_stateless/02834_alter_exception.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6179c70ca57..8938a3b6fbd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3166,6 +3166,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } + if (command.type == AlterCommand::MODIFY_QUERY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ALTER MODIFY QUERY is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/tests/queries/0_stateless/02834_alter_exception.reference b/tests/queries/0_stateless/02834_alter_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02834_alter_exception.sql b/tests/queries/0_stateless/02834_alter_exception.sql new file mode 100644 index 00000000000..d42f40fcbf7 --- /dev/null +++ b/tests/queries/0_stateless/02834_alter_exception.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS alter_02834; +CREATE TABLE alter_02834 (a UInt64) ENGINE=MergeTree() ORDER BY a; +ALTER TABLE alter_02834 MODIFY QUERY SELECT a FROM alter_02834; -- { serverError NOT_IMPLEMENTED } +DROP TABLE alter_02834; From ce30b93b34348700351e0d462edb5baa6ddb5685 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 23:46:45 +0200 Subject: [PATCH 157/249] Move repeated code for ClickHouseHelper to _insert_post --- tests/ci/clickhouse_helper.py | 63 ++++++++--------------------------- 1 file changed, 14 insertions(+), 49 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index dbf26288469..b165cda37e8 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -47,45 +47,9 @@ class ClickHouseHelper: params[k] = v with open(file, "rb") as data_fd: - for i in range(5): - try: - response = requests.post( - url, params=params, data=data_fd, headers=auth - ) - except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" - logging.warning(error) - continue - - logging.info("Response content '%s'", response.content) - - if response.ok: - break - - error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" - ) - - if response.status_code >= 500: - # A retriable error - time.sleep(1) - continue - - logging.info( - "Request headers '%s', body '%s'", - response.request.headers, - response.request.body, - ) - - raise InsertException(error) - else: - raise InsertException(error) + ClickHouseHelper._insert_post( + url, params=params, data=data_fd, headers=auth + ) @staticmethod def insert_json_str(url, auth, db, table, json_str): @@ -95,12 +59,18 @@ class ClickHouseHelper: "date_time_input_format": "best_effort", "send_logs_level": "warning", } + ClickHouseHelper._insert_post(url, params=params, data=json_str, headers=auth) + + @staticmethod + def _insert_post(*args, **kwargs): + url = "" + if args: + url = args[0] + url = kwargs.get("url", url) for i in range(5): try: - response = requests.post( - url, params=params, data=json_str, headers=auth - ) + response = requests.post(*args, **kwargs) except Exception as e: error = f"Received exception while sending data to {url} on {i} attempt: {e}" logging.warning(error) @@ -112,13 +82,8 @@ class ClickHouseHelper: break error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" + f"Cannot insert data into clickhouse at try {i}: HTTP code " + f"{response.status_code}: '{response.text}'" ) if response.status_code >= 500: From 8c63088a31d2b1d829a356f810392f9cf1484e93 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Aug 2023 21:54:58 +0000 Subject: [PATCH 158/249] Fixes --- .../02415_all_new_functions_must_be_documented.reference | 1 + utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..231f268ba57 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -881,6 +881,7 @@ tumble tumbleEnd tumbleStart tuple +tupleConcat tupleDivide tupleDivideByNumber tupleElement diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..1a7e8e16bd1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2421,6 +2421,7 @@ tsv tui tumbleEnd tumbleStart +tupleConcat tupleDivide tupleDivideByNumber tupleElement From a79f2f140d25657a248f0d96bbe0241065a4260d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 00:19:24 +0200 Subject: [PATCH 159/249] Fix bad test `00417_kill_query` --- .../0_stateless/00417_kill_query.reference | 4 +-- tests/queries/0_stateless/00417_kill_query.sh | 25 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00417_kill_query.reference b/tests/queries/0_stateless/00417_kill_query.reference index 1a3b47964c0..0dd1d78ac82 100644 --- a/tests/queries/0_stateless/00417_kill_query.reference +++ b/tests/queries/0_stateless/00417_kill_query.reference @@ -1,2 +1,2 @@ -SELECT sleep(1) FROM system.numbers LIMIT 30 -SELECT sleep(1) FROM system.numbers LIMIT 31 +SELECT sleep(1) FROM system.numbers LIMIT 300 +SELECT sleep(1) FROM system.numbers LIMIT 301 diff --git a/tests/queries/0_stateless/00417_kill_query.sh b/tests/queries/0_stateless/00417_kill_query.sh index cd5b788a147..e64e93fd4a0 100755 --- a/tests/queries/0_stateless/00417_kill_query.sh +++ b/tests/queries/0_stateless/00417_kill_query.sh @@ -7,19 +7,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -QUERY_FIELND_NUM=4 +QUERY_FIELD_NUM=4 -$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 30" &>/dev/null & -sleep 1 -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELND_NUM +$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 300" &>/dev/null & -# 31 is for the query to be different from the previous one -$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 31" &>/dev/null & -sleep 1 -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query = 'SELECT sleep(1) FROM system.numbers LIMIT 31' ASYNC" | cut -f $QUERY_FIELND_NUM +while true +do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELD_NUM | grep '.' && break + sleep 0.1 +done + +# 31 is for the query to be different from the previous one +$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 301" &>/dev/null & + +while true +do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query = 'SELECT sleep(1) FROM system.numbers LIMIT 301' ASYNC" | cut -f $QUERY_FIELD_NUM | grep '.' && break + sleep 0.1 +done $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 ASYNC" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 SYNC FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 1 TEST" &>/dev/null - From 2802508c07917dc0b1d9b2d9ba91752c21a59cdf Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 22:27:49 +0000 Subject: [PATCH 160/249] different linker --- cmake/linux/toolchain-s390x.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index f5e1cf660c9..349d7fc83a6 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -23,7 +23,6 @@ set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 087cc5bfed6736fc31f29e20a16f8161e40018c6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 23:18:27 +0000 Subject: [PATCH 161/249] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 349d7fc83a6..188d9ed75b7 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=gold") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 65d3778b031dab996556e7e95597d7802daf3432 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 8 Aug 2023 20:15:04 +0000 Subject: [PATCH 162/249] Fix warning in test_replicated_database --- tests/integration/test_replicated_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 1235f7d34df..d0a04f40b69 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1226,7 +1226,7 @@ def test_force_synchronous_settings(started_cluster): def select_func(): dummy_node.query( - "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t" + "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t SETTINGS function_sleep_max_microseconds_per_block = 0" ) select_thread = threading.Thread(target=select_func) From eb61074e070587edb048fb02f26852dd3a2fd398 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 02:37:23 +0200 Subject: [PATCH 163/249] Fix test `02428_delete_with_settings` --- tests/queries/0_stateless/02428_delete_with_settings.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02428_delete_with_settings.sql b/tests/queries/0_stateless/02428_delete_with_settings.sql index 071a3f74184..618c08608fc 100644 --- a/tests/queries/0_stateless/02428_delete_with_settings.sql +++ b/tests/queries/0_stateless/02428_delete_with_settings.sql @@ -1,5 +1,5 @@ drop table if exists test; -create table test (id Int32, key String) engine=MergeTree() order by tuple(); +create table test (id Int32, key String) engine=MergeTree() order by tuple() settings index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into test select number, toString(number) from numbers(1000000); delete from test where id % 2 = 0 SETTINGS mutations_sync=0; select count() from test; From a9b760a15bf284dbe20882ddc696bdf28efed061 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 00:40:06 +0000 Subject: [PATCH 164/249] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 188d9ed75b7..23d14d6ed26 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=gold") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=gold") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=bfd") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From c071f6755d70a5ce9f1924fc44a905cc3122da5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:10:03 +0300 Subject: [PATCH 165/249] Revert "Added new tests for session_log and fixed the inconsistency between login and logout." --- src/Interpreters/Session.cpp | 44 +-- src/Interpreters/Session.h | 2 - src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- .../test.py | 5 +- tests/integration/test_session_log/.gitignore | 1 - .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 - .../test_session_log/configs/ports.xml | 9 - .../test_session_log/configs/session_log.xml | 9 - .../test_session_log/configs/users.xml | 23 -- .../protos/clickhouse_grpc.proto | 1 - tests/integration/test_session_log/test.py | 289 ------------------ .../02833_concurrrent_sessions.reference | 34 --- .../0_stateless/02833_concurrrent_sessions.sh | 138 --------- .../02834_remote_session_log.reference | 13 - .../0_stateless/02834_remote_session_log.sh | 56 ---- .../02835_drop_user_during_session.reference | 8 - .../02835_drop_user_during_session.sh | 113 ------- 20 files changed, 27 insertions(+), 753 deletions(-) delete mode 100644 tests/integration/test_session_log/.gitignore delete mode 100644 tests/integration/test_session_log/__init__.py delete mode 100644 tests/integration/test_session_log/configs/log.xml delete mode 100644 tests/integration/test_session_log/configs/ports.xml delete mode 100644 tests/integration/test_session_log/configs/session_log.xml delete mode 100644 tests/integration/test_session_log/configs/users.xml delete mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto delete mode 100644 tests/integration/test_session_log/test.py delete mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference delete mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh delete mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference delete mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh delete mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference delete mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 0a6435cff75..de2a779b740 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,8 +520,6 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); - recordLoginSucess(session_context); - return session_context; } @@ -584,8 +582,6 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); - recordLoginSucess(session_context); - return session_context; } @@ -659,38 +655,24 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - /// Interserver does not create session context - recordLoginSucess(query_context); + if (!notified_session_log_about_login) + { + if (auto session_log = getSessionLog()) + { + session_log->addLoginSuccess( + auth_id, + named_session ? std::optional(named_session->key.second) : std::nullopt, + *query_context, + user); + + notified_session_log_about_login = true; + } + } return query_context; } -void Session::recordLoginSucess(ContextPtr login_context) const -{ - if (notified_session_log_about_login) - return; - - if (!login_context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); - - if (auto session_log = getSessionLog()) - { - const auto & settings = login_context->getSettingsRef(); - const auto access = login_context->getAccess(); - - session_log->addLoginSuccess(auth_id, - named_session ? named_session->key.second : "", - settings, - access, - getClientInfo(), - user); - - notified_session_log_about_login = true; - } -} - - void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 81ef987b428..51c0e3c71fa 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,8 +97,6 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; - void recordLoginSucess(ContextPtr login_context) const; - mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index b847eaf9824..0a8a7fc18c5 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,13 +199,12 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, - const String & session_id, - const Settings & settings, - const ContextAccessPtr & access, - const ClientInfo & client_info, - const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) { + const auto access = login_context.getAccess(); + const auto & settings = login_context.getSettingsRef(); + const auto & client_info = login_context.getClientInfo(); + DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -216,7 +215,8 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - log_entry.session_id = session_id; + if (session_id) + log_entry.session_id = *session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 8757bc12270..1282ac09c4d 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,7 +20,6 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; -using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -73,13 +72,7 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, - const String & session_id, - const Settings & settings, - const ContextAccessPtr & access, - const ClientInfo & client_info, - const UserPtr & login_user); - + void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 10d5e7a0242..ff5690a3b07 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,7 +561,8 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto context = session->makeQueryContext(); + auto client_info = session->getClientInfo(); + auto context = session->makeQueryContext(std::move(client_info)); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 65587933fed..2930262f63e 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,7 +27,10 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ + {proto_dir}/clickhouse_grpc.proto".format( + proto_dir=proto_dir, gen_dir=gen_dir + ), shell=True, ) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore deleted file mode 100644 index edf565ec632..00000000000 --- a/tests/integration/test_session_log/.gitignore +++ /dev/null @@ -1 +0,0 @@ -_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml deleted file mode 100644 index 7a079b81e69..00000000000 --- a/tests/integration/test_session_log/configs/log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml deleted file mode 100644 index fbaefc16b3a..00000000000 --- a/tests/integration/test_session_log/configs/ports.xml +++ /dev/null @@ -1,9 +0,0 @@ - - 5433 - 9001 - 9100 - - - false - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml deleted file mode 100644 index a0e4e3e2216..00000000000 --- a/tests/integration/test_session_log/configs/session_log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - system - session_log
- - toYYYYMM(event_date) - 7500 -
-
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml deleted file mode 100644 index 0416dfadc8a..00000000000 --- a/tests/integration/test_session_log/configs/users.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - 0 - - - - - - - pass - - - pass - - - pass - - - pass - - - \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto deleted file mode 120000 index 25d15f11e3b..00000000000 --- a/tests/integration/test_session_log/protos/clickhouse_grpc.proto +++ /dev/null @@ -1 +0,0 @@ -../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py deleted file mode 100644 index dbb39993ce3..00000000000 --- a/tests/integration/test_session_log/test.py +++ /dev/null @@ -1,289 +0,0 @@ -import os - -import grpc -import pymysql.connections -import psycopg2 as py_psql -import pytest -import random -import sys -import threading - -from helpers.cluster import ClickHouseCluster, run_and_check - -POSTGRES_SERVER_PORT = 5433 -MYSQL_SERVER_PORT = 9001 -GRPC_PORT = 9100 -SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DEFAULT_ENCODING = "utf-8" - -# Use grpcio-tools to generate *pb2.py files from *.proto. -proto_dir = os.path.join(SCRIPT_DIR, "./protos") -gen_dir = os.path.join(SCRIPT_DIR, "./_gen") -os.makedirs(gen_dir, exist_ok=True) -run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", - shell=True, -) - -sys.path.append(gen_dir) - -import clickhouse_grpc_pb2 -import clickhouse_grpc_pb2_grpc - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "node", - main_configs=[ - "configs/ports.xml", - "configs/log.xml", - "configs/session_log.xml", - ], - user_configs=["configs/users.xml"], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") - }, -) - - -def grpc_get_url(): - return f"{instance.ip_address}:{GRPC_PORT}" - - -def grpc_create_insecure_channel(): - channel = grpc.insecure_channel(grpc_get_url()) - grpc.channel_ready_future(channel).result(timeout=2) - return channel - - -session_id_counter = 0 - - -def next_session_id(): - global session_id_counter - session_id = session_id_counter - session_id_counter += 1 - return str(session_id) - - -def grpc_query(query, user_, pass_, raise_exception): - try: - query_info = clickhouse_grpc_pb2.QueryInfo( - query=query, - session_id=next_session_id(), - user_name=user_, - password=pass_, - ) - channel = grpc_create_insecure_channel() - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) - result = stub.ExecuteQuery(query_info) - if result and result.HasField("exception"): - raise Exception(result.exception.display_text) - - return result.output.decode(DEFAULT_ENCODING) - except Exception: - assert raise_exception - - -def postgres_query(query, user_, pass_, raise_exception): - try: - client = py_psql.connect( - host=instance.ip_address, - port=POSTGRES_SERVER_PORT, - user=user_, - password=pass_, - database="default", - ) - cursor = client.cursor() - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -def mysql_query(query, user_, pass_, raise_exception): - try: - client = pymysql.connections.Connection( - host=instance.ip_address, - user=user_, - password=pass_, - database="default", - port=MYSQL_SERVER_PORT, - ) - cursor = client.cursor(pymysql.cursors.DictCursor) - if raise_exception: - with pytest.raises(Exception): - cursor.execute(query) - else: - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_parallel_sessions(started_cluster): - thread_list = [] - for _ in range(10): - # Sleep time does not significantly matter here, - # test should pass even without sleeping. - for function in [postgres_query, grpc_query, mysql_query]: - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "pass", - False, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "wrong_pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", - "pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - - for thread in thread_list: - thread.join() - - instance.query("SYSTEM FLUSH LOGS") - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" - ) - assert port_0_sessions == "90\n" - - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" - ) - assert port_0_sessions == "0\n" - - address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" - ) - assert address_0_sessions == "0\n" - - grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" - ) - assert grpc_sessions == "30\n" - - mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" - ) - assert mysql_sessions == "30\n" - - postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" - ) - assert postgres_sessions == "30\n" - - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" - - logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" - ) - assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference deleted file mode 100644 index bfe507e8eac..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ /dev/null @@ -1,34 +0,0 @@ -sessions: -150 -port_0_sessions: -0 -address_0_sessions: -0 -tcp_sessions -60 -http_sessions -30 -http_with_session_id_sessions -30 -my_sql_sessions -30 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh deleted file mode 100755 index d1d571c6985..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, long - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -# Each user uses a separate thread. -readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") -readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) - -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" - -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; -done - -# All _session functions execute in separate threads. -# These functions try to create a session with successful login and logout. -# Sleep a small, random amount of time to make concurrency more intense. -# and try to login with an invalid password. -function tcp_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' - done -} - -function http_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" - done -} - -function http_with_session_id_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" - done -} - -function mysql_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" - - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; -export -f mysql_session; - -for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & -done - -for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & -done - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" - -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" - -echo "tcp_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" -echo "http_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "http_with_session_id_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" - echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" - done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference deleted file mode 100644 index e2680982ab0..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.reference +++ /dev/null @@ -1,13 +0,0 @@ -0 -0 -0 -0 -client_port 0 connections: -0 -client_address '::' connections: -0 -login failures: -0 -TCP Login and logout count is equal -HTTP Login and logout count is equal -MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh deleted file mode 100755 index 64f20a70cb1..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "client_port 0 connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" - -echo "client_address '::' connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" - -echo "login failures:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" - -# remote(...) function sometimes reuses old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. - -for interface in 'TCP' 'HTTP' 'MySQL' -do - LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` - CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` - - if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then - echo "${interface} Login and logout count is equal" - else - TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` - echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" - fi -done - -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference deleted file mode 100644 index 7252faab8c6..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.reference +++ /dev/null @@ -1,8 +0,0 @@ -port_0_sessions: -0 -address_0_sessions: -0 -Corresponding LoginSuccess/Logout -9 -LoginFailure -0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh deleted file mode 100755 index e848e1fe90d..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ /dev/null @@ -1,113 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -readonly TEST_USER="02835_USER_${PID}" -readonly TEST_ROLE="02835_ROLE_${PID}" -readonly TEST_PROFILE="02835_PROFILE_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -function tcp_session() -{ - local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" -} - -function http_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -function http_with_session_id_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. -function wait_for_queries_start() -{ - local user=$1 - local queries_count=$2 - # 10 seconds waiting - counter=0 retries=100 - while [[ $counter -lt $retries ]]; do - result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") - if [[ $result == "${queries_count}" ]]; then - break; - fi - sleep 0.1 - ((++counter)) - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -# DROP USE CASE -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP ROLE CASE -${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP PROFILE CASE -${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" -echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 6edf4a777f33a84db9c18b08741207251fa5fc3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:26:11 +0200 Subject: [PATCH 166/249] Remove unrecognizable garbage from the performance test --- docker/test/performance-comparison/compare.sh | 64 ++++++------------- 1 file changed, 21 insertions(+), 43 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 798d2a40b12..c0422816787 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -666,8 +666,8 @@ create view partial_query_times as select * from -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') settings output_format_decimal_trailing_zeros = 1 - as select toDecimal64(time_median, 3) time, - toDecimal64(time_stddev / time_median, 3) relative_time_stddev, + as select round(time_median, 3) time, + round(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name from partial_query_times join query_display_names using (test, query_index) @@ -746,21 +746,21 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') -- of NaN to decimal. (left > right ? left / right : right / left) as times_change_float, isFinite(times_change_float) as times_change_finite, - toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, times_change_finite ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' : '--' as times_change_str select - toDecimal64(left, 3), toDecimal64(right, 3), times_change_str, - toDecimal64(diff, 3), toDecimal64(stat_threshold, 3), + round(left, 3), round(right, 3), times_change_str, + round(diff, 3), round(stat_threshold, 3), changed_fail, test, query_index, query_display_name from queries where changed_show order by abs(diff) desc; create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') settings output_format_decimal_trailing_zeros = 1 as select - toDecimal64(left, 3), toDecimal64(right, 3), toDecimal64(diff, 3), - toDecimal64(stat_threshold, 3), unstable_fail, test, query_index, query_display_name + round(left, 3), round(right, 3), round(diff, 3), + round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name from queries where unstable_show order by stat_threshold desc; @@ -792,8 +792,8 @@ create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes settings output_format_decimal_trailing_zeros = 1 as with (times_speedup >= 1 - ? '-' || toString(toDecimal64(times_speedup, 3)) || 'x' - : '+' || toString(toDecimal64(1 / times_speedup, 3)) || 'x') + ? '-' || toString(round(times_speedup, 3)) || 'x' + : '+' || toString(round(1 / times_speedup, 3)) || 'x') as times_speedup_str select test, times_speedup_str, queries, bad, changed, unstable -- Not sure what's the precedence of UNION ALL vs WHERE & ORDER BY, hence all @@ -818,10 +818,10 @@ create view total_client_time_per_query as select * create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') settings output_format_decimal_trailing_zeros = 1 - as select client, server, toDecimal64(client/server, 3) p, + as select client, server, round(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) - where p > toDecimal64(1.02, 3) order by p desc; + where p > round(1.02, 3) order by p desc; create table wall_clock_time_per_test engine Memory as select * from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float'); @@ -902,12 +902,12 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') settings output_format_decimal_trailing_zeros = 1 as select test, - toDecimal64(real, 3), - toDecimal64(total_client_time, 3), + round(real, 3), + round(total_client_time, 3), queries, - toDecimal64(query_max, 3), - toDecimal64(avg_real_per_query, 3), - toDecimal64(query_min, 3), + round(query_max, 3), + round(avg_real_per_query, 3), + round(query_min, 3), runs from ( select * from test_times_view @@ -926,14 +926,14 @@ create table all_tests_report engine File(TSV, 'report/all-queries.tsv') -- of NaN to decimal. (left > right ? left / right : right / left) as times_change_float, isFinite(times_change_float) as times_change_finite, - toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, times_change_finite ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' : '--' as times_change_str select changed_fail, unstable_fail, - toDecimal64(left, 3), toDecimal64(right, 3), times_change_str, - toDecimal64(isFinite(diff) ? diff : 0, 3), - toDecimal64(isFinite(stat_threshold) ? stat_threshold : 0, 3), + round(left, 3), round(right, 3), times_change_str, + round(isFinite(diff) ? diff : 0, 3), + round(isFinite(stat_threshold) ? stat_threshold : 0, 3), test, query_index, query_display_name from queries order by test, query_index; @@ -1044,27 +1044,6 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, order by count() desc ; -create table metric_devation engine File(TSVWithNamesAndTypes, - 'report/metric-deviation.$version.tsv') - settings output_format_decimal_trailing_zeros = 1 - -- first goes the key used to split the file with grep - as select test, query_index, query_display_name, - toDecimal64(d, 3) d, q, metric - from ( - select - test, query_index, - (q[3] - q[1])/q[2] d, - quantilesExact(0, 0.5, 1)(value) q, metric - from (select * from unstable_run_metrics - union all select * from unstable_run_traces - union all select * from unstable_run_metrics_2) mm - group by test, query_index, metric - having isFinite(d) and d > 0.5 and q[3] > 5 - ) metrics - left join query_display_names using (test, query_index) - order by test, query_index, d desc - ; - create table stacks engine File(TSV, 'report/stacks.$version.tsv') as select -- first goes the key used to split the file with grep @@ -1175,7 +1154,7 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as create table changes engine File(TSV, 'metrics/changes.tsv') settings output_format_decimal_trailing_zeros = 1 as select metric, left, right, - toDecimal64(diff, 3), toDecimal64(times_diff, 3) + round(diff, 3), round(times_diff, 3) from ( select metric, median(left) as left, median(right) as right, (right - left) / left diff, @@ -1226,7 +1205,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') '$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha, '${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME:-Performance}' :: LowCardinality(String) AS check_name, '$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status, - -- TODO toDateTime() can't parse output of 'date', so no time for now. (($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms, fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time, test_name :: LowCardinality(String) AS test_name , From f2621e70aed39f2bf1a061b4820714f84bf7324a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:27:07 +0200 Subject: [PATCH 167/249] Remove unrecognizable garbage from the performance test --- docker/test/performance-comparison/compare.sh | 8 -------- 1 file changed, 8 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index c0422816787..f949e66ab17 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -665,7 +665,6 @@ create view partial_query_times as select * from -- Report for backward-incompatible ('partial') queries that we could only run on the new server (e.g. -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') - settings output_format_decimal_trailing_zeros = 1 as select round(time_median, 3) time, round(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name @@ -739,7 +738,6 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') ; create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') - settings output_format_decimal_trailing_zeros = 1 as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion @@ -757,7 +755,6 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') from queries where changed_show order by abs(diff) desc; create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') - settings output_format_decimal_trailing_zeros = 1 as select round(left, 3), round(right, 3), round(diff, 3), round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name @@ -789,7 +786,6 @@ create view total_speedup as ; create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') - settings output_format_decimal_trailing_zeros = 1 as with (times_speedup >= 1 ? '-' || toString(round(times_speedup, 3)) || 'x' @@ -817,7 +813,6 @@ create view total_client_time_per_query as select * 'test text, query_index int, client float, server float'); create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') - settings output_format_decimal_trailing_zeros = 1 as select client, server, round(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) @@ -899,7 +894,6 @@ create view test_times_view_total as ; create table test_times_report engine File(TSV, 'report/test-times.tsv') - settings output_format_decimal_trailing_zeros = 1 as select test, round(real, 3), @@ -919,7 +913,6 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') -- report for all queries page, only main metric create table all_tests_report engine File(TSV, 'report/all-queries.tsv') - settings output_format_decimal_trailing_zeros = 1 as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion @@ -1152,7 +1145,6 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as -- Show metrics that have changed create table changes engine File(TSV, 'metrics/changes.tsv') - settings output_format_decimal_trailing_zeros = 1 as select metric, left, right, round(diff, 3), round(times_diff, 3) from ( From 3c6d140c179c29ce890f190529e29d1e9ec3db8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 01:31:59 +0000 Subject: [PATCH 168/249] Fix sigle quote escaping in PostgreSQL engine --- src/Storages/StoragePostgreSQL.cpp | 5 +++++ tests/integration/test_storage_postgresql/test.py | 14 ++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 3551ee36819..11558b39ad3 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -45,6 +45,7 @@ #include +#include namespace DB { @@ -123,6 +124,10 @@ Pipe StoragePostgreSQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); + + /// Single quotes in PostgreSQL are escaped through repetition + boost::replace_all(query, "\\'", "''"); + LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 686eb1ea751..3a36d050f17 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,6 +726,20 @@ def test_auto_close_connection(started_cluster): assert count == 2 +def test_single_quotes(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + cursor.execute(f"DROP TABLE IF EXISTS single_quote_fails") + cursor.execute(f"CREATE TABLE single_quote_fails(text varchar(255))") + node1.query( + "CREATE TABLE default.single_quote_fails (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'single_quote_fails', 'postgres', 'mysecretpassword')" + ) + node1.query("SELECT * FROM single_quote_fails WHERE text = ''''") + node1.query("SELECT * FROM single_quote_fails WHERE text = '\\''") + node1.query("SELECT * FROM single_quote_fails WHERE text like '%a''a%'") + node1.query("SELECT * FROM single_quote_fails WHERE text like '%a\\'a%'") + cursor.execute(f"DROP TABLE single_quote_fails") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 4ba5273939804ed2f639cc4748eaaa7f3b9d4af2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:56:27 +0200 Subject: [PATCH 169/249] Disable randomization in `02273_full_sort_join` --- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 6500306356c..2a13e11e116 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-random-settings DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 0b138fdf16eeeebef564a20928e83e6a7b1d7122 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:15:19 +0200 Subject: [PATCH 170/249] Remove outdated Dockerfile --- docker/client/Dockerfile | 34 ---------------------------------- docker/client/README.md | 7 ------- 2 files changed, 41 deletions(-) delete mode 100644 docker/client/Dockerfile delete mode 100644 docker/client/README.md diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile deleted file mode 100644 index 1c185daec75..00000000000 --- a/docker/client/Dockerfile +++ /dev/null @@ -1,34 +0,0 @@ -FROM ubuntu:18.04 - -# ARG for quick switch to a given ubuntu mirror -ARG apt_archive="http://archive.ubuntu.com" -RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list - -ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=22.1.1.* - -RUN apt-get update \ - && apt-get install --yes --no-install-recommends \ - apt-transport-https \ - ca-certificates \ - dirmngr \ - gnupg \ - && mkdir -p /etc/apt/sources.list.d \ - && apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 \ - && echo $repository > /etc/apt/sources.list.d/clickhouse.list \ - && apt-get update \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get install --allow-unauthenticated --yes --no-install-recommends \ - clickhouse-client=$version \ - clickhouse-common-static=$version \ - locales \ - tzdata \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf \ - && apt-get clean - -RUN locale-gen en_US.UTF-8 -ENV LANG en_US.UTF-8 -ENV LANGUAGE en_US:en -ENV LC_ALL en_US.UTF-8 - -ENTRYPOINT ["/usr/bin/clickhouse-client"] diff --git a/docker/client/README.md b/docker/client/README.md deleted file mode 100644 index bbcc7d60794..00000000000 --- a/docker/client/README.md +++ /dev/null @@ -1,7 +0,0 @@ -# ClickHouse Client Docker Image - -For more information see [ClickHouse Server Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/). - -## License - -View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image. From fc3970e3ff46d6795ac811d51edb29c5b6128a2d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:30:03 +0200 Subject: [PATCH 171/249] Update readme --- docker/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..bae02201add 100644 --- a/docker/README.md +++ b/docker/README.md @@ -1,5 +1,5 @@ ## ClickHouse Dockerfiles -This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. +This directory contain Dockerfiles for `clickhouse-server`. They are updated in each release. Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. From 2a89587300a408610591cf251546a6c59affc034 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:32:39 +0200 Subject: [PATCH 172/249] Address review comments --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 290222c9d26..0ef25741b18 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -109,8 +109,8 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] fi -# Wait for the server to start -while true +# Wait for the server to start, but not for too long. +for _ in {1..100} do clickhouse-client --query "SELECT 1" && break sleep 1 From 8fba4d2c4a0bf6b9f9061bf19980f98158ef6846 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:41:19 +0200 Subject: [PATCH 173/249] Protection if the repository is old --- docker/test/stateless/run.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0ef25741b18..6c930fa583b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -117,7 +117,7 @@ do done # Initialize export of system logs to ClickHouse Cloud -if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" -a -f /repository/utils/export-logs-in-ci/setup.sh ] then export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" # TODO: Check if the password will appear in the logs. @@ -131,9 +131,6 @@ then export CLICKHOUSE_CI_LOGS_PASSWORD='' fi -# TODO what is this? Remove it. -sleep 5 - attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 function fn_exists() { From 0ba3fea7fefa1237434e1cca36b8d8ac11fb73e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:44:20 +0200 Subject: [PATCH 174/249] Apply recommendations --- docker/test/stateless/Dockerfile | 2 ++ docker/test/stateless/run.sh | 4 ++-- docker/test/stateless/setup_export_logs.sh | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) create mode 120000 docker/test/stateless/setup_export_logs.sh diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 86ca7924f74..48d26c84ed8 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -87,4 +87,6 @@ RUN npm install -g azurite \ COPY run.sh / COPY setup_minio.sh / COPY setup_hdfs_minicluster.sh / +COPY setup_export_logs.sh / + CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 6c930fa583b..015ba7d7423 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -117,13 +117,13 @@ do done # Initialize export of system logs to ClickHouse Cloud -if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" -a -f /repository/utils/export-logs-in-ci/setup.sh ] +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] then export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" # TODO: Check if the password will appear in the logs. export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" - /repository/utils/export-logs-in-ci/setup.sh + ./setup_export_logs.sh # Unset variables after use export CONNECTION_PARAMETERS='' diff --git a/docker/test/stateless/setup_export_logs.sh b/docker/test/stateless/setup_export_logs.sh new file mode 120000 index 00000000000..b6e2f981cd4 --- /dev/null +++ b/docker/test/stateless/setup_export_logs.sh @@ -0,0 +1 @@ +../../../utils/export-logs-in-ci/setup.sh \ No newline at end of file From e4fbca2e2e7e21229e2a68254570c7effeaddbf9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:47:44 +0200 Subject: [PATCH 175/249] Apply review recommendations --- tests/ci/functional_test_check.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index fdea5be9a6c..c84884bcfaf 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -130,7 +130,6 @@ def get_run_command( return ( f"docker run --volume={builds_path}:/package_folder " f"--volume={repo_path}/tests:/usr/share/clickhouse-test " - f"--volume={repo_path}:/repository " f"{volume_with_broken_test} " f"--volume={result_path}:/test_output --volume={server_log_path}:/var/log/clickhouse-server " f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" From 931d75ec571882c86374b5a2cdfa7c4b1fe07012 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 05:05:17 +0200 Subject: [PATCH 176/249] Maybe better --- src/Core/Field.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 89faaed8a72..de5f7ba9c92 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB From 7a6d438b88c03004c105c614749fc7cbab69da42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 05:11:37 +0200 Subject: [PATCH 177/249] Remove magic_enum --- src/Core/Field.cpp | 64 ++++++++++++++++++++++++---------------------- src/Core/Field.h | 2 +- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index de5f7ba9c92..1fcf663a744 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -9,9 +9,10 @@ #include #include #include -#include +using namespace std::literals; + namespace DB { @@ -21,12 +22,6 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; } -/// Keep in mind, that "magic_enum" is very expensive for compiler. -std::string_view Field::getTypeName() const -{ - return magic_enum::enum_name(which); -} - inline Field getBinaryValue(UInt8 type, ReadBuffer & buf) { switch (type) @@ -590,34 +585,41 @@ String toString(const Field & x) x); } -String fieldTypeToString(Field::Types::Which type) +std::string_view fieldTypeToString(Field::Types::Which type) { switch (type) { - case Field::Types::Which::Null: return "Null"; - case Field::Types::Which::Array: return "Array"; - case Field::Types::Which::Tuple: return "Tuple"; - case Field::Types::Which::Map: return "Map"; - case Field::Types::Which::Object: return "Object"; - case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"; - case Field::Types::Which::Bool: return "Bool"; - case Field::Types::Which::String: return "String"; - case Field::Types::Which::Decimal32: return "Decimal32"; - case Field::Types::Which::Decimal64: return "Decimal64"; - case Field::Types::Which::Decimal128: return "Decimal128"; - case Field::Types::Which::Decimal256: return "Decimal256"; - case Field::Types::Which::Float64: return "Float64"; - case Field::Types::Which::Int64: return "Int64"; - case Field::Types::Which::Int128: return "Int128"; - case Field::Types::Which::Int256: return "Int256"; - case Field::Types::Which::UInt64: return "UInt64"; - case Field::Types::Which::UInt128: return "UInt128"; - case Field::Types::Which::UInt256: return "UInt256"; - case Field::Types::Which::UUID: return "UUID"; - case Field::Types::Which::IPv4: return "IPv4"; - case Field::Types::Which::IPv6: return "IPv6"; - case Field::Types::Which::CustomType: return "CustomType"; + case Field::Types::Which::Null: return "Null"sv; + case Field::Types::Which::Array: return "Array"sv; + case Field::Types::Which::Tuple: return "Tuple"sv; + case Field::Types::Which::Map: return "Map"sv; + case Field::Types::Which::Object: return "Object"sv; + case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"sv; + case Field::Types::Which::Bool: return "Bool"sv; + case Field::Types::Which::String: return "String"sv; + case Field::Types::Which::Decimal32: return "Decimal32"sv; + case Field::Types::Which::Decimal64: return "Decimal64"sv; + case Field::Types::Which::Decimal128: return "Decimal128"sv; + case Field::Types::Which::Decimal256: return "Decimal256"sv; + case Field::Types::Which::Float64: return "Float64"sv; + case Field::Types::Which::Int64: return "Int64"sv; + case Field::Types::Which::Int128: return "Int128"sv; + case Field::Types::Which::Int256: return "Int256"sv; + case Field::Types::Which::UInt64: return "UInt64"sv; + case Field::Types::Which::UInt128: return "UInt128"sv; + case Field::Types::Which::UInt256: return "UInt256"sv; + case Field::Types::Which::UUID: return "UUID"sv; + case Field::Types::Which::IPv4: return "IPv4"sv; + case Field::Types::Which::IPv6: return "IPv6"sv; + case Field::Types::Which::CustomType: return "CustomType"sv; } } +/// Keep in mind, that "magic_enum" is very expensive for compiler, that's why we don't use it. +std::string_view Field::getTypeName() const +{ + return fieldTypeToString(which); +} + + } diff --git a/src/Core/Field.h b/src/Core/Field.h index 6666e66e8d5..12542ca0bf1 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1004,7 +1004,7 @@ void writeFieldText(const Field & x, WriteBuffer & buf); String toString(const Field & x); -String fieldTypeToString(Field::Types::Which type); +std::string_view fieldTypeToString(Field::Types::Which type); } From b8be7eef044052a24d0c00873d150d8de3a5ea4f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 03:45:59 +0000 Subject: [PATCH 178/249] Added new tests for session_log and bugfix --- src/Interpreters/Session.cpp | 44 ++- src/Interpreters/Session.h | 2 + src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- tests/integration/test_session_log/.gitignore | 1 + .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 + .../test_session_log/configs/ports.xml | 9 + .../test_session_log/configs/session_log.xml | 9 + .../test_session_log/configs/users.xml | 23 ++ .../protos/clickhouse_grpc.proto | 1 + tests/integration/test_session_log/test.py | 289 ++++++++++++++++++ .../02833_concurrrent_sessions.reference | 34 +++ .../0_stateless/02833_concurrrent_sessions.sh | 138 +++++++++ .../02834_remote_session_log.reference | 13 + .../0_stateless/02834_remote_session_log.sh | 56 ++++ .../02835_drop_user_during_session.reference | 8 + .../02835_drop_user_during_session.sh | 114 +++++++ 19 files changed, 753 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_session_log/.gitignore create mode 100644 tests/integration/test_session_log/__init__.py create mode 100644 tests/integration/test_session_log/configs/log.xml create mode 100644 tests/integration/test_session_log/configs/ports.xml create mode 100644 tests/integration/test_session_log/configs/session_log.xml create mode 100644 tests/integration/test_session_log/configs/users.xml create mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_session_log/test.py create mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference create mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh create mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference create mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh create mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference create mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index de2a779b740..0a6435cff75 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,6 +520,8 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -582,6 +584,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -655,24 +659,38 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - if (!notified_session_log_about_login) - { - if (auto session_log = getSessionLog()) - { - session_log->addLoginSuccess( - auth_id, - named_session ? std::optional(named_session->key.second) : std::nullopt, - *query_context, - user); - - notified_session_log_about_login = true; - } - } + /// Interserver does not create session context + recordLoginSucess(query_context); return query_context; } +void Session::recordLoginSucess(ContextPtr login_context) const +{ + if (notified_session_log_about_login) + return; + + if (!login_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); + + if (auto session_log = getSessionLog()) + { + const auto & settings = login_context->getSettingsRef(); + const auto access = login_context->getAccess(); + + session_log->addLoginSuccess(auth_id, + named_session ? named_session->key.second : "", + settings, + access, + getClientInfo(), + user); + + notified_session_log_about_login = true; + } +} + + void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 51c0e3c71fa..81ef987b428 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,6 +97,8 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; + void recordLoginSucess(ContextPtr login_context) const; + mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0a8a7fc18c5..b847eaf9824 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,12 +199,13 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user) { - const auto access = login_context.getAccess(); - const auto & settings = login_context.getSettingsRef(); - const auto & client_info = login_context.getClientInfo(); - DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -215,8 +216,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - if (session_id) - log_entry.session_id = *session_id; + log_entry.session_id = session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 1282ac09c4d..8757bc12270 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,6 +20,7 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; +using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -72,7 +73,13 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); + void addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user); + void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ff5690a3b07..10d5e7a0242 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,8 +561,7 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto client_info = session->getClientInfo(); - auto context = session->makeQueryContext(std::move(client_info)); + auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_session_log/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml new file mode 100644 index 00000000000..7a079b81e69 --- /dev/null +++ b/tests/integration/test_session_log/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml new file mode 100644 index 00000000000..fbaefc16b3a --- /dev/null +++ b/tests/integration/test_session_log/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml new file mode 100644 index 00000000000..a0e4e3e2216 --- /dev/null +++ b/tests/integration/test_session_log/configs/session_log.xml @@ -0,0 +1,9 @@ + + + system + session_log
+ + toYYYYMM(event_date) + 7500 +
+
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml new file mode 100644 index 00000000000..0416dfadc8a --- /dev/null +++ b/tests/integration/test_session_log/configs/users.xml @@ -0,0 +1,23 @@ + + + + 0 + + + + + + + pass + + + pass + + + pass + + + pass + + + \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_session_log/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py new file mode 100644 index 00000000000..dbb39993ce3 --- /dev/null +++ b/tests/integration/test_session_log/test.py @@ -0,0 +1,289 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import random +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 +SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/session_log.xml", + ], + user_configs=["configs/users.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") + }, +) + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +session_id_counter = 0 + + +def next_session_id(): + global session_id_counter + session_id = session_id_counter + session_id_counter += 1 + return str(session_id) + + +def grpc_query(query, user_, pass_, raise_exception): + try: + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query, + session_id=next_session_id(), + user_name=user_, + password=pass_, + ) + channel = grpc_create_insecure_channel() + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + + return result.output.decode(DEFAULT_ENCODING) + except Exception: + assert raise_exception + + +def postgres_query(query, user_, pass_, raise_exception): + try: + client = py_psql.connect( + host=instance.ip_address, + port=POSTGRES_SERVER_PORT, + user=user_, + password=pass_, + database="default", + ) + cursor = client.cursor() + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +def mysql_query(query, user_, pass_, raise_exception): + try: + client = pymysql.connections.Connection( + host=instance.ip_address, + user=user_, + password=pass_, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + if raise_exception: + with pytest.raises(Exception): + cursor.execute(query) + else: + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_grpc_session(started_cluster): + grpc_query("SELECT 1", "grpc_user", "pass", False) + grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) + grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "grpc_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" + ) + assert logout_records == "grpc_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "grpc_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_mysql_session(started_cluster): + mysql_query("SELECT 1", "mysql_user", "pass", False) + mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) + mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "mysql_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" + ) + assert logout_records == "mysql_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "mysql_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_postgres_session(started_cluster): + postgres_query("SELECT 1", "postgres_user", "pass", False) + postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) + postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "postgres_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" + ) + assert logout_records == "postgres_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "postgres_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_parallel_sessions(started_cluster): + thread_list = [] + for _ in range(10): + # Sleep time does not significantly matter here, + # test should pass even without sleeping. + for function in [postgres_query, grpc_query, mysql_query]: + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "pass", + False, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "wrong_pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "wrong_parallel_user", + "pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + + for thread in thread_list: + thread.join() + + instance.query("SYSTEM FLUSH LOGS") + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + ) + assert port_0_sessions == "90\n" + + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + ) + assert port_0_sessions == "0\n" + + address_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + ) + assert address_0_sessions == "0\n" + + grpc_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + ) + assert grpc_sessions == "30\n" + + mysql_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + ) + assert mysql_sessions == "30\n" + + postgres_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + ) + assert postgres_sessions == "30\n" + + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "30\n" + + logout_failure_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + ) + assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference new file mode 100644 index 00000000000..bfe507e8eac --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -0,0 +1,34 @@ +sessions: +150 +port_0_sessions: +0 +address_0_sessions: +0 +tcp_sessions +60 +http_sessions +30 +http_with_session_id_sessions +30 +my_sql_sessions +30 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh new file mode 100755 index 00000000000..26b48462a76 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -0,0 +1,138 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +# Each user uses a separate thread. +readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") +readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) + +readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" + +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; +done + +# All _session functions execute in separate threads. +# These functions try to create a session with successful login and logout. +# Sleep a small, random amount of time to make concurrency more intense. +# and try to login with an invalid password. +function tcp_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' + done +} + +function http_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" + done +} + +function http_with_session_id_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" + done +} + +function mysql_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; +export -f mysql_session; + +for user in "${TCP_USERS[@]}"; do + timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_USERS[@]}"; do + timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do + timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & +done + +for user in "${MYSQL_USERS[@]}"; do + timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" + +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" + +echo "tcp_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" +echo "http_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "http_with_session_id_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "my_sql_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" + echo "Corresponding LoginSuccess/Logout" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + echo "LoginFailure" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference new file mode 100644 index 00000000000..e2680982ab0 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.reference @@ -0,0 +1,13 @@ +0 +0 +0 +0 +client_port 0 connections: +0 +client_address '::' connections: +0 +login failures: +0 +TCP Login and logout count is equal +HTTP Login and logout count is equal +MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh new file mode 100755 index 00000000000..3bedfb6c9ee --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "client_port 0 connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" + +echo "client_address '::' connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" + +echo "login failures:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" + +# remote(...) function sometimes reuses old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. + +for interface in 'TCP' 'HTTP' 'MySQL' +do + LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + + if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then + echo "${interface} Login and logout count is equal" + else + TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` + echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" + fi +done + +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference new file mode 100644 index 00000000000..7252faab8c6 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.reference @@ -0,0 +1,8 @@ +port_0_sessions: +0 +address_0_sessions: +0 +Corresponding LoginSuccess/Logout +9 +LoginFailure +0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh new file mode 100755 index 00000000000..347ebd22f96 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash +# Tags: no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TEST_USER="02835_USER_${PID}" +readonly TEST_ROLE="02835_ROLE_${PID}" +readonly TEST_PROFILE="02835_PROFILE_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +function tcp_session() +{ + local user=$1 + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" +} + +function http_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +function http_with_session_id_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. +function wait_for_queries_start() +{ + local user=$1 + local queries_count=$2 + # 10 seconds waiting + counter=0 retries=100 + while [[ $counter -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") + if [[ $result == "${queries_count}" ]]; then + break; + fi + sleep 0.1 + ((++counter)) + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +# DROP USE CASE +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP ROLE CASE +${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP PROFILE CASE +${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" +echo "Corresponding LoginSuccess/Logout" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "LoginFailure" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 7b0036dee3ef050c060ee2aee0ab9e6faf927e41 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 04:23:40 +0000 Subject: [PATCH 179/249] moved notified_session_log_about_login=true line --- src/Interpreters/Session.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 0a6435cff75..f8bd70afdb6 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -685,9 +685,9 @@ void Session::recordLoginSucess(ContextPtr login_context) const access, getClientInfo(), user); - - notified_session_log_about_login = true; } + + notified_session_log_about_login = true; } From d499f8030a34d5076c7a9fc97d94a5f04b2898d6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 05:20:00 +0000 Subject: [PATCH 180/249] reverted change in test --- tests/integration/test_profile_max_sessions_for_user/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 2930262f63e..65587933fed 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,10 +27,7 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ - {proto_dir}/clickhouse_grpc.proto".format( - proto_dir=proto_dir, gen_dir=gen_dir - ), + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", shell=True, ) From 7321f5e543387aa65e20fe12421ee692eb05aa64 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 06:32:28 +0000 Subject: [PATCH 181/249] Better --- src/IO/WriteHelpers.h | 15 ++++++++--- src/Parsers/ASTLiteral.cpp | 26 +++++++++++++++++-- src/Parsers/IAST.h | 7 ++++- src/Parsers/LiteralEscapingStyle.h | 14 ++++++++++ src/Storages/StorageMySQL.cpp | 1 + src/Storages/StoragePostgreSQL.cpp | 6 +---- src/Storages/StorageSQLite.cpp | 1 + src/Storages/StorageXDBC.cpp | 1 + .../transformQueryForExternalDatabase.cpp | 7 ++++- .../transformQueryForExternalDatabase.h | 1 + .../test_storage_postgresql/test.py | 20 +++++++------- 11 files changed, 78 insertions(+), 21 deletions(-) create mode 100644 src/Parsers/LiteralEscapingStyle.h diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..d092c7b8ea5 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -304,9 +304,10 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & /** Will escape quote_character and a list of special characters('\b', '\f', '\n', '\r', '\t', '\0', '\\'). * - when escape_quote_with_quote is true, use backslash to escape list of special characters, * and use quote_character to escape quote_character. such as: 'hello''world' - * - otherwise use backslash to escape list of special characters and quote_character + * otherwise use backslash to escape list of special characters and quote_character + * - when escape_backslash_with_backslash is true, backslash is escaped with another backslash */ -template +template void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & buf) { const char * pos = begin; @@ -360,7 +361,8 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b writeChar('0', buf); break; case '\\': - writeChar('\\', buf); + if constexpr (escape_backslash_with_backslash) + writeChar('\\', buf); writeChar('\\', buf); break; default: @@ -466,6 +468,13 @@ inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) writeAnyQuotedString<'\''>(ref.data(), ref.data() + ref.size(), buf); } +inline void writeQuotedStringPostgreSQL(std::string_view ref, WriteBuffer & buf) +{ + writeChar('\'', buf); + writeAnyEscapedString<'\'', true, false>(ref.data(), ref.data() + ref.size(), buf); + writeChar('\'', buf); +} + inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s, buf); diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 5c76f6f33bf..4a9a3d8df5b 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -93,7 +93,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const { - /// 100 - just arbitrary value. + /// 100 - just arbitrary value. constexpr auto min_elements_for_hashing = 100; /// Special case for very large arrays. Instead of listing all elements, will use hash of them. @@ -118,9 +118,31 @@ void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const } } +/// Use different rules for escaping backslashes and quotes +class FieldVisitorToStringPostgreSQL : public StaticVisitor +{ +public: + template + String operator() (const T & x) const { return visitor(x); } + +private: + FieldVisitorToString visitor; +}; + +template<> +String FieldVisitorToStringPostgreSQL::operator() (const String & x) const +{ + WriteBufferFromOwnString wb; + writeQuotedStringPostgreSQL(x, wb); + return wb.str(); +} + void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << applyVisitor(FieldVisitorToString(), value); + if (settings.literal_escaping_style == LiteralEscapingStyle::Regular) + settings.ostr << applyVisitor(FieldVisitorToString(), value); + else + settings.ostr << applyVisitor(FieldVisitorToStringPostgreSQL(), value); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d217876459f..58bc9702142 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -197,6 +198,7 @@ public: IdentifierQuotingStyle identifier_quoting_style; bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. + LiteralEscapingStyle literal_escaping_style; explicit FormatSettings( WriteBuffer & ostr_, @@ -204,7 +206,8 @@ public: bool hilite_ = false, bool always_quote_identifiers_ = false, IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, - bool show_secrets_ = true) + bool show_secrets_ = true, + LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -212,6 +215,7 @@ public: , identifier_quoting_style(identifier_quoting_style_) , show_secrets(show_secrets_) , nl_or_ws(one_line ? ' ' : '\n') + , literal_escaping_style(literal_escaping_style_) { } @@ -223,6 +227,7 @@ public: , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) , nl_or_ws(other.nl_or_ws) + , literal_escaping_style(other.literal_escaping_style) { } diff --git a/src/Parsers/LiteralEscapingStyle.h b/src/Parsers/LiteralEscapingStyle.h new file mode 100644 index 00000000000..10d4d84a85d --- /dev/null +++ b/src/Parsers/LiteralEscapingStyle.h @@ -0,0 +1,14 @@ +#pragma once + + +namespace DB +{ + +/// Method to escape single quotes. +enum class LiteralEscapingStyle +{ + Regular, /// Escape backslashes with backslash (\\) and quotes with backslash (\') + PostgreSQL, /// Do not escape backslashes (\), escape quotes with quote ('') +}; + +} diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b0a220eb1d2..76a439eabaf 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -104,6 +104,7 @@ Pipe StorageMySQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, context_); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 11558b39ad3..f233d4ff213 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -123,11 +123,7 @@ Pipe StoragePostgreSQL::read( query_info_, column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), - IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); - - /// Single quotes in PostgreSQL are escaped through repetition - boost::replace_all(query, "\\'", "''"); - + IdentifierQuotingStyle::DoubleQuotes, LiteralEscapingStyle::PostgreSQL, remote_table_schema, remote_table_name, context_); LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d5ae6f2383f..d5db5763da9 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -91,6 +91,7 @@ Pipe StorageSQLite::read( column_names, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "", remote_table_name, context_); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index b532d1c91f0..1715cde9d1e 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -79,6 +79,7 @@ std::function StorageXDBC::getReadPOSTDataCallback( column_names, columns_description.getOrdinary(), bridge_helper->getIdentifierQuotingStyle(), + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, local_context); diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 375510e62bf..84a696a1e9c 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -258,6 +258,7 @@ String transformQueryForExternalDatabaseImpl( Names used_columns, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -337,7 +338,8 @@ String transformQueryForExternalDatabaseImpl( IAST::FormatSettings settings( out, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, - /*identifier_quoting_style*/ identifier_quoting_style); + /*identifier_quoting_style*/ identifier_quoting_style, /*show_secrets_*/ true, + /*literal_escaping_style*/ literal_escaping_style); select->format(settings); @@ -351,6 +353,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -375,6 +378,7 @@ String transformQueryForExternalDatabase( column_names, available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); @@ -386,6 +390,7 @@ String transformQueryForExternalDatabase( query_info.syntax_analyzer_result->requiredSourceColumns(), available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); diff --git a/src/Storages/transformQueryForExternalDatabase.h b/src/Storages/transformQueryForExternalDatabase.h index 0f2b0a5822f..fb6af21907e 100644 --- a/src/Storages/transformQueryForExternalDatabase.h +++ b/src/Storages/transformQueryForExternalDatabase.h @@ -31,6 +31,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context); diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3a36d050f17..d4f8fab3a82 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,18 +726,20 @@ def test_auto_close_connection(started_cluster): assert count == 2 -def test_single_quotes(started_cluster): +def test_literal_escaping(started_cluster): cursor = started_cluster.postgres_conn.cursor() - cursor.execute(f"DROP TABLE IF EXISTS single_quote_fails") - cursor.execute(f"CREATE TABLE single_quote_fails(text varchar(255))") + cursor.execute(f"DROP TABLE IF EXISTS escaping") + cursor.execute(f"CREATE TABLE escaping(text varchar(255))") node1.query( - "CREATE TABLE default.single_quote_fails (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'single_quote_fails', 'postgres', 'mysecretpassword')" + "CREATE TABLE default.escaping (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'escaping', 'postgres', 'mysecretpassword')" ) - node1.query("SELECT * FROM single_quote_fails WHERE text = ''''") - node1.query("SELECT * FROM single_quote_fails WHERE text = '\\''") - node1.query("SELECT * FROM single_quote_fails WHERE text like '%a''a%'") - node1.query("SELECT * FROM single_quote_fails WHERE text like '%a\\'a%'") - cursor.execute(f"DROP TABLE single_quote_fails") + node1.query("SELECT * FROM escaping WHERE text = ''''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text like '%a''a%'") # %a'a% -> %a''a% + node1.query("SELECT * FROM escaping WHERE text like '%a\\'a%'") # %a'a% -> %a''a% + cursor.execute(f"DROP TABLE escaping") if __name__ == "__main__": From 5cdeacf4cf61c0ac228eb63b7178392a6436d41c Mon Sep 17 00:00:00 2001 From: Ruslan Mardugalliamov Date: Sun, 6 Aug 2023 15:33:36 -0400 Subject: [PATCH 182/249] Add hints for HTTP handlers Add hints to HTTP handlers to help users avoid misspellings. For example, if a user mistakenly writes `/dashboad` instead of `/dashboard`, they will now get a hint that /dashboard is the correct handler. This change will improve the user experience by making it easier for users to find the correct handlers. #47662 --- programs/keeper/CMakeLists.txt | 1 + src/Server/HTTPHandlerFactory.cpp | 4 ++++ src/Server/HTTPPathHints.cpp | 16 ++++++++++++++ src/Server/HTTPPathHints.h | 22 +++++++++++++++++++ src/Server/HTTPRequestHandlerFactoryMain.cpp | 2 +- src/Server/HTTPRequestHandlerFactoryMain.h | 4 ++++ src/Server/NotFoundHandler.cpp | 3 ++- src/Server/NotFoundHandler.h | 3 +++ ...ggest_http_page_in_error_message.reference | 4 ++++ ...2842_suggest_http_page_in_error_message.sh | 12 ++++++++++ 10 files changed, 69 insertions(+), 2 deletions(-) create mode 100644 src/Server/HTTPPathHints.cpp create mode 100644 src/Server/HTTPPathHints.h create mode 100644 tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference create mode 100755 tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 43a8d84b513..a43a312ba54 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -57,6 +57,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/IO/ReadBuffer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPPathHints.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 78e374ee9e0..1c911034da1 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -132,21 +132,25 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS auto ping_handler = std::make_shared>(server, ping_response_expression); ping_handler->attachStrictPath("/ping"); ping_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/ping"); factory.addHandler(ping_handler); auto replicas_status_handler = std::make_shared>(server); replicas_status_handler->attachNonStrictPath("/replicas_status"); replicas_status_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/replicas_status"); factory.addHandler(replicas_status_handler); auto play_handler = std::make_shared>(server); play_handler->attachNonStrictPath("/play"); play_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/play"); factory.addHandler(play_handler); auto dashboard_handler = std::make_shared>(server); dashboard_handler->attachNonStrictPath("/dashboard"); dashboard_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/dashboard"); factory.addHandler(dashboard_handler); auto js_handler = std::make_shared>(server); diff --git a/src/Server/HTTPPathHints.cpp b/src/Server/HTTPPathHints.cpp new file mode 100644 index 00000000000..51ef3eabffe --- /dev/null +++ b/src/Server/HTTPPathHints.cpp @@ -0,0 +1,16 @@ +#include + +namespace DB +{ + +void HTTPPathHints::add(const String & http_path) +{ + http_paths.push_back(http_path); +} + +std::vector HTTPPathHints::getAllRegisteredNames() const +{ + return http_paths; +} + +} diff --git a/src/Server/HTTPPathHints.h b/src/Server/HTTPPathHints.h new file mode 100644 index 00000000000..708816ebf07 --- /dev/null +++ b/src/Server/HTTPPathHints.h @@ -0,0 +1,22 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class HTTPPathHints : public IHints<1, HTTPPathHints> +{ +public: + std::vector getAllRegisteredNames() const override; + void add(const String & http_path); + +private: + std::vector http_paths; +}; + +using HTTPPathHintsPtr = std::shared_ptr; + +} diff --git a/src/Server/HTTPRequestHandlerFactoryMain.cpp b/src/Server/HTTPRequestHandlerFactoryMain.cpp index 61a2909d30f..5481bcd5083 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.cpp +++ b/src/Server/HTTPRequestHandlerFactoryMain.cpp @@ -29,7 +29,7 @@ std::unique_ptr HTTPRequestHandlerFactoryMain::createRequest || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return std::unique_ptr(new NotFoundHandler); + return std::unique_ptr(new NotFoundHandler(hints.getHints(request.getURI()))); } return nullptr; diff --git a/src/Server/HTTPRequestHandlerFactoryMain.h b/src/Server/HTTPRequestHandlerFactoryMain.h index b0e57bd6b3b..07b278d831c 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.h +++ b/src/Server/HTTPRequestHandlerFactoryMain.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -15,11 +16,14 @@ public: void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); } + void addPathToHints(const std::string & http_path) { hints.add(http_path); } + std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: Poco::Logger * log; std::string name; + HTTPPathHints hints; std::vector child_factories; }; diff --git a/src/Server/NotFoundHandler.cpp b/src/Server/NotFoundHandler.cpp index 3181708b9b7..5b1db508551 100644 --- a/src/Server/NotFoundHandler.cpp +++ b/src/Server/NotFoundHandler.cpp @@ -10,7 +10,8 @@ void NotFoundHandler::handleRequest(HTTPServerRequest & request, HTTPServerRespo try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - *response.send() << "There is no handle " << request.getURI() << "\n\n" + *response.send() << "There is no handle " << request.getURI() + << (!hints.empty() ? fmt::format(". Maybe you meant {}.", hints.front()) : "") << "\n\n" << "Use / or /ping for health checks.\n" << "Or /replicas_status for more sophisticated health checks.\n\n" << "Send queries from your program with POST method or GET /?query=...\n\n" diff --git a/src/Server/NotFoundHandler.h b/src/Server/NotFoundHandler.h index 749ac388c4d..1cbfcd57f8f 100644 --- a/src/Server/NotFoundHandler.h +++ b/src/Server/NotFoundHandler.h @@ -9,7 +9,10 @@ namespace DB class NotFoundHandler : public HTTPRequestHandler { public: + NotFoundHandler(std::vector hints_) : hints(std::move(hints_)) {} void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; +private: + std::vector hints; }; } diff --git a/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference new file mode 100644 index 00000000000..0025187be30 --- /dev/null +++ b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference @@ -0,0 +1,4 @@ +There is no handle /sashboards. Maybe you meant /dashboard +There is no handle /sashboard. Maybe you meant /dashboard +There is no handle /sashboarb. Maybe you meant /dashboard +There is no handle /sashboaxb. Maybe you meant /dashboard diff --git a/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh new file mode 100755 index 00000000000..cf69c742777 --- /dev/null +++ b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export CLICKHOUSE_URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboards" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboard" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboarb" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboaxb" | grep -o ".* Maybe you meant /dashboard" From 4b30900fed3dcafa015a7d6dc4d6d91ea362a966 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 14:44:16 +0300 Subject: [PATCH 183/249] Revert "Improve CHECK TABLE system query" --- .../sql-reference/statements/check-table.md | 107 +++------- src/Common/FileChecker.cpp | 42 ++-- src/Common/FileChecker.h | 37 +--- src/Interpreters/InterpreterCheckQuery.cpp | 201 ++++-------------- src/Storages/CheckResults.h | 2 + src/Storages/IStorage.cpp | 11 - src/Storages/IStorage.h | 41 +--- src/Storages/StorageLog.cpp | 9 +- src/Storages/StorageLog.h | 16 +- src/Storages/StorageMergeTree.cpp | 26 +-- src/Storages/StorageMergeTree.h | 29 +-- src/Storages/StorageProxy.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 37 ++-- src/Storages/StorageReplicatedMergeTree.h | 31 +-- src/Storages/StorageStripeLog.cpp | 10 +- src/Storages/StorageStripeLog.h | 17 +- .../0_stateless/00063_check_query.reference | 1 - .../queries/0_stateless/00063_check_query.sql | 3 - .../queries/0_stateless/00961_check_table.sql | 12 +- ...1042_check_query_and_last_granule_size.sql | 12 +- ...02235_check_table_sparse_serialization.sql | 3 +- .../02841_check_table_progress.reference | 2 - .../0_stateless/02841_check_table_progress.sh | 29 --- 23 files changed, 138 insertions(+), 544 deletions(-) delete mode 100644 tests/queries/0_stateless/02841_check_table_progress.reference delete mode 100755 tests/queries/0_stateless/02841_check_table_progress.sh diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index db8c32249ef..0209d59b018 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -5,38 +5,19 @@ sidebar_label: CHECK TABLE title: "CHECK TABLE Statement" --- -The `CHECK TABLE` query in ClickHouse is used to perform a validation check on a specific table or its partitions. It ensures the integrity of the data by verifying the checksums and other internal data structures. +Checks if the data in the table is corrupted. -Particularly it compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. - -:::note -The `CHECK TABLE`` query may read all the data in the table and hold some resources, making it resource-intensive. -Consider the potential impact on performance and resource utilization before executing this query. -::: - -## Syntax - -The basic syntax of the query is as follows: - -```sql -CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings] ] +``` sql +CHECK TABLE [db.]name [PARTITION partition_expr] ``` -- `table_name`: Specifies the name of the table that you want to check. -- `partition_expression`: (Optional) If you want to check a specific partition of the table, you can use this expression to specify the partition. -- `FORMAT format`: (Optional) Allows you to specify the output format of the result. -- `SETTINGS`: (Optional) Allows additional settings. - - **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`). - - Other settings (e.g. `max_threads` can be applied as well). +The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. +The query response contains the `result` column with a single row. The row has a value of +[Boolean](../../sql-reference/data-types/boolean.md) type: -The query response depends on the value of contains `check_query_single_value_result` setting. -In case of `check_query_single_value_result = 1` only `result` column with a single row is returned. Value inside this row is `1` if the integrity check is passed and `0` if data is corrupted. - -With `check_query_single_value_result = 0` the query returns the following columns: - - `part_path`: Indicates the path to the data part or file name. - - `is_passed`: Returns 1 if the check for this part is successful, 0 otherwise. - - `message`: Any additional messages related to the check, such as errors or success messages. +- 0 - The data in the table is corrupted. +- 1 - The data maintains integrity. The `CHECK TABLE` query supports the following table engines: @@ -45,15 +26,30 @@ The `CHECK TABLE` query supports the following table engines: - [StripeLog](../../engines/table-engines/log-family/stripelog.md) - [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) -Performed over the tables with another table engines causes an `NOT_IMPLEMETED` exception. +Performed over the tables with another table engines causes an exception. Engines from the `*Log` family do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. -## Examples +## Checking the MergeTree Family Tables -By default `CHECK TABLE` query shows the general table check status: +For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server. ```sql +SET check_query_single_value_result = 0; +CHECK TABLE test_table; +``` + +```text +┌─part_path─┬─is_passed─┬─message─┐ +│ all_1_4_1 │ 1 │ │ +│ all_1_4_2 │ 1 │ │ +└───────────┴───────────┴─────────┘ +``` + +If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status. + +```sql +SET check_query_single_value_result = 1; CHECK TABLE test_table; ``` @@ -63,60 +59,11 @@ CHECK TABLE test_table; └────────┘ ``` -If you want to see the check status for every individual data part you may use `check_query_single_value_result` setting. - -Also, to check a specific partition of the table, you can use the `PARTITION` keyword. - -```sql -CHECK TABLE t0 PARTITION ID '201003' -FORMAT PrettyCompactMonoBlock -SETTINGS check_query_single_value_result = 0 -``` - -Output: - -```text -┌─part_path────┬─is_passed─┬─message─┐ -│ 201003_7_7_0 │ 1 │ │ -│ 201003_3_3_0 │ 1 │ │ -└──────────────┴───────────┴─────────┘ -``` - -### Receiving a 'Corrupted' Result - -:::warning -Disclaimer: The procedure described here, including the manual manipulating or removing files directly from the data directory, is for experimental or development environments only. Do **not** attempt this on a production server, as it may lead to data loss or other unintended consequences. -::: - -Remove the existing checksum file: - -```bash -rm /var/lib/clickhouse-server/data/default/t0/201003_3_3_0/checksums.txt -``` - -```sql -CHECK TABLE t0 PARTITION ID '201003' -FORMAT PrettyCompactMonoBlock -SETTINGS check_query_single_value_result = 0 - - -Output: - -```text -┌─part_path────┬─is_passed─┬─message──────────────────────────────────┐ -│ 201003_7_7_0 │ 1 │ │ -│ 201003_3_3_0 │ 1 │ Checksums recounted and written to disk. │ -└──────────────┴───────────┴──────────────────────────────────────────┘ -``` - -If the checksums.txt file is missing, it can be restored. It will be recalculated and rewritten during the execution of the CHECK TABLE command for the specific partition, and the status will still be reported as 'success.'" - - ## If the Data Is Corrupted If the table is corrupted, you can copy the non-corrupted data to another table. To do this: 1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. -2. Set the `max_threads` value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +2. Set the [max_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. 3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. 4. Restart the `clickhouse-client` to reset the `max_threads` value. diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 122ea83835d..876bc4e641c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -82,35 +82,33 @@ size_t FileChecker::getTotalSize() const } -FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks() +CheckResults FileChecker::check() const { - return std::make_unique(map); -} - -CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const -{ - String name; - size_t expected_size; - bool is_finished = check_data_tasks->next(name, expected_size); - if (is_finished) - { - has_nothing_to_do = true; + if (map.empty()) return {}; - } - String path = parentPath(files_info_path) + name; - bool exists = fileReallyExists(path); - auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + CheckResults results; - if (real_size != expected_size) + for (const auto & name_size : map) { - String failure_message = exists - ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(expected_size)) - : ("File " + path + " doesn't exist"); - return CheckResult(name, false, failure_message); + const String & name = name_size.first; + String path = parentPath(files_info_path) + name; + bool exists = fileReallyExists(path); + auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + + if (real_size != name_size.second) + { + String failure_message = exists + ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second)) + : ("File " + path + " doesn't exist"); + results.emplace_back(name, false, failure_message); + break; + } + + results.emplace_back(name, true, ""); } - return CheckResult(name, true, ""); + return results; } void FileChecker::repair() diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 8ffc310b84d..bb0383e4b56 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace Poco { class Logger; } @@ -29,11 +28,7 @@ public: bool empty() const { return map.empty(); } /// Check the files whose parameters are specified in sizes.json - /// See comment in IStorage::checkDataNext - struct DataValidationTasks; - using DataValidationTasksPtr = std::unique_ptr; - DataValidationTasksPtr getDataValidationTasks(); - CheckResult checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const; + CheckResults check() const; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. @@ -46,36 +41,6 @@ public: /// Returns total size of all files. size_t getTotalSize() const; - struct DataValidationTasks - { - DataValidationTasks(const std::map & map_) - : map(map_), it(map.begin()) - {} - - bool next(String & out_name, size_t & out_size) - { - std::lock_guard lock(mutex); - if (it == map.end()) - return true; - out_name = it->first; - out_size = it->second; - ++it; - return false; - } - - size_t size() const - { - std::lock_guard lock(mutex); - return std::distance(it, map.end()); - } - - const std::map & map; - - mutable std::mutex mutex; - using Iterator = std::map::const_iterator; - Iterator it; - }; - private: void load(); diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index bd530654dd2..333aed84873 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -8,201 +8,72 @@ #include #include #include -#include #include -#include - -#include -#include - namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { -Block getSingleValueBlock(UInt8 value) +NamesAndTypes getBlockStructure() { - return Block{{ColumnUInt8::create(1, value), std::make_shared(), "result"}}; -} - -Block getHeaderForCheckResult() -{ - auto names_and_types = NamesAndTypes{ + return { {"part_path", std::make_shared()}, {"is_passed", std::make_shared()}, {"message", std::make_shared()}, }; - - return Block({ - {names_and_types[0].type->createColumn(), names_and_types[0].type, names_and_types[0].name}, - {names_and_types[1].type->createColumn(), names_and_types[1].type, names_and_types[1].name}, - {names_and_types[2].type->createColumn(), names_and_types[2].type, names_and_types[2].name}, - }); } -Chunk getChunkFromCheckResult(const CheckResult & check_result) -{ - MutableColumns columns = getHeaderForCheckResult().cloneEmptyColumns(); - columns[0]->insert(check_result.fs_path); - columns[1]->insert(static_cast(check_result.success)); - columns[2]->insert(check_result.failure_message); - return Chunk(std::move(columns), 1); -} - -class TableCheckWorkerProcessor : public ISource -{ - -public: - TableCheckWorkerProcessor(IStorage::DataValidationTasksPtr check_data_tasks_, StoragePtr table_) - : ISource(getHeaderForCheckResult()) - , table(table_) - , check_data_tasks(check_data_tasks_) - { - } - - String getName() const override { return "TableCheckWorkerProcessor"; } - -protected: - - std::optional tryGenerate() override - { - bool has_nothing_to_do = false; - auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do); - if (has_nothing_to_do) - return {}; - - /// We can omit manual `progess` call, ISource will may count it automatically by returned chunk - /// However, we want to report only rows in progress - progress(1, 0); - - if (!check_result.success) - { - LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"), - "Check query for table {} failed, path {}, reason: {}", - table->getStorageID().getNameForLogs(), - check_result.fs_path, - check_result.failure_message); - } - - return getChunkFromCheckResult(check_result); - } - -private: - StoragePtr table; - IStorage::DataValidationTasksPtr check_data_tasks; -}; - -class TableCheckResultEmitter : public IAccumulatingTransform -{ -public: - TableCheckResultEmitter() : IAccumulatingTransform(getHeaderForCheckResult(), getSingleValueBlock(1).cloneEmpty()) {} - - String getName() const override { return "TableCheckResultEmitter"; } - - void consume(Chunk chunk) override - { - if (result_value == 0) - return; - - auto columns = chunk.getColumns(); - if (columns.size() != 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}", columns.size()); - - const auto * col = checkAndGetColumn(columns[1].get()); - for (size_t i = 0; i < col->size(); ++i) - { - if (col->getElement(i) == 0) - { - result_value = 0; - return; - } - } - } - - Chunk generate() override - { - if (is_value_emitted.exchange(true)) - return {}; - auto block = getSingleValueBlock(result_value); - return Chunk(block.getColumns(), block.rows()); - } - -private: - std::atomic result_value{1}; - std::atomic_bool is_value_emitted{false}; -}; - } -InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_) - , query_ptr(query_ptr_) + +InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { } + BlockIO InterpreterCheckQuery::execute() { const auto & check = query_ptr->as(); - const auto & context = getContext(); - auto table_id = context->resolveStorageID(check, Context::ResolveOrdinary); + auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary); - context->checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + getContext()->checkAccess(AccessType::SHOW_TABLES, table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + auto check_results = table->checkData(query_ptr, getContext()); - auto check_data_tasks = table->getCheckTaskList(query_ptr, context); + Block block; + if (getContext()->getSettingsRef().check_query_single_value_result) + { + bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); + auto column = ColumnUInt8::create(); + column->insertValue(static_cast(result)); + block = Block{{std::move(column), std::make_shared(), "result"}}; + } + else + { + auto block_structure = getBlockStructure(); + auto path_column = block_structure[0].type->createColumn(); + auto is_passed_column = block_structure[1].type->createColumn(); + auto message_column = block_structure[2].type->createColumn(); - const auto & settings = context->getSettingsRef(); + for (const auto & check_result : check_results) + { + path_column->insert(check_result.fs_path); + is_passed_column->insert(static_cast(check_result.success)); + message_column->insert(check_result.failure_message); + } + + block = Block({ + {std::move(path_column), block_structure[0].type, block_structure[0].name}, + {std::move(is_passed_column), block_structure[1].type, block_structure[1].name}, + {std::move(message_column), block_structure[2].type, block_structure[2].name}}); + } BlockIO res; - { - auto processors = std::make_shared(); + res.pipeline = QueryPipeline(std::make_shared(std::move(block))); - std::vector worker_ports; - - size_t num_streams = std::max(1, settings.max_threads); - - for (size_t i = 0; i < num_streams; ++i) - { - auto worker_processor = std::make_shared(check_data_tasks, table); - if (i == 0) - worker_processor->addTotalRowsApprox(check_data_tasks->size()); - worker_ports.emplace_back(&worker_processor->getPort()); - processors->emplace_back(worker_processor); - } - - OutputPort * resize_outport; - { - auto resize_processor = std::make_shared(getHeaderForCheckResult(), worker_ports.size(), 1); - - auto & resize_inputs = resize_processor->getInputs(); - auto resize_inport_it = resize_inputs.begin(); - for (size_t i = 0; i < worker_ports.size(); ++i, ++resize_inport_it) - connect(*worker_ports[i], *resize_inport_it); - - resize_outport = &resize_processor->getOutputs().front(); - processors->emplace_back(resize_processor); - } - - if (settings.check_query_single_value_result) - { - auto emitter_processor = std::make_shared(); - auto * input_port = &emitter_processor->getInputPort(); - processors->emplace_back(emitter_processor); - - connect(*resize_outport, *input_port); - } - - res.pipeline = QueryPipeline(Pipe(std::move(processors))); - res.pipeline.setNumThreads(num_streams); - } return res; } diff --git a/src/Storages/CheckResults.h b/src/Storages/CheckResults.h index 2e4652fea29..b342b014fa4 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -22,4 +22,6 @@ struct CheckResult {} }; +using CheckResults = std::vector; + } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 85299f63165..ae7659e074f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -273,17 +273,6 @@ bool IStorage::isStaticStorage() const return false; } -IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); -} - -CheckResult IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */, bool & has_nothing_to_do) -{ - has_nothing_to_do = true; - return {}; -} - void IStorage::adjustCreateQueryForBackup(ASTPtr &) const { } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b1e20c55782..ec92f57aeda 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -595,45 +595,8 @@ public: /// Provides a hint that the storage engine may evaluate the IN-condition by using an index. virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; } - - /** A list of tasks to check a validity of data. - * Each IStorage implementation may interpret this task in its own way. - * E.g. for some storages it to check data it need to check a list of files in filesystem, for others it can be a list of parts. - * Also it may hold resources (e.g. locks) required during check. - */ - struct DataValidationTasksBase - { - /// Number of entries left to check. - /// It decreases after each call to checkDataNext(). - virtual size_t size() const = 0; - virtual ~DataValidationTasksBase() = default; - }; - - using DataValidationTasksPtr = std::shared_ptr; - - virtual DataValidationTasksPtr getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */); - - /** Executes one task from the list. - * If no tasks left, sets has_nothing_to_do to true. - * Note: Function `checkDataNext` is accessing `check_task_list` thread-safely, - * and can be called simultaneously for the same `getCheckTaskList` result - * to process different tasks in parallel. - * Usage: - * - * auto check_task_list = storage.getCheckTaskList(query, context); - * size_t total_tasks = check_task_list->size(); - * while (true) - * { - * size_t tasks_left = check_task_list->size(); - * std::cout << "Checking data: " << (total_tasks - tasks_left) << " / " << total_tasks << " tasks done." << std::endl; - * bool has_nothing_to_do = false; - * auto result = storage.checkDataNext(check_task_list, has_nothing_to_do); - * if (has_nothing_to_do) - * break; - * doSomething(result); - * } - */ - virtual CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do); + /// Checks validity of the data + virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); } /// Checks that table could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0e9f83e886a..87aa71f3e8d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -866,18 +866,15 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); + + return file_checker.check(); } -CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); -} IStorage::ColumnSizeByName StorageLog::getColumnSizes() const { diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 95f95088aa2..f1d05ed39ac 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,8 +59,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; @@ -143,19 +142,6 @@ private: std::atomic total_rows = 0; std::atomic total_bytes = 0; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) - : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) - {} - - size_t size() const override { return file_checker_tasks->size(); } - - FileChecker::DataValidationTasksPtr file_checker_tasks; - /// Lock to prevent table modification while checking - ReadLock lock; - }; - FileChecker file_checker; const size_t max_compress_block_size; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 97fc7a6731f..ad9013d9f13 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2197,8 +2197,9 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } -IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) +CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { + CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -2208,14 +2209,7 @@ IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr else data_parts = getVisibleDataPartsVector(local_context); - return std::make_unique(std::move(data_parts), local_context); -} - -CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - auto * data_validation_tasks = assert_cast(check_task_list.get()); - auto local_context = data_validation_tasks->context; - if (auto part = data_validation_tasks->next()) + for (auto & part : data_parts) { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; @@ -2230,12 +2224,12 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings()); part->checkMetadata(); - return CheckResult(part->name, true, "Checksums recounted and written to disk."); + results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } catch (const Exception & ex) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } else @@ -2244,19 +2238,15 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ { checkDataPart(part, true); part->checkMetadata(); - return CheckResult(part->name, true, ""); + results.emplace_back(part->name, true, ""); } catch (const Exception & ex) { - return CheckResult(part->name, false, ex.message()); + results.emplace_back(part->name, false, ex.message()); } } } - else - { - has_nothing_to_do = true; - return {}; - } + return results; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ec4796e4941..c77e5140d75 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,8 +108,7 @@ public: void onActionLockRemove(StorageActionBlockType action_type) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; @@ -279,32 +278,6 @@ private: friend class MergePlainMergeTreeTask; friend class MutatePlainMergeTreeTask; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(DataPartsVector && parts_, ContextPtr context_) - : parts(std::move(parts_)), it(parts.begin()), context(std::move(context_)) - {} - - DataPartPtr next() - { - std::lock_guard lock(mutex); - if (it == parts.end()) - return nullptr; - return *(it++); - } - - size_t size() const override - { - std::lock_guard lock(mutex); - return std::distance(it, parts.end()); - } - - mutable std::mutex mutex; - DataPartsVector parts; - DataPartsVector::const_iterator it; - - ContextPtr context; - }; protected: std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index ea908bea032..21ed4b91c62 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,9 +149,7 @@ public: return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override { return getNested()->getCheckTaskList(query, context); } - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override { return getNested()->checkDataNext(check_task_list, has_nothing_to_do); } - + CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c08f1ebcc48..7fce373e26b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8481,8 +8481,9 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t part_check_thread.enqueuePart(part_name, delay_to_check_seconds); } -IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) +CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { + CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -8492,30 +8493,24 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(co else data_parts = getVisibleDataPartsVector(local_context); - auto part_check_lock = part_check_thread.pausePartsCheck(); - return std::make_unique(std::move(data_parts), std::move(part_check_lock)); -} - -CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - - if (auto part = assert_cast(check_task_list.get())->next()) { - try + auto part_check_lock = part_check_thread.pausePartsCheck(); + + for (auto & part : data_parts) { - return CheckResult(part_check_thread.checkPartAndFix(part->name)); - } - catch (const Exception & ex) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + try + { + results.push_back(part_check_thread.checkPartAndFix(part->name)); + } + catch (const Exception & ex) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + } } } - else - { - has_nothing_to_do = true; - return {}; - } + + return results; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2bc18aa3b0a..78ef39f032f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,8 +230,7 @@ public: /// Add a part to the queue of parts whose data you want to check in the background thread. void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0); - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; @@ -991,34 +990,6 @@ private: bool waitZeroCopyLockToDisappear(const ZeroCopyLock & lock, size_t milliseconds_to_wait) override; void startupImpl(bool from_attach_thread); - - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - explicit DataValidationTasks(DataPartsVector && parts_, std::unique_lock && parts_check_lock_) - : parts_check_lock(std::move(parts_check_lock_)), parts(std::move(parts_)), it(parts.begin()) - {} - - DataPartPtr next() - { - std::lock_guard lock(mutex); - if (it == parts.end()) - return nullptr; - return *(it++); - } - - size_t size() const override - { - std::lock_guard lock(mutex); - return std::distance(it, parts.end()); - } - - std::unique_lock parts_check_lock; - - mutable std::mutex mutex; - DataPartsVector parts; - DataPartsVector::const_iterator it; - }; - }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index a3cbff96199..0bfef5ed5e5 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,18 +403,16 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) + +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); + + return file_checker.check(); } -CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); -} void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 5d4e2fcbd3a..f889a1de71b 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,8 +53,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } @@ -94,20 +93,6 @@ private: const DiskPtr disk; String table_path; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) - : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) - {} - - size_t size() const override { return file_checker_tasks->size(); } - - FileChecker::DataValidationTasksPtr file_checker_tasks; - - /// Lock to prevent table modification while checking - ReadLock lock; - }; - String data_file_path; String index_file_path; FileChecker file_checker; diff --git a/tests/queries/0_stateless/00063_check_query.reference b/tests/queries/0_stateless/00063_check_query.reference index e8183f05f5d..6ed281c757a 100644 --- a/tests/queries/0_stateless/00063_check_query.reference +++ b/tests/queries/0_stateless/00063_check_query.reference @@ -1,3 +1,2 @@ 1 1 -1 diff --git a/tests/queries/0_stateless/00063_check_query.sql b/tests/queries/0_stateless/00063_check_query.sql index 263cf94fb4a..e7362074a05 100644 --- a/tests/queries/0_stateless/00063_check_query.sql +++ b/tests/queries/0_stateless/00063_check_query.sql @@ -8,9 +8,6 @@ INSERT INTO check_query_tiny_log VALUES (1, 'A'), (2, 'B'), (3, 'C'); CHECK TABLE check_query_tiny_log; --- Settings and FORMAT are supported -CHECK TABLE check_query_tiny_log SETTINGS max_threads = 16; -CHECK TABLE check_query_tiny_log FORMAT Null SETTINGS max_threads = 8, check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_log; diff --git a/tests/queries/0_stateless/00961_check_table.sql b/tests/queries/0_stateless/00961_check_table.sql index 079acc8cdbb..0e0b2c3b483 100644 --- a/tests/queries/0_stateless/00961_check_table.sql +++ b/tests/queries/0_stateless/00961_check_table.sql @@ -3,29 +3,29 @@ DROP TABLE IF EXISTS mt_table; CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 3, 'quick'), (toDate('2019-01-02'), 4, 'brown'); SELECT '========'; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; OPTIMIZE TABLE mt_table FINAL; SELECT '========'; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; SELECT '========'; INSERT INTO mt_table VALUES (toDate('2019-02-03'), 5, '!'), (toDate('2019-02-03'), 6, '?'); -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; SELECT '========'; @@ -33,6 +33,6 @@ INSERT INTO mt_table VALUES (toDate('2019-02-03'), 7, 'jump'), (toDate('2019-02- OPTIMIZE TABLE mt_table FINAL; -CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; +CHECK TABLE mt_table PARTITION 201902; DROP TABLE IF EXISTS mt_table; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index eccb2d25878..b66aff8384d 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -7,11 +7,11 @@ CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeT -- Rows in this table are short, so granularity will be 8192. INSERT INTO check_query_test SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test SETTINGS max_threads = 1; +CHECK TABLE check_query_test; OPTIMIZE TABLE check_query_test; -CHECK TABLE check_query_test SETTINGS max_threads = 1; +CHECK TABLE check_query_test; DROP TABLE IF EXISTS check_query_test; @@ -21,18 +21,18 @@ CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) EN INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 77; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; DROP TABLE IF EXISTS check_query_test_non_adaptive; diff --git a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql index 625be63e0c0..0ac97404c46 100644 --- a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql +++ b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql @@ -12,6 +12,7 @@ SELECT name, column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_02235' ORDER BY name, column; -CHECK TABLE t_sparse_02235 SETTINGS check_query_single_value_result = 0, max_threads = 1; +SET check_query_single_value_result = 0; +CHECK TABLE t_sparse_02235; DROP TABLE t_sparse_02235; diff --git a/tests/queries/0_stateless/02841_check_table_progress.reference b/tests/queries/0_stateless/02841_check_table_progress.reference deleted file mode 100644 index 541dab48def..00000000000 --- a/tests/queries/0_stateless/02841_check_table_progress.reference +++ /dev/null @@ -1,2 +0,0 @@ -Ok -Ok diff --git a/tests/queries/0_stateless/02841_check_table_progress.sh b/tests/queries/0_stateless/02841_check_table_progress.sh deleted file mode 100755 index 166386b999b..00000000000 --- a/tests/queries/0_stateless/02841_check_table_progress.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t0"; -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t0 (x UInt64, val String) ENGINE = MergeTree ORDER BY x PARTITION BY x % 100"; -${CLICKHOUSE_CLIENT} -q "INSERT INTO t0 SELECT sipHash64(number), randomPrintableASCII(1000) FROM numbers(1000)"; - - -# Check that we have at least 3 different values for read_rows -UNIQUE_VALUES=$( - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"read_rows"\s*:\s*"[0-9]*"' - } | uniq | wc -l -) - -[ "$UNIQUE_VALUES" -ge "3" ] && echo "Ok" || echo "Fail: got $UNIQUE_VALUES" - - -# Check that we have we have at least 100 total_rows_to_read (at least one check task per partition) -MAX_TOTAL_VALUE=$( - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"total_rows_to_read"\s*:\s*"[0-9]*"' | grep -o '[0-9]*' - } | sort -n | tail -1 -) - -[ "$MAX_TOTAL_VALUE" -ge "100" ] && echo "Ok" || echo "Fail: got $MAX_TOTAL_VALUE" From 1377d86ed9d6aaf17878b8c7d2960a0053b1111d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 17:01:09 +0300 Subject: [PATCH 184/249] Update src/Functions/array/arrayAUC.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Functions/array/arrayAUC.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index caf929ba038..b7bd7dcc0ad 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -127,8 +127,8 @@ private: } static void vector( - const IColumn & data1, - const IColumn & data2, + const IColumn & scores, + const IColumn & labels, const ColumnArray::Offsets & offsets, PaddedPODArray & result) { From 635e7e74a86b80519544167eeb2d771a612d6a34 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 16:52:57 +0200 Subject: [PATCH 185/249] add garbage --- .../test_s3_zero_copy_ttl/__init__.py | 0 .../configs/max_delayed_streams.xml | 9 ++ .../test_s3_zero_copy_ttl/configs/s3.xml | 39 ++++++++ .../integration/test_s3_zero_copy_ttl/test.py | 94 +++++++++++++++++++ .../test_vertical_merge_memory_usage.py | 46 +++++++++ 5 files changed, 188 insertions(+) create mode 100644 tests/integration/test_s3_zero_copy_ttl/__init__.py create mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml create mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/s3.xml create mode 100644 tests/integration/test_s3_zero_copy_ttl/test.py create mode 100644 tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py diff --git a/tests/integration/test_s3_zero_copy_ttl/__init__.py b/tests/integration/test_s3_zero_copy_ttl/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml new file mode 100644 index 00000000000..54f7152690b --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml @@ -0,0 +1,9 @@ + + + + + + 10 + + + diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml new file mode 100644 index 00000000000..7bb7fa875e4 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml @@ -0,0 +1,39 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + +
+ default +
+ + s3_disk + +
+
+ + +
+ s3_disk +
+
+
+
+
+ + + true + 1.0 + + + true +
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py new file mode 100644 index 00000000000..04bff4a44fb --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -0,0 +1,94 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_ttl_move_and_s3(started_cluster): + for i, node in enumerate([node1, node2, node3]): + node.query( + """ + CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') + ORDER BY id + PARTITION BY id + TTL date TO DISK 's3_disk' + SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 + """.format( + i + ) + ) + + node1.query("SYSTEM STOP MOVES s3_test_with_ttl") + + node2.query("SYSTEM STOP MOVES s3_test_with_ttl") + + for i in range(30): + if i % 2 == 0: + node = node1 + else: + node = node2 + + node.query( + f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" + ) + + node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + node1.query("SYSTEM START MOVES s3_test_with_ttl") + node2.query("SYSTEM START MOVES s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + for attempt in reversed(range(5)): + time.sleep(5) + + print( + node1.query( + "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" + ) + ) + + minio = cluster.minio_client + objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) + counter = 0 + for obj in objects: + print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") + counter += 1 + + print(f"Total objects: {counter}") + + if counter == 330: + break + + print(f"Attempts remaining: {attempt}") + + assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py new file mode 100644 index 00000000000..fb9f3eb67b9 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + + +single_node_cluster = ClickHouseCluster(__file__) +small_node = single_node_cluster.add_instance( + "small_node", + main_configs=["configs/s3.xml"], + user_configs=["configs/max_delayed_streams.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module") +def started_single_node_cluster(): + try: + single_node_cluster.start() + + yield single_node_cluster + finally: + single_node_cluster.shutdown() + + +def test_vertical_merge_memory_usage(started_single_node_cluster): + if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): + pytest.skip("Disabled for debug and sanitizers. Too slow.") + + small_node.query( + "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" + ) + + small_node.query( + "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" + ) + small_node.query("optimize table tvm2 final") + small_node.query("system flush logs") + + # Should be about 25M + res = small_node.query( + "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" + ) + + assert res == "" From 708fd914bc708345fcc46fadd19b47eceddc786f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 5 Aug 2023 13:47:57 +0200 Subject: [PATCH 186/249] Fix 02263_format_insert_settings flakiness I guess the problem was with the async nature of the process substitution ("2> >(cmd)"), let's avoid using this feature of bash. CI: https://s3.amazonaws.com/clickhouse-test-reports/52683/b98cb7fa145d1a92c2c78421be1eeb8fe8353d53/stateless_tests__aarch64_.html Signed-off-by: Azat Khuzhin Co-authored-by: Alexey Milovidov Update 02263_format_insert_settings.sh --- .../02263_format_insert_settings.reference | 6 +++--- .../0_stateless/02263_format_insert_settings.sh | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index 721e7960875..e2d1ec3980e 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -1,6 +1,6 @@ -insert into foo settings max_threads=1 +[multi] insert into foo settings max_threads=1 Syntax error (query): failed at position 40 (end of query): -insert into foo format tsv settings max_threads=1 +[multi] insert into foo format tsv settings max_threads=1 Can't format ASTInsertQuery with data, since data will be lost. [multi] insert into foo format tsv settings max_threads=1 INSERT INTO foo @@ -8,7 +8,7 @@ SETTINGS max_threads = 1 FORMAT tsv [oneline] insert into foo format tsv settings max_threads=1 INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv -insert into foo settings max_threads=1 format tsv settings max_threads=1 +[multi] insert into foo settings max_threads=1 format tsv settings max_threads=1 You have SETTINGS before and after FORMAT Cannot parse input: expected '\n' before: 'settings max_threads=1 1' 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index efb3d39ab6c..8b156ffec83 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -8,7 +8,7 @@ function run_format() { local q="$1" && shift - echo "$q" + echo "[multi] $q" $CLICKHOUSE_FORMAT "$@" <<<"$q" } function run_format_both() @@ -22,20 +22,20 @@ function run_format_both() } # NOTE: that those queries may work slow, due to stack trace obtaining -run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") +run_format 'insert into foo settings max_threads=1' |& grep --max-count 2 --only-matching -e "Syntax error (query): failed at position .* (end of query):" -e '^\[.*$' # compatibility -run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") +run_format 'insert into foo format tsv settings max_threads=1' |& grep --max-count 2 --only-matching -e "Can't format ASTInsertQuery with data, since data will be lost." -e '^\[.*$' run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert -run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert |& grep --max-count 2 --only-matching -e "You have SETTINGS before and after FORMAT" -e '^\[.*$' # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' -$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'") +$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' |& grep --max-count 1 -F --only-matching "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'" $CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1' $CLICKHOUSE_CLIENT -q 'select * from data_02263' -$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' |& grep --max-count 1 -F --only-matching "You have SETTINGS before and after FORMAT" $CLICKHOUSE_CLIENT -q 'drop table data_02263' run_format_both 'insert into foo values' From cda633a1f66d68f282722d9cdfa7ab445a50f9cc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 15:48:51 +0000 Subject: [PATCH 187/249] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 23d14d6ed26..4d4d1fab473 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=bfd") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 4f0be777c5e186f3ebeabd5c1d8fd9adc2fe761b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 13:18:32 +0000 Subject: [PATCH 188/249] Fix build --- .../tests/gtest_transform_query_for_external_database.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 5c1442ece11..749a154c19d 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -127,7 +127,8 @@ static void checkOld( std::string transformed_query = transformQueryForExternalDatabase( query_info, query_info.syntax_analyzer_result->requiredSourceColumns(), - state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } @@ -180,7 +181,8 @@ static void checkNewAnalyzer( query_info.table_expression = findTableExpression(query_node->getJoinTree(), "table"); std::string transformed_query = transformQueryForExternalDatabase( - query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } From 3acb10b8bc08077c9ac39149517709ddc2c3163a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:51:03 +0200 Subject: [PATCH 189/249] Inhibit randomization in `00906_low_cardinality_cache` --- tests/ci/stress.py | 2 +- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index eb829cf519c..2c566144f2c 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -302,7 +302,7 @@ if __name__ == "__main__": have_long_running_queries = prepare_for_hung_check(args.drop_databases) except Exception as ex: have_long_running_queries = True - logging.error("Failed to prepare for hung check %s", str(ex)) + logging.error("Failed to prepare for hung check: %s", str(ex)) logging.info("Checking if some queries hung") cmd = " ".join( [ diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index cd2ceabcf6d..55eacd0db44 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,5 +1,5 @@ drop table if exists lc_00906; -create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b; +create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_00906 select '0123456789' from numbers(100000000); select count(), b from lc_00906 group by b; drop table if exists lc_00906; From 3aca2408548bc149f933379506250e49238a24de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:59:33 +0200 Subject: [PATCH 190/249] Change the default of max_concurrent_queries from 100 to 1000 --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 14b8954fc39..85cdda63558 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -317,7 +317,7 @@ 0 - 100 + 1000 - false - -
\ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml deleted file mode 100644 index a0e4e3e2216..00000000000 --- a/tests/integration/test_session_log/configs/session_log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - system - session_log
- - toYYYYMM(event_date) - 7500 -
-
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml deleted file mode 100644 index 0416dfadc8a..00000000000 --- a/tests/integration/test_session_log/configs/users.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - 0 - - - - - - - pass - - - pass - - - pass - - - pass - - - \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto deleted file mode 120000 index 25d15f11e3b..00000000000 --- a/tests/integration/test_session_log/protos/clickhouse_grpc.proto +++ /dev/null @@ -1 +0,0 @@ -../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py deleted file mode 100644 index dbb39993ce3..00000000000 --- a/tests/integration/test_session_log/test.py +++ /dev/null @@ -1,289 +0,0 @@ -import os - -import grpc -import pymysql.connections -import psycopg2 as py_psql -import pytest -import random -import sys -import threading - -from helpers.cluster import ClickHouseCluster, run_and_check - -POSTGRES_SERVER_PORT = 5433 -MYSQL_SERVER_PORT = 9001 -GRPC_PORT = 9100 -SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DEFAULT_ENCODING = "utf-8" - -# Use grpcio-tools to generate *pb2.py files from *.proto. -proto_dir = os.path.join(SCRIPT_DIR, "./protos") -gen_dir = os.path.join(SCRIPT_DIR, "./_gen") -os.makedirs(gen_dir, exist_ok=True) -run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", - shell=True, -) - -sys.path.append(gen_dir) - -import clickhouse_grpc_pb2 -import clickhouse_grpc_pb2_grpc - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "node", - main_configs=[ - "configs/ports.xml", - "configs/log.xml", - "configs/session_log.xml", - ], - user_configs=["configs/users.xml"], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") - }, -) - - -def grpc_get_url(): - return f"{instance.ip_address}:{GRPC_PORT}" - - -def grpc_create_insecure_channel(): - channel = grpc.insecure_channel(grpc_get_url()) - grpc.channel_ready_future(channel).result(timeout=2) - return channel - - -session_id_counter = 0 - - -def next_session_id(): - global session_id_counter - session_id = session_id_counter - session_id_counter += 1 - return str(session_id) - - -def grpc_query(query, user_, pass_, raise_exception): - try: - query_info = clickhouse_grpc_pb2.QueryInfo( - query=query, - session_id=next_session_id(), - user_name=user_, - password=pass_, - ) - channel = grpc_create_insecure_channel() - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) - result = stub.ExecuteQuery(query_info) - if result and result.HasField("exception"): - raise Exception(result.exception.display_text) - - return result.output.decode(DEFAULT_ENCODING) - except Exception: - assert raise_exception - - -def postgres_query(query, user_, pass_, raise_exception): - try: - client = py_psql.connect( - host=instance.ip_address, - port=POSTGRES_SERVER_PORT, - user=user_, - password=pass_, - database="default", - ) - cursor = client.cursor() - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -def mysql_query(query, user_, pass_, raise_exception): - try: - client = pymysql.connections.Connection( - host=instance.ip_address, - user=user_, - password=pass_, - database="default", - port=MYSQL_SERVER_PORT, - ) - cursor = client.cursor(pymysql.cursors.DictCursor) - if raise_exception: - with pytest.raises(Exception): - cursor.execute(query) - else: - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_parallel_sessions(started_cluster): - thread_list = [] - for _ in range(10): - # Sleep time does not significantly matter here, - # test should pass even without sleeping. - for function in [postgres_query, grpc_query, mysql_query]: - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "pass", - False, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "wrong_pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", - "pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - - for thread in thread_list: - thread.join() - - instance.query("SYSTEM FLUSH LOGS") - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" - ) - assert port_0_sessions == "90\n" - - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" - ) - assert port_0_sessions == "0\n" - - address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" - ) - assert address_0_sessions == "0\n" - - grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" - ) - assert grpc_sessions == "30\n" - - mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" - ) - assert mysql_sessions == "30\n" - - postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" - ) - assert postgres_sessions == "30\n" - - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" - - logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" - ) - assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference deleted file mode 100644 index bfe507e8eac..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ /dev/null @@ -1,34 +0,0 @@ -sessions: -150 -port_0_sessions: -0 -address_0_sessions: -0 -tcp_sessions -60 -http_sessions -30 -http_with_session_id_sessions -30 -my_sql_sessions -30 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh deleted file mode 100755 index 26b48462a76..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -# Each user uses a separate thread. -readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") -readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) - -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" - -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; -done - -# All _session functions execute in separate threads. -# These functions try to create a session with successful login and logout. -# Sleep a small, random amount of time to make concurrency more intense. -# and try to login with an invalid password. -function tcp_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' - done -} - -function http_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" - done -} - -function http_with_session_id_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" - done -} - -function mysql_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" - - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; -export -f mysql_session; - -for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & -done - -for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & -done - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" - -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" - -echo "tcp_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" -echo "http_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "http_with_session_id_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" - echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" - done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference deleted file mode 100644 index e2680982ab0..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.reference +++ /dev/null @@ -1,13 +0,0 @@ -0 -0 -0 -0 -client_port 0 connections: -0 -client_address '::' connections: -0 -login failures: -0 -TCP Login and logout count is equal -HTTP Login and logout count is equal -MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh deleted file mode 100755 index 3bedfb6c9ee..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "client_port 0 connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" - -echo "client_address '::' connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" - -echo "login failures:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" - -# remote(...) function sometimes reuses old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. - -for interface in 'TCP' 'HTTP' 'MySQL' -do - LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` - CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` - - if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then - echo "${interface} Login and logout count is equal" - else - TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` - echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" - fi -done - -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference deleted file mode 100644 index 7252faab8c6..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.reference +++ /dev/null @@ -1,8 +0,0 @@ -port_0_sessions: -0 -address_0_sessions: -0 -Corresponding LoginSuccess/Logout -9 -LoginFailure -0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh deleted file mode 100755 index 347ebd22f96..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ /dev/null @@ -1,114 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -readonly TEST_USER="02835_USER_${PID}" -readonly TEST_ROLE="02835_ROLE_${PID}" -readonly TEST_PROFILE="02835_PROFILE_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -function tcp_session() -{ - local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" -} - -function http_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -function http_with_session_id_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. -function wait_for_queries_start() -{ - local user=$1 - local queries_count=$2 - # 10 seconds waiting - counter=0 retries=100 - while [[ $counter -lt $retries ]]; do - result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") - if [[ $result == "${queries_count}" ]]; then - break; - fi - sleep 0.1 - ((++counter)) - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -# DROP USE CASE -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP ROLE CASE -${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP PROFILE CASE -${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" -echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From b9638c6387d46e85abe55a333049efb0d2c757e7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 22:16:30 +0000 Subject: [PATCH 205/249] path to libc --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 4d4d1fab473..03314a9d2ba 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From ea8ae28c7dfeae6c3641c8617152750aa0d94be2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:40:18 +0200 Subject: [PATCH 206/249] Fix build with clang-17 --- src/Core/examples/coro.cpp | 194 ------------------------------------- 1 file changed, 194 deletions(-) delete mode 100644 src/Core/examples/coro.cpp diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp deleted file mode 100644 index fbccc261e9d..00000000000 --- a/src/Core/examples/coro.cpp +++ /dev/null @@ -1,194 +0,0 @@ -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#if defined(__clang__) -#include - -namespace std // NOLINT(cert-dcl58-cpp) -{ - using namespace experimental::coroutines_v1; // NOLINT(cert-dcl58-cpp) -} - -#if __has_warning("-Wdeprecated-experimental-coroutine") -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wdeprecated-experimental-coroutine" -#endif - -#else -#include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wzero-as-null-pointer-constant" -#endif - - -template -struct suspend_value // NOLINT(readability-identifier-naming) -{ - constexpr bool await_ready() const noexcept { return true; } // NOLINT(readability-identifier-naming) - constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} // NOLINT(readability-identifier-naming) - constexpr T await_resume() const noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " ret " << val << std::endl; - return val; - } - - T val; -}; - -template -struct Task -{ - struct promise_type // NOLINT(readability-identifier-naming) - { - using coro_handle = std::coroutine_handle; - auto get_return_object() { return coro_handle::from_promise(*this); } // NOLINT(readability-identifier-naming) - auto initial_suspend() { return std::suspend_never(); } // NOLINT(readability-identifier-naming) - auto final_suspend() noexcept { return suspend_value{*r->value}; } // NOLINT(readability-identifier-naming) - //void return_void() {} - void return_value(T value_) { r->value = value_; } // NOLINT(readability-identifier-naming) - void unhandled_exception() // NOLINT(readability-identifier-naming) - { - DB::tryLogCurrentException("Logger"); - r->exception = std::current_exception(); // NOLINT(bugprone-throw-keyword-missing) - } - - explicit promise_type(std::string tag_) : tag(tag_) {} - ~promise_type() { std::cout << "~promise_type " << tag << std::endl; } - std::string tag; - coro_handle next; - Task * r = nullptr; - }; - - using coro_handle = std::coroutine_handle; - - bool await_ready() const noexcept { return false; } // NOLINT(readability-identifier-naming) - void await_suspend(coro_handle g) noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_suspend " << my.promise().tag << std::endl; - std::cout << " g tag " << g.promise().tag << std::endl; - g.promise().next = my; - } - T await_resume() noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_res " << my.promise().tag << std::endl; - return *value; - } - - Task(coro_handle handle) : my(handle), tag(handle.promise().tag) // NOLINT(google-explicit-constructor) - { - assert(handle); - my.promise().r = this; - std::cout << " Task " << tag << std::endl; - } - Task(Task &) = delete; - Task(Task &&rhs) noexcept : my(rhs.my), tag(rhs.tag) - { - rhs.my = {}; - std::cout << " Task&& " << tag << std::endl; - } - static bool resumeImpl(Task *r) - { - if (r->value) - return false; - - auto & next = r->my.promise().next; - - if (next) - { - if (resumeImpl(next.promise().r)) - return true; - next = {}; - } - - if (!r->value) - { - r->my.resume(); - if (r->exception) - std::rethrow_exception(r->exception); - } - return !r->value; - } - - bool resume() - { - return resumeImpl(this); - } - - T res() - { - return *value; - } - - ~Task() - { - std::cout << " ~Task " << tag << std::endl; - } - -private: - coro_handle my; - std::string tag; - std::optional value; - std::exception_ptr exception; -}; - -Task boo([[maybe_unused]] std::string tag) -{ - std::cout << "x" << std::endl; - co_await std::suspend_always(); - std::cout << StackTrace().toString(); - std::cout << "y" << std::endl; - co_return 1; -} - -Task bar([[maybe_unused]] std::string tag) -{ - std::cout << "a" << std::endl; - int res1 = co_await boo("boo1"); - std::cout << "b " << res1 << std::endl; - int res2 = co_await boo("boo2"); - if (res2 == 1) - throw DB::Exception(1, "hello"); - std::cout << "c " << res2 << std::endl; - co_return res1 + res2; // 1 + 1 = 2 -} - -Task foo([[maybe_unused]] std::string tag) -{ - std::cout << "Hello" << std::endl; - auto res1 = co_await bar("bar1"); - std::cout << "Coro " << res1 << std::endl; - auto res2 = co_await bar("bar2"); - std::cout << "World " << res2 << std::endl; - co_return res1 * res2; // 2 * 2 = 4 -} - -int main() -{ - Poco::AutoPtr app_channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(app_channel); - Poco::Logger::root().setLevel("trace"); - - LOG_INFO(&Poco::Logger::get(""), "Starting"); - - try - { - auto t = foo("foo"); - std::cout << ".. started" << std::endl; - while (t.resume()) - std::cout << ".. yielded" << std::endl; - std::cout << ".. done: " << t.res() << std::endl; - } - catch (DB::Exception & e) - { - std::cout << "Got exception " << e.what() << std::endl; - std::cout << e.getStackTraceString() << std::endl; - } -} From ea62629d83ee487e64b1ba3c051211103f258069 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:40:26 +0200 Subject: [PATCH 207/249] Fix linking of examples --- src/Core/examples/CMakeLists.txt | 3 --- src/Parsers/examples/CMakeLists.txt | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index 868173e0e31..2326eada96d 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -9,6 +9,3 @@ target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) clickhouse_add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) - -clickhouse_add_executable (coro coro.cpp) -target_link_libraries (coro PRIVATE clickhouse_common_io) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index e411574bd65..261f234081c 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -4,7 +4,7 @@ clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(select_parser PRIVATE clickhouse_parsers) +target_link_libraries(select_parser PRIVATE dbms) clickhouse_add_executable(create_parser create_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(create_parser PRIVATE clickhouse_parsers) +target_link_libraries(create_parser PRIVATE dbms) From 3193c6d4ccc7ef02857f14825e1a781e8c2e83c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:55:57 +0200 Subject: [PATCH 208/249] Fix build --- src/Functions/array/arrayAUC.cpp | 2 +- src/Server/ServerType.cpp | 2 +- src/Server/ServerType.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index b7bd7dcc0ad..499fe4ce7b2 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -139,7 +139,7 @@ private: for (size_t i = 0; i < size; ++i) { auto next_offset = offsets[i]; - result[i] = apply(data1, data2, current_offset, next_offset); + result[i] = apply(scores, labels, current_offset, next_offset); current_offset = next_offset; } } diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index 4952cd1bd24..29ba7224c75 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -2,10 +2,10 @@ #include #include -#include #include + namespace DB { diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 1fab492222a..eafe4f941dd 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -1,13 +1,13 @@ #pragma once #include + namespace DB { class ServerType { public: - enum Type { TCP, From 396db7ab3cee77955173de76f355e1d08c1ec755 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 01:01:43 +0200 Subject: [PATCH 209/249] Fix test `00002_log_and_exception_messages_formatting` --- .../00002_log_and_exception_messages_formatting.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 86fe01dc0e3..54daeb928a3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,9 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}', +'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', +'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From c98e38611fccca5acfaf466a7ce22e9ae2bc0a6a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 23:36:47 +0000 Subject: [PATCH 210/249] path to libc --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 03314a9d2ba..945eb9affa4 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 3b9d2c708096e51b106acffbfd3cc2c4ac8606e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 03:06:07 +0200 Subject: [PATCH 211/249] Simplification --- .../config.d/create_logs_at_startup.yaml | 1 - programs/server/config.xml | 3 - src/Common/SystemLogBase.h | 2 +- src/Interpreters/Context.cpp | 16 ++++ src/Interpreters/Context.h | 3 + src/Interpreters/InterpreterSystemQuery.cpp | 82 ++++++++----------- src/Interpreters/SystemLog.cpp | 14 +--- src/Interpreters/SystemLog.h | 5 +- .../config.d/create_logs_at_startup.yaml | 34 -------- tests/config/install.sh | 1 - 10 files changed, 57 insertions(+), 104 deletions(-) delete mode 120000 programs/server/config.d/create_logs_at_startup.yaml delete mode 100644 tests/config/config.d/create_logs_at_startup.yaml diff --git a/programs/server/config.d/create_logs_at_startup.yaml b/programs/server/config.d/create_logs_at_startup.yaml deleted file mode 120000 index a09d2783a9e..00000000000 --- a/programs/server/config.d/create_logs_at_startup.yaml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/create_logs_at_startup.yaml \ No newline at end of file diff --git a/programs/server/config.xml b/programs/server/config.xml index 3d1c92b135c..14b8954fc39 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1037,9 +1037,6 @@ - - - false