From 77e45e5669eae8ae05c5acdc5b0cbf6647d1f50a Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 10 Jul 2023 20:23:03 -0400 Subject: [PATCH 001/101] 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 6179c823b56c..0c61a9e3e1be 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 069a966c8e9c..e3bd32c02921 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 f36a670ddbfb..f36beb3f80b6 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 6a0c1bdfc028..1898c6ba5d34 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -41,6 +41,11 @@ class MultipleAccessStorage : public IAccessStorage 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 b533ccbf200d..d8ddf07ff40d 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 3386dfb8792a..87abe002d3da 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 d4c37064065d..54e3b95226c3 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 d624b9a9157f..14946f2d3cd2 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 42d1a4031b61..4e465553164d 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.h +++ b/src/Parsers/Access/ASTCreateRoleQuery.h @@ -28,6 +28,7 @@ class ASTCreateRoleQuery : public IAST, public ASTQueryWithOnCluster 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 88f2d7bce630..02b29f3fd2cd 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 f3a065c50df9..32f4a8f80473 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -21,6 +21,7 @@ class ASTDropAccessEntityQuery : public IAST, public ASTQueryWithOnCluster 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 da9749958eed..de9e5baed189 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 d21164bc1a23..d5ecdf27449e 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 fbad78e7bced..717911bf2f1a 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 cf5a24f88cd3..8fcd0fd7cf5a 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 000000000000..e69de29bb2d1 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 000000000000..672c7e247286 --- /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 000000000000..473a37695e92 --- /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 002/101] 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 473a37695e92..d9a4a73c56b6 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 1a4d48324e3ffe1f8c0c05a337b590ce8aa9ff33 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 11 Jul 2023 19:58:28 -0400 Subject: [PATCH 003/101] 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 9b3c4eabd478..42f091bddd52 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 dbb73bca8511..a10d5758752f 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 6e3169b7fec9..d73d9752bc10 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 6b844e0d2f61..308bbf790241 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 3834b05601fa..aaff6d92d329 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 2a00e2aa6ae7c417fdd5aafd63225ef44460de6f Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 20 Jul 2023 22:20:36 -0400 Subject: [PATCH 004/101] 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 0c61a9e3e1be..22b00d5dde70 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 2a8293a49e77..85feaa704620 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ class AccessControl : public MultipleAccessStorage 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 938881fafff2..acbd7b125694 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 e3bd32c02921..3541c3228e35 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ class DiskAccessStorage : public IAccessStorage 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 9468e8d220ab..11bc97f6cc32 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 aa3947201e7b..894220e95ad5 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -92,6 +92,7 @@ class IAccessStorage : public boost::noncopyable /// 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 @@ class IAccessStorage : public boost::noncopyable 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 @@ class IAccessStorage : public boost::noncopyable /// 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 @@ class IAccessStorage : public boost::noncopyable 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 c4192ee45524..16f3b3793a06 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 b63132147da2..7c19ab9c4ab9 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -44,7 +44,7 @@ class MemoryAccessStorage : public IAccessStorage 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 f36beb3f80b6..169dc8895092 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 1898c6ba5d34..1d95296b796b 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -46,6 +46,11 @@ class MultipleAccessStorage : public IAccessStorage 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 @@ class MultipleAccessStorage : public IAccessStorage 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 @@ class MultipleAccessStorage : public IAccessStorage 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 56d68be92689..efa9dbbdfaa0 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 555d58e6b044..0a0c53d3bc4e 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,7 +46,7 @@ class ReplicatedAccessStorage : public IAccessStorage 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 83a620d11c6e..e271497ff5cc 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 87abe002d3da..f6c3103a7a9c 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 87dc9039c174..a938d7afc160 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 0727b6f2182d..bac0146e8217 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 165937560ccf..f1d921a9eb22 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 000000000000..ac7ff4efcb62 --- /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 000000000000..1a70bf7c0652 --- /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 9cd1f2a251c5..27c941197501 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 56abedf5235b..091b62b0a9fd 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 f5eb59800ec5..aecbbb00f9ae 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -38,6 +38,7 @@ class ASTCreateQuotaQuery : public IAST, public ASTQueryWithOnCluster 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 14946f2d3cd2..eeeb34c97e45 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 ca888be2cfe2..a423581e300a 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 32d98fab8220..3f2418e72879 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -35,6 +35,7 @@ class ASTCreateRowPolicyQuery : public IAST, public ASTQueryWithOnCluster 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 56ddef433ef5..3379486d789d 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 c1a64998f29e..be01aae1e26d 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h @@ -28,6 +28,7 @@ class ASTCreateSettingsProfileQuery : public IAST, public ASTQueryWithOnCluster 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 0611545adf02..dbb4c1c6d574 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 62ddbfd00406..06940e67d914 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -45,6 +45,7 @@ class ASTCreateUserQuery : public IAST, public ASTQueryWithOnCluster 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 02b29f3fd2cd..bcd7105d0e97 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 000000000000..285f07854c0c --- /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 000000000000..aa2b3b0f98c4 --- /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 2330ba7771ca..05c2820b012c 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 de9e5baed189..b548c07ec70f 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 2c25fc14e7d5..b2177ca4ade6 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 c58a3035dc64..634ff68427a5 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 0344fb99c043..d95d41296dc7 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 000000000000..6ac41b77b84f --- /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 000000000000..3accec883dd5 --- /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 717911bf2f1a..64fed2ce21c5 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 aa5302a15b9c..92fb8ed275bd 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -290,6 +290,7 @@ class IAST : public std::enable_shared_from_this, public TypePromotion #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 ee5d4b5df932..8e040a18a885 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 000000000000..c2f60c8bbc1d --- /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 672c7e247286..faaf1ee27aec 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 d9a4a73c56b6..fb4bbd6b7187 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 @@ 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 9a352f43eabc..43a402d9fdac 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 13acc3f1042e..8770cc82064a 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 45afb86f464c..704fb30b2fdc 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 005/101] 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 fb4bbd6b7187..4c2696158ec0 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 @@ 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 393481181d5fb7c878af2f1e0bbbd5d9c8d56807 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 24 Jul 2023 13:35:07 -0400 Subject: [PATCH 006/101] 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 a423581e300a..e95959703ee4 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 dbb4c1c6d574..b7290c160135 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 007/101] 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 9890d257e84b..a398a7c09931 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 b15bab47ae00..3565af326fee 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 @@ class FunctionStartsEndsWith : public IFunction 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 @@ class FunctionStartsEndsWith : public IFunction 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 @@ class FunctionStartsEndsWith : public IFunction 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 @@ class FunctionStartsEndsWith : public IFunction 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 @@ class FunctionStartsEndsWith : public IFunction 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 @@ class FunctionStartsEndsWith : public IFunction 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 000000000000..377683ce47c8 --- /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 000000000000..e623a55f4dcd --- /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 000000000000..ca2a5bc50f88 --- /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 000000000000..3a783dc280ec --- /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 008/101] 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 3565af326fee..7671129fcfcb 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 009/101] 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 a398a7c09931..f36de0dc0d88 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 247df64f741a..d70214e99308 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 65271830555e..3cc25108eb17 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 010/101] 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 f36de0dc0d88..b1ffb32e7cd9 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 377683ce47c8..1b042452298c 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 e623a55f4dcd..3f46916d7607 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 d70214e99308..247df64f741a 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 011/101] 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 11bc97f6cc32..db73db900aa6 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 43a402d9fdac..0e1bbea9cea0 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 eb89a5ed38c0..b7c30b304bf9 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 012/101] 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 3cc25108eb17..240238cab128 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 013/101] 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 22b00d5dde70..3759b6c89194 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 85feaa704620..60b40775a564 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ class AccessControl : public MultipleAccessStorage 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 acbd7b125694..190c7567b854 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 3541c3228e35..5d94008b34fa 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ class DiskAccessStorage : public IAccessStorage 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 @@ class DiskAccessStorage : public IAccessStorage 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 db73db900aa6..468c93c1116e 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 894220e95ad5..b82537df6395 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 @@ class IAccessStorage : public boost::noncopyable /// 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 @@ class IAccessStorage : public boost::noncopyable 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 16f3b3793a06..999e2c41d0b2 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 7c19ab9c4ab9..f877860041a6 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -22,11 +22,6 @@ class MemoryAccessStorage : public IAccessStorage 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 @@ class MemoryAccessStorage : public IAccessStorage 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 169dc8895092..e6665349ae64 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 1d95296b796b..069d414f6013 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -63,7 +63,7 @@ class MultipleAccessStorage : public IAccessStorage 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 efa9dbbdfaa0..275672da1154 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 0a0c53d3bc4e..cddb20860f78 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,11 +46,10 @@ class ReplicatedAccessStorage : public IAccessStorage 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 05c2820b012c..718498ed1d13 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 b548c07ec70f..99a97f6901d4 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 b2177ca4ade6..f9e9466e35d1 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 634ff68427a5..fcb6ebd88067 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 d95d41296dc7..577b6772ab27 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 d5ecdf27449e..7fd34a6f8e42 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 6ac41b77b84f..798e200bfb3a 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 64fed2ce21c5..10c548d88263 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 014/101] 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 b0f19f0d8bec..40f756b95888 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 2636898c1b3c..1fe08a738c28 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 b98cb7fa145d1a92c2c78421be1eeb8fe8353d53 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 17:50:39 +0000 Subject: [PATCH 015/101] 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 1fe08a738c28..bbb1c517d7c9 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 016/101] 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 c69285171ab5..a6ced870c18f 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 9b14e220e1ff..4b6fffe4f60a 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 83bb2e6bb9a8..cd7718793bd5 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 8e221a4d82fb..d8afce9d6de2 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 d168be63c36c..11d4eae7bc81 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 b6208c2fd52a..245fd22e57c3 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 000000000000..fac738ff7117 --- /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 468c93c1116e..ec25fd505f07 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 e6665349ae64..c6c9ede9e1c3 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 ac7ff4efcb62..49e90783a59f 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 4c2696158ec0..77aea4113942 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 4f98b9a0d0d8..cec14b0af73a 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 017/101] 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 509b4de1a373..b620e88e7ebb 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 018/101] 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 1933823f5d2a..acf544c98aaa 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 6f2404b79209bf332e54e83b86eff2f677bbd2bc Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 22:18:53 -0400 Subject: [PATCH 019/101] 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 c6c9ede9e1c3..0550c140c179 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 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 020/101] 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 b1ffb32e7cd9..fff37257917d 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 48b1a0b44f85034d53245d9dd941929378f10d83 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Mon, 31 Jul 2023 08:09:57 -0700 Subject: [PATCH 021/101] Fix for issue #40220 improve error message for table function remote --- src/TableFunctions/TableFunctionRemote.cpp | 13 +++++++++++-- .../02841_remote_parameter_parsing_error.reference | 0 .../02841_remote_parameter_parsing_error.sql | 14 ++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index e4442c565dfb..d88d41901914 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -159,7 +159,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name is missing"); } else { @@ -211,7 +211,16 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (is_cluster_function) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + } + } } if (!cluster_name.empty()) diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql new file mode 100644 index 000000000000..f2ac88b62fb8 --- /dev/null +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -0,0 +1,14 @@ +-- Tags: shard + + +select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } +select * from remote('127.0.0.1', system.one, default); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, ''); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, default, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From dadf4678ee9d678459c96cd13b8231e33dc02cc3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 12:50:40 +0000 Subject: [PATCH 022/101] 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 6b5ea643998f..eb394da8bb6d 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 f67b51f6703954c8e2b345192318ed651d7336d0 Mon Sep 17 00:00:00 2001 From: jiyoungyoooo <140271086+jiyoungyoooo@users.noreply.github.com> Date: Fri, 4 Aug 2023 22:46:22 -0700 Subject: [PATCH 023/101] Update src/TableFunctions/TableFunctionRemote.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index d88d41901914..13a24e96e4b8 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -159,7 +159,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name is missing"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); } else { From 9de95879655cb0ebbd895b0e95198e6ad759f09a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 13:54:36 +0200 Subject: [PATCH 024/101] 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 5e36a97b3ca5..34ac2cfa5723 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 3d5f780a38c9..efb3d39ab6c8 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 025/101] 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 34ac2cfa5723..3b5ae7186fbc 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 026/101] 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 3b5ae7186fbc..4008df3f84f2 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 027/101] 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 4008df3f84f2..0c0f77ec597c 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 da2cb99c126c6610150d71c736d1e7e671eb2659 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Mon, 31 Jul 2023 08:09:57 -0700 Subject: [PATCH 028/101] Fix for issue #40220 improve error message for table function remote --- src/TableFunctions/TableFunctionRemote.cpp | 13 +++++++++++-- .../02841_remote_parameter_parsing_error.reference | 0 .../02841_remote_parameter_parsing_error.sql | 14 ++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 78618d4928fe..0d015196b3ca 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -175,7 +175,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); } else { @@ -228,7 +228,16 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (is_cluster_function) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + } + } } if (!cluster_name.empty()) diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql new file mode 100644 index 000000000000..f2ac88b62fb8 --- /dev/null +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -0,0 +1,14 @@ +-- Tags: shard + + +select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } +select * from remote('127.0.0.1', system.one, default); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, ''); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, default, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 608b5ace3610cf970e898e34b450f54b693faca5 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Sat, 5 Aug 2023 00:26:32 -0700 Subject: [PATCH 029/101] Additional fix for issue#40220 --- src/TableFunctions/TableFunctionRemote.cpp | 15 +++++---------- .../02841_remote_parameter_parsing_error.sql | 2 +- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 0d015196b3ca..d00cda2462f7 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -175,7 +175,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", static_cast(help_message)); } else { @@ -220,8 +220,10 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr ++arg_num; } - if (arg_num < args.size() && !sharding_key) + if (arg_num < args.size()) { + if (sharding_key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); sharding_key = args[arg_num]; ++arg_num; } @@ -229,14 +231,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (arg_num < args.size()) { - if (is_cluster_function) - { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); - } + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql index f2ac88b62fb8..9e467a1f69a8 100644 --- a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -1,7 +1,6 @@ -- Tags: shard -select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } @@ -11,4 +10,5 @@ select * from remote('127.0.0.1', system.one, default, key1); -- { serverError B select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From afb1a55bf3789196e2b9f96aea5318ea0a9f9219 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:56:33 +0200 Subject: [PATCH 030/101] 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 437b1b2a6bb4..c4c2b0b24998 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -131,8 +131,6 @@ class SystemLog : public SystemLogBase, private boost::noncopyable, 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 031/101] 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 c4c2b0b24998..cf287ad77754 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -111,7 +111,7 @@ class SystemLog : public SystemLogBase, private boost::noncopyable, * 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 9428040d05d914fc4d19e9b0f7692743a971db6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 13:07:48 +0200 Subject: [PATCH 032/101] Fix hung check in stress test --- tests/ci/stress.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 60ddeb85d355..eb829cf519cd 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -202,13 +202,14 @@ def prepare_for_hung_check(drop_databases): call( make_query_command( """ - select sleepEachRow(( - select maxOrDefault(300 - elapsed) + 1 - from system.processes - where query not like '%from system.processes%' and elapsed < 300 + SELECT sleepEachRow(( + SELECT maxOrDefault(300 - elapsed) + 1 + FROM system.processes + WHERE query NOT LIKE '%FROM system.processes%' AND elapsed < 300 ) / 300) - from numbers(300) - format Null + FROM numbers(300) + FORMAT Null + SETTINGS function_sleep_max_microseconds_per_block = 0 """ ), shell=True, From c3edb57953d41192f3c39034e105f56db000b035 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 02:29:29 +0200 Subject: [PATCH 033/101] Remove old util --- utils/CMakeLists.txt | 1 - utils/wikistat-loader/CMakeLists.txt | 2 - utils/wikistat-loader/main.cpp | 225 --------------------------- 3 files changed, 228 deletions(-) delete mode 100644 utils/wikistat-loader/CMakeLists.txt delete mode 100644 utils/wikistat-loader/main.cpp diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 47dd2fc9f2d8..4e1184cc9a52 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -20,7 +20,6 @@ if (ENABLE_UTILS) add_subdirectory (zookeeper-cli) add_subdirectory (zookeeper-dump-tree) add_subdirectory (zookeeper-remove-by-list) - add_subdirectory (wikistat-loader) add_subdirectory (check-marks) add_subdirectory (checksum-for-compressed-block) add_subdirectory (check-mysql-binlog) diff --git a/utils/wikistat-loader/CMakeLists.txt b/utils/wikistat-loader/CMakeLists.txt deleted file mode 100644 index fc5416dea2e5..000000000000 --- a/utils/wikistat-loader/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -clickhouse_add_executable (wikistat-loader main.cpp ${SRCS}) -target_link_libraries (wikistat-loader PRIVATE clickhouse_common_io boost::program_options) diff --git a/utils/wikistat-loader/main.cpp b/utils/wikistat-loader/main.cpp deleted file mode 100644 index 493f1df05da0..000000000000 --- a/utils/wikistat-loader/main.cpp +++ /dev/null @@ -1,225 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include - - -/** Reads uncompressed wikistat data from stdin, - * and writes transformed data in tsv format, - * ready to be loaded into ClickHouse. - * - * Input data has format: - * - * aa Wikipedia 1 17224 - * aa.b Main_Page 2 21163 - * - * project, optional subproject, path, hits, total size in bytes. - */ - - -template -static void readString(std::string & s, DB::ReadBuffer & buf) -{ - s.clear(); - - while (!buf.eof()) - { - const char * next_pos; - - if (break_at_dot) - next_pos = find_first_symbols<' ', '\n', '.'>(buf.position(), buf.buffer().end()); - else - next_pos = find_first_symbols<' ', '\n'>(buf.position(), buf.buffer().end()); - - s.append(buf.position(), next_pos - buf.position()); - buf.position() += next_pos - buf.position(); - - if (!buf.hasPendingData()) - continue; - - if (*buf.position() == ' ' || *buf.position() == '\n' || (break_at_dot && *buf.position() == '.')) - return; - } -} - - -/** Reads path before whitespace and decodes %xx sequences (to more compact and handy representation), - * except %2F '/', %26 '&', %3D '=', %3F '?', %23 '#' (to not break structure of URL). - */ -static void readPath(std::string & s, DB::ReadBuffer & buf) -{ - s.clear(); - - while (!buf.eof()) - { - const char * next_pos = find_first_symbols<' ', '\n', '%'>(buf.position(), buf.buffer().end()); - - s.append(buf.position(), next_pos - buf.position()); - buf.position() += next_pos - buf.position(); - - if (!buf.hasPendingData()) - continue; - - if (*buf.position() == ' ' || *buf.position() == '\n') - return; - - if (*buf.position() == '%') - { - ++buf.position(); - - char c1; - char c2; - - if (buf.eof() || *buf.position() == ' ') - break; - - DB::readChar(c1, buf); - - if (buf.eof() || *buf.position() == ' ') - break; - - DB::readChar(c2, buf); - - if ((c1 == '2' && (c2 == 'f' || c2 == '6' || c2 == '3' || c2 == 'F')) - || (c1 == '3' && (c2 == 'd' || c2 == 'f' || c2 == 'D' || c2 == 'F'))) - { - s += '%'; - s += c1; - s += c2; - } - else - s += static_cast(static_cast(unhex(c1)) * 16 + static_cast(unhex(c2))); - } - } -} - - -static void skipUntilNewline(DB::ReadBuffer & buf) -{ - while (!buf.eof()) - { - const char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); - - buf.position() += next_pos - buf.position(); - - if (!buf.hasPendingData()) - continue; - - if (*buf.position() == '\n') - { - ++buf.position(); - return; - } - } -} - - -namespace DB -{ - namespace ErrorCodes - { - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - } -} - - -int main(int argc, char ** argv) -try -{ - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "produce help message") - ("time", boost::program_options::value()->required(), - "time of data in YYYY-MM-DD hh:mm:ss form") - ; - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Reads uncompressed wikistat data from stdin and writes transformed data in tsv format." << std::endl; - std::cout << "Usage: " << argv[0] << " --time='YYYY-MM-DD hh:00:00' < in > out" << std::endl; - std::cout << desc << std::endl; - return 1; - } - - std::string time_str = options.at("time").as(); - LocalDateTime time(time_str); - LocalDate date(time_str); - - DB::ReadBufferFromFileDescriptor in(STDIN_FILENO); - DB::WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - std::string project; - std::string subproject; - std::string path; - UInt64 hits = 0; - UInt64 size = 0; - - size_t row_num = 0; - while (!in.eof()) - { - try - { - ++row_num; - readString(project, in); - - if (in.eof()) - break; - - if (*in.position() == '.') - readString(subproject, in); - else - subproject.clear(); - - DB::assertChar(' ', in); - readPath(path, in); - DB::assertChar(' ', in); - DB::readIntText(hits, in); - DB::assertChar(' ', in); - DB::readIntText(size, in); - DB::assertChar('\n', in); - } - catch (const DB::Exception & e) - { - /// Sometimes, input data has errors. For example, look at first lines in pagecounts-20130210-130000.gz - /// To save rest of data, just skip lines with errors. - if (e.code() == DB::ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED) - { - std::cerr << "At row " << row_num << ": " << DB::getCurrentExceptionMessage(false) << '\n'; - skipUntilNewline(in); - continue; - } - else - throw; - } - - DB::writeText(date, out); - DB::writeChar('\t', out); - DB::writeText(time, out); - DB::writeChar('\t', out); - DB::writeText(project, out); - DB::writeChar('\t', out); - DB::writeText(subproject, out); - DB::writeChar('\t', out); - DB::writeText(path, out); - DB::writeChar('\t', out); - DB::writeText(hits, out); - DB::writeChar('\t', out); - DB::writeText(size, out); - DB::writeChar('\n', out); - } - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; - throw; -} From d37283e9fb23c4065d13d3b4a97ecbb810943b30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 7 Aug 2023 10:46:55 +0200 Subject: [PATCH 034/101] Fix query_id in part_log with async flush queries --- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +++- ...02790_async_queries_in_query_log.reference | 24 +++++++++++++++++++ .../02790_async_queries_in_query_log.sh | 18 ++++++++++++-- 3 files changed, 43 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 4f15192f46cb..c8a3f77556a8 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -460,7 +460,6 @@ try const auto * log = &Poco::Logger::get("AsynchronousInsertQueue"); const auto & insert_query = assert_cast(*key.query); auto insert_context = Context::createCopy(global_context); - DB::CurrentThread::QueryScope query_scope_holder(insert_context); bool internal = false; // To enable logging this query bool async_insert = true; @@ -482,6 +481,9 @@ try insert_context->setInitialQueryStartTime(query_start_time); insert_context->setCurrentQueryId(insert_query_id); insert_context->setInitialQueryId(insert_query_id); + + DB::CurrentThread::QueryScope query_scope_holder(insert_context); + size_t log_queries_cut_to_length = insert_context->getSettingsRef().log_queries_cut_to_length; String query_for_logging = insert_query.hasSecretParts() ? insert_query.formatForLogging(log_queries_cut_to_length) diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference index e202a38f068f..aa18817f4e6a 100644 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference @@ -36,6 +36,14 @@ exception_code: 0 system.query_views_log +system.part_log +Row 1: +────── +database: default +table: async_insert_landing +partition_id: all +rows: 4 + system.query_log Row 1: ────── @@ -85,6 +93,14 @@ written_bytes: 0 status: QueryFinish exception_code: 0 +system.part_log +Row 1: +────── +database: default +table: async_insert_landing +partition_id: all +rows: 3 + system.query_log Row 1: ────── @@ -133,3 +149,11 @@ written_rows: 0 written_bytes: 0 status: Exc*****onWhileProcessing exception_code: 395 + +system.part_log +Row 1: +────── +database: default +table: async_insert_landing +partition_id: all +rows: 3 diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.sh b/tests/queries/0_stateless/02790_async_queries_in_query_log.sh index d1c8fe122cb9..1ff97031acba 100755 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.sh +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.sh @@ -28,7 +28,7 @@ function print_flush_query_logs() FROM system.query_log WHERE event_date >= yesterday() - AND initial_query_id = (SELECT flush_query_id FROM system.asynchronous_insert_log WHERE query_id = '$1') + AND initial_query_id = (SELECT flush_query_id FROM system.asynchronous_insert_log WHERE event_date >= yesterday() AND query_id = '$1') -- AND current_database = currentDatabase() -- Just to silence style check: this is not ok for this test since the query uses default values ORDER BY type DESC FORMAT Vertical" @@ -50,7 +50,21 @@ function print_flush_query_logs() FROM system.query_views_log WHERE event_date >= yesterday() - AND initial_query_id = (SELECT flush_query_id FROM system.asynchronous_insert_log WHERE query_id = '$1') + AND initial_query_id = (SELECT flush_query_id FROM system.asynchronous_insert_log WHERE event_date >= yesterday() AND query_id = '$1') + FORMAT Vertical" + + echo "" + echo "system.part_log" + ${CLICKHOUSE_CLIENT} -q " + SELECT + database, + table, + partition_id, + rows + FROM system.part_log + WHERE + event_date >= yesterday() + AND query_id = (SELECT flush_query_id FROM system.asynchronous_insert_log WHERE event_date >= yesterday() AND query_id = '$1') FORMAT Vertical" } From 11b5a3dd2249adf3cbb8fdd871a6713145c979ad Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Aug 2023 11:46:14 +0200 Subject: [PATCH 035/101] Update TableFunctionRemote.cpp --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index d00cda2462f7..1f42ce4ba30c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -223,7 +223,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (arg_num < args.size()) { if (sharding_key) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arguments `user` and `password` should be string literals (in single quotes)"); sharding_key = args[arg_num]; ++arg_num; } From 8e6f7b4d0bf554abe2f79b4d442fbe871755b307 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 7 Aug 2023 12:08:31 -0400 Subject: [PATCH 036/101] Fix typo, fix tests, `touch` command. --- .../utilities/clickhouse-keeper-client.md | 5 +- programs/keeper-client/Commands.cpp | 19 +++++- programs/keeper-client/Commands.h | 17 ++++- programs/keeper-client/KeeperClient.cpp | 3 +- tests/integration/test_keeper_client/test.py | 65 ++++++++++--------- 5 files changed, 69 insertions(+), 40 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 37eb0bb71ff5..443bf19de4c8 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -45,7 +45,8 @@ keeper foo bar - `ls [path]` -- Lists the nodes for the given path (default: cwd) - `cd [path]` -- Change the working path (default `.`) - `set [version]` -- Updates the node's value. Only update if version matches (default: -1) -- `create ` -- Creates new node +- `create [mode]` -- Creates new node with the set value +- `touch ` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists - `get ` -- Returns the node's value - `remove ` -- Remove the node - `rmr ` -- Recursively deletes path. Confirmation required @@ -53,5 +54,5 @@ keeper foo bar - `help` -- Prints this message - `get_stat [path]` -- Returns the node's stat (default `.`) - `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) -- `delete_stable_backups` -- Deletes ClickHouse nodes used for backups that are now inactive +- `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive - `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index fd0a00d59db3..b60b543ffce9 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -126,6 +126,21 @@ void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) static_cast(query->args[2].safeGet())); } +bool TouchCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void TouchCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + client->zookeeper->createIfNotExists(client->getAbsolutePath(query->args[0].safeGet()), ""); +} + bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String arg; @@ -215,12 +230,12 @@ void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client } } -bool DeleteStableBackups::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const +bool DeleteStaleBackups::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const { return true; } -void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const +void DeleteStaleBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const { client->askConfirmation( "You are going to delete all inactive backups in /clickhouse/backups.", diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 093920cb10de..7d64fef17386 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -76,7 +76,18 @@ class CreateCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Creates new node"; } + String getHelpMessage() const override { return "{} [mode] -- Creates new node with the set value"; } +}; + +class TouchCommand : public IKeeperClientCommand +{ + String getName() const override { return "touch"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "{} -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists"; } }; class GetCommand : public IKeeperClientCommand @@ -115,9 +126,9 @@ class FindSuperNodes : public IKeeperClientCommand } }; -class DeleteStableBackups : public IKeeperClientCommand +class DeleteStaleBackups : public IKeeperClientCommand { - String getName() const override { return "delete_stable_backups"; } + String getName() const override { return "delete_stale_backups"; } bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 561a1f41f7af..22d17c7a263c 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -176,10 +176,11 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), - std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 3187ce10d2a5..8f7056a5afd6 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -42,17 +42,17 @@ def keeper_query(query: str): def test_big_family(): command = keeper_query( - "create test_big_family foo;" - "create test_big_family/1 foo;" - "create test_big_family/1/1 foo;" - "create test_big_family/1/2 foo;" - "create test_big_family/1/3 foo;" - "create test_big_family/1/4 foo;" - "create test_big_family/1/5 foo;" - "create test_big_family/2 foo;" - "create test_big_family/2/1 foo;" - "create test_big_family/2/2 foo;" - "create test_big_family/2/3 foo;" + "touch test_big_family;" + "touch test_big_family/1;" + "touch test_big_family/1/1;" + "touch test_big_family/1/2;" + "touch test_big_family/1/3;" + "touch test_big_family/1/4;" + "touch test_big_family/1/5;" + "touch test_big_family/2;" + "touch test_big_family/2/1;" + "touch test_big_family/2/2;" + "touch test_big_family/2/3;" "find_big_family test_big_family;" ) @@ -82,18 +82,18 @@ def test_big_family(): def test_find_super_nodes(): command = keeper_query( - "create test_find_super_nodes foo;" - "create test_find_super_nodes/1 foo;" - "create test_find_super_nodes/1/1 foo;" - "create test_find_super_nodes/1/2 foo;" - "create test_find_super_nodes/1/3 foo;" - "create test_find_super_nodes/1/4 foo;" - "create test_find_super_nodes/1/5 foo;" - "create test_find_super_nodes/2 foo;" - "create test_find_super_nodes/2/1 foo;" - "create test_find_super_nodes/2/2 foo;" - "create test_find_super_nodes/2/3 foo;" - "create test_find_super_nodes/2/4 foo;" + "touch test_find_super_nodes;" + "touch test_find_super_nodes/1;" + "touch test_find_super_nodes/1/1;" + "touch test_find_super_nodes/1/2;" + "touch test_find_super_nodes/1/3;" + "touch test_find_super_nodes/1/4;" + "touch test_find_super_nodes/1/5;" + "touch test_find_super_nodes/2;" + "touch test_find_super_nodes/2/1;" + "touch test_find_super_nodes/2/2;" + "touch test_find_super_nodes/2/3;" + "touch test_find_super_nodes/2/4;" "cd test_find_super_nodes;" "find_super_nodes 4;" ) @@ -106,16 +106,17 @@ def test_find_super_nodes(): ) -def test_delete_stable_backups(): +def test_delete_stale_backups(): command = keeper_query( - "create /clickhouse/backups foo;" - "create /clickhouse/backups/1 foo;" - "create /clickhouse/backups/1/stage foo;" - "create /clickhouse/backups/1/stage/alive123 foo;" - "create /clickhouse/backups/2 foo;" - "create /clickhouse/backups/2/stage foo;" - "create /clickhouse/backups/2/stage/dead123 foo;" - "delete_stable_backups;" + "touch /clickhouse;" + "touch /clickhouse/backups;" + "touch /clickhouse/backups/1;" + "touch /clickhouse/backups/1/stage;" + "touch /clickhouse/backups/1/stage/alive123;" + "touch /clickhouse/backups/2;" + "touch /clickhouse/backups/2/stage;" + "touch /clickhouse/backups/2/stage/dead123;" + "delete_stale_backups;" "y;" "ls clickhouse/backups;" ) From 473fb60e6fb4e27aa712435482bf49109ae3d64f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 7 Aug 2023 18:56:18 +0200 Subject: [PATCH 037/101] Added retry for TransportException for azure read & download --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 48 +++++++++++++------ 1 file changed, 34 insertions(+), 14 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 129bb97be097..15b632a54d43 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -102,6 +102,19 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() size_t bytes_read = 0; size_t sleep_time_with_backoff_milliseconds = 100; + + auto handle_exception = [&, this](const auto & e, size_t i) + { + LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); + if (i + 1 == max_single_read_retries) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + initialized = false; + initialize(); + }; + for (size_t i = 0; i < max_single_read_retries; ++i) { try @@ -111,16 +124,13 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); break; } + catch (const Azure::Core::Http::TransportException & e) + { + handle_exception(e, i); + } catch (const Azure::Storage::StorageException & e) { - LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); - if (i + 1 == max_single_read_retries) - throw; - - sleepForMilliseconds(sleep_time_with_backoff_milliseconds); - sleep_time_with_backoff_milliseconds *= 2; - initialized = false; - initialize(); + handle_exception(e, i); } } @@ -211,6 +221,17 @@ void ReadBufferFromAzureBlobStorage::initialize() blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); size_t sleep_time_with_backoff_milliseconds = 100; + + auto handle_exception = [&, this](const auto & e, size_t i) + { + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); + if (i + 1 == max_single_download_retries) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + }; + for (size_t i = 0; i < max_single_download_retries; ++i) { try @@ -219,14 +240,13 @@ void ReadBufferFromAzureBlobStorage::initialize() data_stream = std::move(download_response.Value.BodyStream); break; } + catch (const Azure::Core::Http::TransportException & e) + { + handle_exception(e, i); + } catch (const Azure::Core::RequestFailedException & e) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); - if (i + 1 == max_single_download_retries) - throw; - - sleepForMilliseconds(sleep_time_with_backoff_milliseconds); - sleep_time_with_backoff_milliseconds *= 2; + handle_exception(e,i); } } From 109c8d8be6b3fdac2293d0a6e01b31882fc006d7 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 7 Aug 2023 13:35:35 -0400 Subject: [PATCH 038/101] Add no-parallel tag to tests --- tests/queries/0_stateless/01293_create_role.sql | 2 ++ tests/queries/0_stateless/01294_create_settings_profile.sql | 2 ++ tests/queries/0_stateless/01295_create_row_policy.sql | 2 ++ 3 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01293_create_role.sql b/tests/queries/0_stateless/01293_create_role.sql index f22edfeec3ef..fd75d62964de 100644 --- a/tests/queries/0_stateless/01293_create_role.sql +++ b/tests/queries/0_stateless/01293_create_role.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP ROLE IF EXISTS r1_01293, r2_01293, r3_01293, r4_01293, r5_01293, r6_01293, r7_01293, r8_01293, r9_01293; DROP ROLE IF EXISTS r2_01293_renamed; DROP ROLE IF EXISTS r1_01293@'%', 'r2_01293@%.myhost.com'; diff --git a/tests/queries/0_stateless/01294_create_settings_profile.sql b/tests/queries/0_stateless/01294_create_settings_profile.sql index 565b4e70367b..f71eefa6975f 100644 --- a/tests/queries/0_stateless/01294_create_settings_profile.sql +++ b/tests/queries/0_stateless/01294_create_settings_profile.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP SETTINGS PROFILE IF EXISTS s1_01294, s2_01294, s3_01294, s4_01294, s5_01294, s6_01294, s7_01294, s8_01294, s9_01294, s10_01294; DROP SETTINGS PROFILE IF EXISTS s2_01294_renamed; DROP USER IF EXISTS u1_01294; diff --git a/tests/queries/0_stateless/01295_create_row_policy.sql b/tests/queries/0_stateless/01295_create_row_policy.sql index b484d0ea0f30..5ccd815c89a3 100644 --- a/tests/queries/0_stateless/01295_create_row_policy.sql +++ b/tests/queries/0_stateless/01295_create_row_policy.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP ROW POLICY IF EXISTS p1_01295, p2_01295, p3_01295, p4_01295, p5_01295, p6_01295, p7_01295, p8_01295, p9_01295, p10_01295 ON db.table; DROP ROW POLICY IF EXISTS p2_01295_renamed ON db.table; DROP ROW POLICY IF EXISTS p3_01295 ON db.table, db2.table2; From 5dd99db3692f9e2fb7264275c43385e274467674 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 8 Aug 2023 03:55:06 +0200 Subject: [PATCH 039/101] Add diagnostic info about file name during schema inference --- src/Formats/ReadSchemaUtils.cpp | 16 ++++++++++++---- src/Processors/Formats/IRowInputFormat.cpp | 16 +++++++++++++++- ...le_diagnostics_while_reading_header.reference | 2 ++ ...2836_file_diagnostics_while_reading_header.sh | 11 +++++++++++ 4 files changed, 40 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.reference create mode 100755 tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.sh diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index cbfa05bfba64..d183442ba44a 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -1,13 +1,11 @@ -#include -#include #include -#include -#include #include #include #include #include #include +#include + namespace DB { @@ -53,6 +51,7 @@ ColumnsDescription readSchemaFromFormat( bool retry, ContextPtr & context, std::unique_ptr & buf) +try { NamesAndTypesList names_and_types; if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) @@ -209,12 +208,21 @@ ColumnsDescription readSchemaFromFormat( ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference. You must specify the structure manually", format_name); + /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and further processing can fail with an exception. Let's just remove columns with empty names from the structure. names_and_types.erase( std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); return ColumnsDescription(names_and_types); } +catch (Exception & e) +{ + auto file_name = getFileNameFromReadBuffer(*buf); + if (!file_name.empty()) + e.addMessage(fmt::format("(in file/uri {})", file_name)); + throw; +} + ColumnsDescription readSchemaFromFormat( const String & format_name, diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 8bcf293033be..7395d9b916ad 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -86,7 +86,21 @@ void IRowInputFormat::logError() Chunk IRowInputFormat::generate() { if (total_rows == 0) - readPrefix(); + { + try + { + readPrefix(); + } + catch (Exception & e) + { + auto file_name = getFileNameFromReadBuffer(getReadBuffer()); + if (!file_name.empty()) + e.addMessage(fmt::format("(in file/uri {})", file_name)); + + e.addMessage("(while reading header)"); + throw; + } + } const Block & header = getPort().getHeader(); diff --git a/tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.reference b/tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.reference new file mode 100644 index 000000000000..6829f972684f --- /dev/null +++ b/tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.reference @@ -0,0 +1,2 @@ +in file/uri +test.csv diff --git a/tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.sh b/tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.sh new file mode 100755 index 000000000000..d1b5ffa2af8f --- /dev/null +++ b/tests/queries/0_stateless/02836_file_diagnostics_while_reading_header.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +FILENAME="${CLICKHOUSE_TMP}/test.csv" + +printf 'Bad\rHeader\n123\n' > "${FILENAME}" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM file('${CLICKHOUSE_TMP}/t*e*s*t.csv')" 2>&1 | grep -o -P 'in file/uri|test\.csv' +rm "${FILENAME}" From ede710ce83f269cd05d33ec79ee66bc6d686fbd0 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Tue, 8 Aug 2023 16:38:16 +1000 Subject: [PATCH 040/101] Small fix for HTTPHeaderFilter --- src/Common/HTTPHeaderFilter.cpp | 3 +++ tests/queries/0_stateless/02752_forbidden_headers.sql | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/src/Common/HTTPHeaderFilter.cpp b/src/Common/HTTPHeaderFilter.cpp index 622bc475d33b..c6e5675303de 100644 --- a/src/Common/HTTPHeaderFilter.cpp +++ b/src/Common/HTTPHeaderFilter.cpp @@ -18,6 +18,9 @@ void HTTPHeaderFilter::checkHeaders(const HTTPHeaderEntries & entries) const for (const auto & entry : entries) { + if (entry.name.contains('\n') || entry.value.contains('\n')) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP header \"{}\" has invalid character", entry.name); + if (forbidden_headers.contains(entry.name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP header \"{}\" is forbidden in configuration file, " "see ", entry.name); diff --git a/tests/queries/0_stateless/02752_forbidden_headers.sql b/tests/queries/0_stateless/02752_forbidden_headers.sql index d966fc0a187f..dd0dcb53b160 100644 --- a/tests/queries/0_stateless/02752_forbidden_headers.sql +++ b/tests/queries/0_stateless/02752_forbidden_headers.sql @@ -3,16 +3,24 @@ SELECT * FROM url('http://localhost:8123/', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM url('http://localhost:8123/', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM url('http://localhost:8123/', LineAsString, headers('bad_header_name: test\nexact_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM url('http://localhost:8123/', LineAsString, headers('bad_header_value' = 'test\nexact_header: value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM url('http://localhost:8123/', LineAsString, headers('random_header' = 'value')) FORMAT Null; SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('bad_header_name: test\nexact_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('bad_header_value' = 'test\nexact_header: value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('random_header' = 'value')) FORMAT Null; SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('bad_header_name: test\nexact_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('bad_header_value' = 'test\nexact_header: value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('random_header' = 'value')); -- { serverError S3_ERROR } SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('bad_header_name: test\nexact_header' = 'value')); -- { serverError BAD_ARGUMENTS } +SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('bad_header_value' = 'test\nexact_header: value')); -- { serverError BAD_ARGUMENTS } SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('random_header' = 'value')); -- { serverError S3_ERROR } From 4f6f8fce652adae458a766f76af153d20501b864 Mon Sep 17 00:00:00 2001 From: San Date: Tue, 8 Aug 2023 17:13:43 +1000 Subject: [PATCH 041/101] Update src/Common/HTTPHeaderFilter.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Common/HTTPHeaderFilter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/HTTPHeaderFilter.cpp b/src/Common/HTTPHeaderFilter.cpp index c6e5675303de..edd549f8b7de 100644 --- a/src/Common/HTTPHeaderFilter.cpp +++ b/src/Common/HTTPHeaderFilter.cpp @@ -20,7 +20,7 @@ void HTTPHeaderFilter::checkHeaders(const HTTPHeaderEntries & entries) const { if (entry.name.contains('\n') || entry.value.contains('\n')) throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP header \"{}\" has invalid character", entry.name); - + if (forbidden_headers.contains(entry.name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP header \"{}\" is forbidden in configuration file, " "see ", entry.name); From a882ef295f5fdad5a805aa9ed6761ba1129ff081 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 28 Jul 2023 16:00:01 +0000 Subject: [PATCH 042/101] Query CHECK TABLE takes care about progress and cancellation --- src/Common/FileChecker.cpp | 22 ++- src/Common/FileChecker.h | 1 + src/Interpreters/InterpreterCheckQuery.cpp | 168 +++++++++++++++----- src/Storages/CheckResults.h | 5 + src/Storages/IStorage.cpp | 8 + src/Storages/IStorage.h | 4 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMergeTree.cpp | 20 ++- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageProxy.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 13 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 5 +- src/Storages/StorageStripeLog.h | 2 +- 15 files changed, 194 insertions(+), 67 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 876bc4e641cd..dc3800eb29af 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -84,10 +84,20 @@ size_t FileChecker::getTotalSize() const CheckResults FileChecker::check() const { - if (map.empty()) - return {}; - CheckResults results; + auto callback = [&results](const CheckResult & result, size_t) -> bool + { + results.push_back(result); + return true; + }; + check(callback); + return results; +} + +void FileChecker::check(CheckDataCallback check_callback) const +{ + if (map.empty()) + return; for (const auto & name_size : map) { @@ -101,14 +111,12 @@ CheckResults FileChecker::check() const 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); + check_callback(CheckResult(name, false, failure_message), map.size()); break; } - results.emplace_back(name, true, ""); + check_callback(CheckResult(name, true, ""), map.size()); } - - return results; } void FileChecker::repair() diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index bb0383e4b563..28dc17a4610a 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -29,6 +29,7 @@ class FileChecker /// Check the files whose parameters are specified in sizes.json CheckResults check() const; + void check(CheckDataCallback check_callback) const; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 333aed848734..00d2749eaf93 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -17,63 +18,158 @@ namespace DB namespace { -NamesAndTypes getBlockStructure() +Block getBlockFromCheckResult(const CheckResults & check_results, bool check_query_single_value_result) { - return { + if (check_query_single_value_result) + { + bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); + return Block{{ColumnUInt8::create(1, static_cast(result)), std::make_shared(), "result"}}; + } + + NamesAndTypes block_structure = NamesAndTypes{ {"part_path", std::make_shared()}, {"is_passed", std::make_shared()}, {"message", std::make_shared()}, }; -} - -} + 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(); + 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); + } -InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) -{ + return 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 InterpreterCheckQuery::execute() +class TableCheckResultSource : public ISource { - const auto & check = query_ptr->as(); - auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary); - - getContext()->checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); - auto check_results = table->checkData(query_ptr, getContext()); - Block block; - if (getContext()->getSettingsRef().check_query_single_value_result) +public: + explicit TableCheckResultSource(const ASTPtr & query_ptr_, StoragePtr table_, bool check_query_single_value_result_, ContextPtr context_) + : ISource(getBlockFromCheckResult({}, check_query_single_value_result_).cloneEmpty()) + , query_ptr(query_ptr_) + , table(table_) + , context(context_) + , check_query_single_value_result(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"}}; + worker_result = std::async(std::launch::async, [this]{ worker(); }); } - else + + String getName() const override { return "TableCheckResultSource"; } + +protected: + + std::optional tryGenerate() override { - 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(); - for (const auto & check_result : check_results) + if (is_check_completed) + return {}; + + auto status = worker_result.wait_for(std::chrono::milliseconds(100)); + is_check_completed = (status == std::future_status::ready); + + if (is_check_completed) { - path_column->insert(check_result.fs_path); - is_passed_column->insert(static_cast(check_result.success)); - message_column->insert(check_result.failure_message); + worker_result.get(); + auto result_block = getBlockFromCheckResult(check_results, check_query_single_value_result); + check_results.clear(); + return Chunk(result_block.getColumns(), result_block.rows()); } - 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}}); + std::lock_guard lock(mutex); + progress(progress_rows, 0); + progress_rows = 0; + + if (check_query_single_value_result || check_results.empty()) + { + return Chunk(); + } + + auto result_block = getBlockFromCheckResult(check_results, check_query_single_value_result); + check_results.clear(); + return Chunk(result_block.getColumns(), result_block.rows()); } - BlockIO res; - res.pipeline = QueryPipeline(std::make_shared(std::move(block))); +private: + void worker() + { + table->checkData(query_ptr, context, + [this](const CheckResult & check_result, size_t new_total_rows) + { + if (isCancelled()) + return false; + + std::lock_guard lock(mutex); + if (new_total_rows > total_rows) + { + addTotalRowsApprox(new_total_rows - total_rows); + total_rows = new_total_rows; + } + progress_rows++; + + 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); + } + + check_results.push_back(check_result); + + bool should_continue = check_result.success || !check_query_single_value_result; + return should_continue; + }); + } + + ASTPtr query_ptr; + StoragePtr table; + ContextPtr context; + bool check_query_single_value_result; + + std::future worker_result; + + std::mutex mutex; + CheckResults check_results; + size_t progress_rows = 0; + size_t total_rows = 0; + + bool is_check_completed = false; +}; + +} + +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); + + context->checkAccess(AccessType::SHOW_TABLES, table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + + BlockIO res; + { + bool check_query_single_value_result = context->getSettingsRef().check_query_single_value_result; + auto result_source = std::make_shared(query_ptr, table, check_query_single_value_result, context); + res.pipeline = QueryPipeline(result_source); + } return res; } diff --git a/src/Storages/CheckResults.h b/src/Storages/CheckResults.h index b342b014fa47..b84262b50c98 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -22,6 +22,11 @@ struct CheckResult {} }; +/// Process single result of checkData +/// Second argument is an estimated number of check results +/// Return true to continue checking, false to stop +using CheckDataCallback = std::function; + using CheckResults = std::vector; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ae7659e074f5..467a34ca8553 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -273,6 +273,14 @@ bool IStorage::isStaticStorage() const return false; } +CheckResults IStorage::checkData(const ASTPtr & query, ContextPtr context) +{ + CheckResults results; + auto callback = [&](const CheckResult & result, size_t) { results.push_back(result); return true;}; + checkData(query, context, callback); + return results; +} + void IStorage::adjustCreateQueryForBackup(ASTPtr &) const { } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ec92f57aeda5..48ebf6b1242a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -596,7 +596,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; } /// 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()); } + virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */); + + virtual void checkData(const ASTPtr & /* query */, ContextPtr /* context */, CheckDataCallback /* callback */) { 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 87aa71f3e8d5..bb58e05d756c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -866,13 +866,13 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) +void StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context, CheckDataCallback check_callback) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return file_checker.check(); + file_checker.check(check_callback); } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index f1d05ed39ac2..f2539a6c48cb 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,7 +59,7 @@ class StorageLog final : public IStorage, public WithMutableContext void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override; + void checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad9013d9f131..4754412a0dec 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2197,9 +2197,8 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } -CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) +void StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) { - CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -2224,12 +2223,16 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings()); part->checkMetadata(); - results.emplace_back(part->name, true, "Checksums recounted and written to disk."); + bool should_continue = check_callback(CheckResult(part->name, true, "Checksums recounted and written to disk."), data_parts.size()); + if (!should_continue) + break; } catch (const Exception & ex) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + bool should_continue = check_callback(CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"), data_parts.size()); + if (!should_continue) + break; } } else @@ -2238,15 +2241,18 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ { checkDataPart(part, true); part->checkMetadata(); - results.emplace_back(part->name, true, ""); + bool should_continue = check_callback(CheckResult(part->name, true, ""), data_parts.size()); + if (!should_continue) + break; } catch (const Exception & ex) { - results.emplace_back(part->name, false, ex.message()); + bool should_continue = check_callback(CheckResult(part->name, false, ex.message()), data_parts.size()); + if (!should_continue) + break; } } } - return results; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c77e5140d75e..6b1b1b7b4154 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,7 +108,7 @@ class StorageMergeTree final : public MergeTreeData void onActionLockRemove(StorageActionBlockType action_type) override; - CheckResults checkData(const ASTPtr & query, ContextPtr context) override; + void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 21ed4b91c621..5339656782ec 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,7 +149,8 @@ class StorageProxy : public IStorage return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } + void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override { getNested()->checkData(query, context, check_callback); } + 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 7fce373e26bc..71fbd009d705 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8481,9 +8481,8 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t part_check_thread.enqueuePart(part_name, delay_to_check_seconds); } -CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) +void StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) { - CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -8500,17 +8499,19 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context { try { - results.push_back(part_check_thread.checkPartAndFix(part->name)); + bool should_continue = check_callback(part_check_thread.checkPartAndFix(part->name), data_parts.size()); + if (!should_continue) + break; } catch (const Exception & ex) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + bool should_continue = check_callback(CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"), data_parts.size()); + if (!should_continue) + break; } } } - - return results; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 78ef39f032fc..6f9853122cbb 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,7 +230,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// 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); - CheckResults checkData(const ASTPtr & query, ContextPtr context) override; + void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0bfef5ed5e58..5a3a007035c0 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,14 +403,13 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot, std::move(lock)); } - -CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) +void StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context, CheckDataCallback check_callback) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return file_checker.check(); + file_checker.check(check_callback); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index f889a1de71bc..61ce49f43da8 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,7 +53,7 @@ friend class StripeLogSink; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override; + void checkData(const ASTPtr & query, ContextPtr ocal_context, CheckDataCallback check_callback) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } From 1aedc4e89292b39d29367fc41b3485c893c08be3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Aug 2023 10:57:59 +0000 Subject: [PATCH 043/101] Build proper pipeline for CHECK TABLE --- src/Common/FileChecker.cpp | 48 ++--- src/Common/FileChecker.h | 38 +++- src/Interpreters/InterpreterCheckQuery.cpp | 219 +++++++++++--------- src/Storages/CheckResults.h | 7 - src/Storages/IStorage.cpp | 13 +- src/Storages/IStorage.h | 41 +++- src/Storages/StorageLog.cpp | 9 +- src/Storages/StorageLog.h | 16 +- src/Storages/StorageMergeTree.cpp | 32 +-- src/Storages/StorageMergeTree.h | 29 ++- src/Storages/StorageProxy.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 38 ++-- src/Storages/StorageReplicatedMergeTree.h | 31 ++- src/Storages/StorageStripeLog.cpp | 9 +- src/Storages/StorageStripeLog.h | 17 +- 15 files changed, 371 insertions(+), 179 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index dc3800eb29af..122ea83835da 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -82,41 +82,35 @@ size_t FileChecker::getTotalSize() const } -CheckResults FileChecker::check() const +FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks() { - CheckResults results; - auto callback = [&results](const CheckResult & result, size_t) -> bool - { - results.push_back(result); - return true; - }; - check(callback); - return results; + return std::make_unique(map); } -void FileChecker::check(CheckDataCallback check_callback) const +CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const { - if (map.empty()) - return; - - for (const auto & name_size : map) + String name; + size_t expected_size; + bool is_finished = check_data_tasks->next(name, expected_size); + if (is_finished) { - 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. + has_nothing_to_do = true; + return {}; + } - 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"); - check_callback(CheckResult(name, false, failure_message), map.size()); - break; - } + 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. - check_callback(CheckResult(name, true, ""), map.size()); + if (real_size != expected_size) + { + 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); } + + return CheckResult(name, true, ""); } void FileChecker::repair() diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 28dc17a4610a..8ffc310b84dc 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -28,8 +29,11 @@ class FileChecker bool empty() const { return map.empty(); } /// Check the files whose parameters are specified in sizes.json - CheckResults check() const; - void check(CheckDataCallback check_callback) const; + /// 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; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. @@ -42,6 +46,36 @@ class FileChecker /// 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 00d2749eaf93..113a5d9fc90b 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -11,139 +11,133 @@ #include #include +#include + +#include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { -Block getBlockFromCheckResult(const CheckResults & check_results, bool check_query_single_value_result) +Block getSingleValueBlock(UInt8 value) { - if (check_query_single_value_result) - { - bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); - return Block{{ColumnUInt8::create(1, static_cast(result)), std::make_shared(), "result"}}; - } + return Block{{ColumnUInt8::create(1, value), std::make_shared(), "result"}}; +} - NamesAndTypes block_structure = NamesAndTypes{ +Block getHeaderForCheckResult() +{ + auto names_and_types = NamesAndTypes{ {"part_path", std::make_shared()}, {"is_passed", std::make_shared()}, {"message", std::make_shared()}, }; - 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(); - - 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); - } return 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}, + {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}, }); } -class TableCheckResultSource : public ISource +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: - explicit TableCheckResultSource(const ASTPtr & query_ptr_, StoragePtr table_, bool check_query_single_value_result_, ContextPtr context_) - : ISource(getBlockFromCheckResult({}, check_query_single_value_result_).cloneEmpty()) - , query_ptr(query_ptr_) + TableCheckWorkerProcessor(IStorage::DataValidationTasksPtr check_data_tasks_, StoragePtr table_) + : ISource(getHeaderForCheckResult()) , table(table_) - , context(context_) - , check_query_single_value_result(check_query_single_value_result_) + , check_data_tasks(check_data_tasks_) { - worker_result = std::async(std::launch::async, [this]{ worker(); }); } - String getName() const override { return "TableCheckResultSource"; } + String getName() const override { return "TableCheckWorkerProcessor"; } protected: std::optional tryGenerate() override { - - if (is_check_completed) + bool has_nothing_to_do = false; + auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do); + if (has_nothing_to_do) return {}; - auto status = worker_result.wait_for(std::chrono::milliseconds(100)); - is_check_completed = (status == std::future_status::ready); + /// 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 (is_check_completed) + if (!check_result.success) { - worker_result.get(); - auto result_block = getBlockFromCheckResult(check_results, check_query_single_value_result); - check_results.clear(); - return Chunk(result_block.getColumns(), result_block.rows()); + LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"), + "Check query for table {} failed, path {}, reason: {}", + table->getStorageID().getNameForLogs(), + check_result.fs_path, + check_result.failure_message); } - std::lock_guard lock(mutex); - progress(progress_rows, 0); - progress_rows = 0; - - if (check_query_single_value_result || check_results.empty()) - { - return Chunk(); - } - - auto result_block = getBlockFromCheckResult(check_results, check_query_single_value_result); - check_results.clear(); - return Chunk(result_block.getColumns(), result_block.rows()); + return getChunkFromCheckResult(check_result); } private: - void worker() + 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 { - table->checkData(query_ptr, context, - [this](const CheckResult & check_result, size_t new_total_rows) - { - if (isCancelled()) - return false; - - std::lock_guard lock(mutex); - if (new_total_rows > total_rows) - { - addTotalRowsApprox(new_total_rows - total_rows); - total_rows = new_total_rows; - } - progress_rows++; - - 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); - } - - check_results.push_back(check_result); - - bool should_continue = check_result.success || !check_query_single_value_result; - return should_continue; - }); - } + if (result_value == 0) + return; - ASTPtr query_ptr; - StoragePtr table; - ContextPtr context; - bool check_query_single_value_result; + auto columns = chunk.getColumns(); + if (columns.size() != 3) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}", columns.size()); - std::future worker_result; + 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; + } + } + } - std::mutex mutex; - CheckResults check_results; - size_t progress_rows = 0; - size_t total_rows = 0; + Chunk generate() override + { + if (is_valuer_emitted.exchange(true)) + return {}; + auto block = getSingleValueBlock(result_value); + return Chunk(block.getColumns(), block.rows()); + } - bool is_check_completed = false; +private: + std::atomic result_value{1}; + std::atomic_bool is_valuer_emitted{false}; }; } @@ -154,7 +148,6 @@ InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextP { } - BlockIO InterpreterCheckQuery::execute() { const auto & check = query_ptr->as(); @@ -164,11 +157,51 @@ BlockIO InterpreterCheckQuery::execute() context->checkAccess(AccessType::SHOW_TABLES, table_id); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + auto check_data_tasks = table->getCheckTaskList(query_ptr, context); + + const auto & settings = context->getSettingsRef(); + BlockIO res; { - bool check_query_single_value_result = context->getSettingsRef().check_query_single_value_result; - auto result_source = std::make_shared(query_ptr, table, check_query_single_value_result, context); - res.pipeline = QueryPipeline(result_source); + auto processors = std::make_shared(); + + 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 b84262b50c98..2e4652fea293 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -22,11 +22,4 @@ struct CheckResult {} }; -/// Process single result of checkData -/// Second argument is an estimated number of check results -/// Return true to continue checking, false to stop -using CheckDataCallback = std::function; - -using CheckResults = std::vector; - } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 467a34ca8553..85299f63165a 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -273,12 +273,15 @@ bool IStorage::isStaticStorage() const return false; } -CheckResults IStorage::checkData(const ASTPtr & query, ContextPtr context) +IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */) { - CheckResults results; - auto callback = [&](const CheckResult & result, size_t) { results.push_back(result); return true;}; - checkData(query, context, callback); - return results; + 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 48ebf6b1242a..b1e20c557821 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -595,10 +595,45 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// 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; } - /// Checks validity of the data - virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */); - virtual void checkData(const ASTPtr & /* query */, ContextPtr /* context */, CheckDataCallback /* callback */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); } + /** 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 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 bb58e05d756c..6fb786927258 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -866,15 +866,18 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -void StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context, CheckDataCallback check_callback) +IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - - file_checker.check(check_callback); + return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } +CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + return file_checker.checkNextEntry(static_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 f2539a6c48cb..95f95088aa2e 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,7 +59,8 @@ class StorageLog final : public IStorage, public WithMutableContext void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - void checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; @@ -142,6 +143,19 @@ class StorageLog final : public IStorage, public WithMutableContext 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 4754412a0dec..4593ed6c84a3 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2197,7 +2197,7 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } -void StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) +IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) { DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) @@ -2208,7 +2208,14 @@ void StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, else data_parts = getVisibleDataPartsVector(local_context); - for (auto & part : data_parts) + return std::make_unique(std::move(data_parts), local_context); +} + +CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + auto local_context = static_cast(check_task_list.get())->context; + + if (auto part = static_cast(check_task_list.get())->next()) { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; @@ -2223,16 +2230,12 @@ void StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings()); part->checkMetadata(); - bool should_continue = check_callback(CheckResult(part->name, true, "Checksums recounted and written to disk."), data_parts.size()); - if (!should_continue) - break; + return CheckResult(part->name, true, "Checksums recounted and written to disk."); } catch (const Exception & ex) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - bool should_continue = check_callback(CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"), data_parts.size()); - if (!should_continue) - break; + return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } else @@ -2241,18 +2244,19 @@ void StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, { checkDataPart(part, true); part->checkMetadata(); - bool should_continue = check_callback(CheckResult(part->name, true, ""), data_parts.size()); - if (!should_continue) - break; + return CheckResult(part->name, true, ""); } catch (const Exception & ex) { - bool should_continue = check_callback(CheckResult(part->name, false, ex.message()), data_parts.size()); - if (!should_continue) - break; + return CheckResult(part->name, false, ex.message()); } } } + else + { + has_nothing_to_do = true; + return {}; + } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6b1b1b7b4154..ec4796e49410 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,7 +108,8 @@ class StorageMergeTree final : public MergeTreeData void onActionLockRemove(StorageActionBlockType action_type) override; - void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; @@ -278,6 +279,32 @@ class StorageMergeTree final : public MergeTreeData 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 5339656782ec..ea908bea0328 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,7 +149,8 @@ class StorageProxy : public IStorage return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override { getNested()->checkData(query, context, check_callback); } + 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); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 71fbd009d705..b04d824e323d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8481,7 +8481,7 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t part_check_thread.enqueuePart(part_name, delay_to_check_seconds); } -void StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) +IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) { DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) @@ -8492,26 +8492,30 @@ void StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr loca else data_parts = getVisibleDataPartsVector(local_context); - { - auto part_check_lock = part_check_thread.pausePartsCheck(); + 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) +{ - for (auto & part : data_parts) + if (auto part = static_cast(check_task_list.get())->next()) + { + try { - try - { - bool should_continue = check_callback(part_check_thread.checkPartAndFix(part->name), data_parts.size()); - if (!should_continue) - break; - } - catch (const Exception & ex) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - bool should_continue = check_callback(CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"), data_parts.size()); - if (!should_continue) - break; - } + 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() + "'"); } } + else + { + has_nothing_to_do = true; + return {}; + } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 6f9853122cbb..2bc18aa3b0a3 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,7 +230,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// 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); - void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; @@ -990,6 +991,34 @@ class StorageReplicatedMergeTree final : public MergeTreeData 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 5a3a007035c0..c242830e657c 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,15 +403,18 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -void StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context, CheckDataCallback check_callback) +IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - - file_checker.check(check_callback); + return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } +CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + return file_checker.checkNextEntry(static_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 61ce49f43da8..5d4e2fcbd3ab 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,7 +53,8 @@ friend class StripeLogSink; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - void checkData(const ASTPtr & query, ContextPtr ocal_context, CheckDataCallback check_callback) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } @@ -93,6 +94,20 @@ friend class StripeLogSink; 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; From 11554fe9c0efae873fb112fbde718cd33217f4ed Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Aug 2023 12:39:26 +0000 Subject: [PATCH 044/101] Upd tests for CHECK TABLE --- .../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 +++++++++++++++++++ 7 files changed, 48 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02841_check_table_progress.reference create mode 100755 tests/queries/0_stateless/02841_check_table_progress.sh diff --git a/tests/queries/0_stateless/00063_check_query.reference b/tests/queries/0_stateless/00063_check_query.reference index 6ed281c757a9..e8183f05f5db 100644 --- a/tests/queries/0_stateless/00063_check_query.reference +++ b/tests/queries/0_stateless/00063_check_query.reference @@ -1,2 +1,3 @@ 1 1 +1 diff --git a/tests/queries/0_stateless/00063_check_query.sql b/tests/queries/0_stateless/00063_check_query.sql index e7362074a059..263cf94fb4af 100644 --- a/tests/queries/0_stateless/00063_check_query.sql +++ b/tests/queries/0_stateless/00063_check_query.sql @@ -8,6 +8,9 @@ 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 0e0b2c3b4837..079acc8cdbb2 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; +CHECK TABLE mt_table SETTINGS max_threads = 1; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 3, 'quick'), (toDate('2019-01-02'), 4, 'brown'); SELECT '========'; -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; OPTIMIZE TABLE mt_table FINAL; SELECT '========'; -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; SELECT '========'; INSERT INTO mt_table VALUES (toDate('2019-02-03'), 5, '!'), (toDate('2019-02-03'), 6, '?'); -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; 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; +CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; 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 b66aff8384d7..eccb2d258781 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; +CHECK TABLE check_query_test SETTINGS max_threads = 1; OPTIMIZE TABLE check_query_test; -CHECK TABLE check_query_test; +CHECK TABLE check_query_test SETTINGS max_threads = 1; 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; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 77; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; 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 0ac97404c46f..625be63e0c07 100644 --- a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql +++ b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql @@ -12,7 +12,6 @@ SELECT name, column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_02235' ORDER BY name, column; -SET check_query_single_value_result = 0; -CHECK TABLE t_sparse_02235; +CHECK TABLE t_sparse_02235 SETTINGS check_query_single_value_result = 0, max_threads = 1; 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 new file mode 100644 index 000000000000..541dab48def8 --- /dev/null +++ b/tests/queries/0_stateless/02841_check_table_progress.reference @@ -0,0 +1,2 @@ +Ok +Ok diff --git a/tests/queries/0_stateless/02841_check_table_progress.sh b/tests/queries/0_stateless/02841_check_table_progress.sh new file mode 100755 index 000000000000..166386b999b5 --- /dev/null +++ b/tests/queries/0_stateless/02841_check_table_progress.sh @@ -0,0 +1,29 @@ +#!/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 aeee50466cb237c0458c552a8e15d957dce013bc Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Aug 2023 12:39:49 +0000 Subject: [PATCH 045/101] Upd doc for CHECK TABLE --- .../sql-reference/statements/check-table.md | 97 ++++++++++++++----- 1 file changed, 75 insertions(+), 22 deletions(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 0209d59b018e..588be8417557 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -5,19 +5,38 @@ sidebar_label: CHECK TABLE title: "CHECK TABLE Statement" --- -Checks if the data in the table is corrupted. +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. -``` sql -CHECK TABLE [db.]name [PARTITION partition_expr] +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] ] ``` -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. +- `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 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. -- 0 - The data in the table is corrupted. -- 1 - The data maintains integrity. +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. The `CHECK TABLE` query supports the following table engines: @@ -26,39 +45,73 @@ 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 exception. +Performed over the tables with another table engines causes an `NOT_IMPLEMETED` 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. -## Checking the MergeTree Family Tables +## Examples -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. +By default `CHECK TABLE` query shows the general table check status: ```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 │ │ -└───────────┴───────────┴─────────┘ +┌─result─┐ +│ 1 │ +└────────┘ ``` -If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status. +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 -SET check_query_single_value_result = 1; -CHECK TABLE test_table; +CHECK TABLE t0 PARTITION ID '201003' +FORMAT PrettyCompactMonoBlock +SETTINGS check_query_single_value_result = 0 ``` +Output: + ```text -┌─result─┐ -│ 1 │ -└────────┘ +┌─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: From 358ef26385fd3b56233babae4b9ab335c16cf6dd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Aug 2023 09:22:00 +0000 Subject: [PATCH 046/101] small fixes for CHECK TABLE --- src/Interpreters/InterpreterCheckQuery.cpp | 6 +++--- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 113a5d9fc90b..bd530654dd2b 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -129,7 +129,7 @@ class TableCheckResultEmitter : public IAccumulatingTransform Chunk generate() override { - if (is_valuer_emitted.exchange(true)) + if (is_value_emitted.exchange(true)) return {}; auto block = getSingleValueBlock(result_value); return Chunk(block.getColumns(), block.rows()); @@ -137,7 +137,7 @@ class TableCheckResultEmitter : public IAccumulatingTransform private: std::atomic result_value{1}; - std::atomic_bool is_valuer_emitted{false}; + std::atomic_bool is_value_emitted{false}; }; } @@ -194,7 +194,7 @@ BlockIO InterpreterCheckQuery::execute() if (settings.check_query_single_value_result) { auto emitter_processor = std::make_shared(); - auto input_port = &emitter_processor->getInputPort(); + auto * input_port = &emitter_processor->getInputPort(); processors->emplace_back(emitter_processor); connect(*resize_outport, *input_port); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 6fb786927258..0e9f83e886a9 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -876,7 +876,7 @@ IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) { - return file_checker.checkNextEntry(static_cast(check_task_list.get())->file_checker_tasks, 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/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4593ed6c84a3..97fc7a6731f1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2213,9 +2213,9 @@ IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) { - auto local_context = static_cast(check_task_list.get())->context; - - if (auto part = static_cast(check_task_list.get())->next()) + auto * data_validation_tasks = assert_cast(check_task_list.get()); + auto local_context = data_validation_tasks->context; + if (auto part = data_validation_tasks->next()) { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b04d824e323d..c08f1ebcc485 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8499,7 +8499,7 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(co CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) { - if (auto part = static_cast(check_task_list.get())->next()) + if (auto part = assert_cast(check_task_list.get())->next()) { try { diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c242830e657c..a3cbff961998 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -413,7 +413,7 @@ IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) { - return file_checker.checkNextEntry(static_cast(check_task_list.get())->file_checker_tasks, 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 &) From 50715ca4fc2682e810ec7cc3d74212fec8098e49 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Aug 2023 11:57:11 +0000 Subject: [PATCH 047/101] fix doc links --- docs/en/sql-reference/statements/check-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 588be8417557..45fc17b044a3 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -117,6 +117,6 @@ If the checksums.txt file is missing, it can be restored. It will be recalculate 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](../../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`. +2. Set the [max_threads](/docs/en/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. From 1183dac293d5d474b8a9bbad34819d1ac14def44 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 8 Aug 2023 09:45:23 +0000 Subject: [PATCH 048/101] fix doc links --- docs/en/sql-reference/statements/check-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 45fc17b044a3..db8c32249efb 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -117,6 +117,6 @@ If the checksums.txt file is missing, it can be restored. It will be recalculate 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](/docs/en/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`. +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`. 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. From 263730820751e89a0f0ea76b9ba04a005f25c934 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 8 Aug 2023 19:54:07 +0000 Subject: [PATCH 049/101] Analyzer: fix test_system_flush_logs --- tests/analyzer_integration_broken_tests.txt | 1 - tests/integration/test_system_flush_logs/test.py | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 91e2f1aa5687..b9cca4bf6771 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -114,7 +114,6 @@ test_quota/test.py::test_tracking_quota test_quota/test.py::test_users_xml_is_readonly test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database test_profile_events_s3/test.py::test_profile_events -test_system_flush_logs/test.py::test_system_logs[system.text_log-0] test_user_defined_object_persistence/test.py::test_persistence test_settings_profile/test.py::test_show_profiles test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index bf225ac30f87..186dbc217c53 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -45,7 +45,8 @@ def test_system_logs(flush_logs, table, exists): if exists: node.query(q) else: - assert "Table {} doesn't exist".format(table) in node.query_and_get_error(q) + response = node.query_and_get_error(q) + assert "Table {} doesn't exist".format(table) in response or "Unknown table expression identifier '{}'".format(table) in response # Logic is tricky, let's check that there is no hang in case of message queue From 4b6ade18b2a7dd43e06ba838460a9b76d313af10 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 8 Aug 2023 20:06:59 +0000 Subject: [PATCH 050/101] Automatic style fix --- tests/integration/test_system_flush_logs/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 186dbc217c53..5bf3eff8b2d5 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -46,7 +46,10 @@ def test_system_logs(flush_logs, table, exists): node.query(q) else: response = node.query_and_get_error(q) - assert "Table {} doesn't exist".format(table) in response or "Unknown table expression identifier '{}'".format(table) in response + assert ( + "Table {} doesn't exist".format(table) in response + or "Unknown table expression identifier '{}'".format(table) in response + ) # Logic is tricky, let's check that there is no hang in case of message queue From 1ab99e5fdd2a52551a7f4f049a3feb06023079c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 8 Aug 2023 22:38:52 +0200 Subject: [PATCH 051/101] Replace old docs with symlinks --- .../example-datasets/wikistat.md | 33 +------------------ .../example-datasets/wikistat.md | 33 +------------------ src/Coordination/FourLetterCommand.cpp | 1 - 3 files changed, 2 insertions(+), 65 deletions(-) mode change 100644 => 120000 docs/ru/getting-started/example-datasets/wikistat.md mode change 100644 => 120000 docs/zh/getting-started/example-datasets/wikistat.md diff --git a/docs/ru/getting-started/example-datasets/wikistat.md b/docs/ru/getting-started/example-datasets/wikistat.md deleted file mode 100644 index 479616d667b9..000000000000 --- a/docs/ru/getting-started/example-datasets/wikistat.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -slug: /ru/getting-started/example-datasets/wikistat -sidebar_position: 17 -sidebar_label: WikiStat ---- - -# WikiStat {#wikistat} - -См: http://dumps.wikimedia.org/other/pagecounts-raw/ - -Создание таблицы: - -``` sql -CREATE TABLE wikistat -( - date Date, - time DateTime, - project String, - subproject String, - path String, - hits UInt64, - size UInt64 -) ENGINE = MergeTree(date, (path, time), 8192); -``` - -Загрузка данных: - -``` bash -$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt -$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done -$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done -``` diff --git a/docs/ru/getting-started/example-datasets/wikistat.md b/docs/ru/getting-started/example-datasets/wikistat.md new file mode 120000 index 000000000000..2d429d00984e --- /dev/null +++ b/docs/ru/getting-started/example-datasets/wikistat.md @@ -0,0 +1 @@ +../../../en/getting-started/example-datasets/wikistat.md \ No newline at end of file diff --git a/docs/zh/getting-started/example-datasets/wikistat.md b/docs/zh/getting-started/example-datasets/wikistat.md deleted file mode 100644 index 4ce13b0f1d3a..000000000000 --- a/docs/zh/getting-started/example-datasets/wikistat.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -slug: /zh/getting-started/example-datasets/wikistat -sidebar_position: 17 -sidebar_label: WikiStat ---- - -# WikiStat {#wikistat} - -参考: http://dumps.wikimedia.org/other/pagecounts-raw/ - -创建表结构: - -``` sql -CREATE TABLE wikistat -( - date Date, - time DateTime, - project String, - subproject String, - path String, - hits UInt64, - size UInt64 -) ENGINE = MergeTree(date, (path, time), 8192); -``` - -加载数据: - -``` bash -$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt -$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done -$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done -``` diff --git a/docs/zh/getting-started/example-datasets/wikistat.md b/docs/zh/getting-started/example-datasets/wikistat.md new file mode 120000 index 000000000000..2d429d00984e --- /dev/null +++ b/docs/zh/getting-started/example-datasets/wikistat.md @@ -0,0 +1 @@ +../../../en/getting-started/example-datasets/wikistat.md \ No newline at end of file diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index d1ff03482b6a..379a24e5aab5 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -577,7 +577,6 @@ String FeatureFlagsCommand::run() } return ret.str(); - } } From 040d6665464a65e3aebb83fe3d2b5830890183fa Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 8 Aug 2023 21:57:27 +0000 Subject: [PATCH 052/101] Fix: 00838_unique_index test with analyzer --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 5 ++--- tests/analyzer_tech_debt.txt | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 05ea7d156032..651dd731e97e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -457,11 +457,10 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio if (arguments_size != 1) return nullptr; - auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context); - const auto & bit_wrapper_func_node = result_dag->addFunction(bit_wrapper_function, {arguments[0]}, {}); + const ActionsDAG::Node * argument = &traverseDAG(*arguments[0], result_dag, context, node_to_result_node); auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context); - return &result_dag->addFunction(bit_swap_last_two_function, {&bit_wrapper_func_node}, {}); + return &result_dag->addFunction(bit_swap_last_two_function, {argument}, {}); } else if (function_name == "and" || function_name == "indexHint" || function_name == "or") { diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index c8f2bb9f43d3..7b61de546fea 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -7,7 +7,6 @@ 00725_memory_tracking 00754_distributed_optimize_skip_select_on_unused_shards 00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere -00838_unique_index 00927_asof_joins 00940_order_by_read_in_order_query_plan 00945_bloom_filter_index From 3296daad96e3df8b1f610458f46ea11afe4a586c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 8 Aug 2023 22:27:12 +0000 Subject: [PATCH 053/101] Don't build AST filter with enabled analyzer --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 651dd731e97e..eaa695c7f7a1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -256,10 +256,6 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!key_columns.contains(name)) key_columns.insert(name); - ASTPtr ast_filter_node = buildFilterNode(query_info.query); - if (!ast_filter_node) - return; - if (context->getSettingsRef().allow_experimental_analyzer) { if (!query_info.filter_actions_dag) @@ -280,6 +276,10 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( } else { + ASTPtr ast_filter_node = buildFilterNode(query_info.query); + if (!ast_filter_node) + return; + if (checkASTUseless(ast_filter_node)) return; From 5561e3e198b91a7596188d98ba2f213e0405614b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 01:44:39 +0200 Subject: [PATCH 054/101] Remove garbage and speed up Debug and Tidy builds --- CMakeLists.txt | 3 --- src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp | 2 -- src/Functions/array/arrayEnumerateRanked.h | 1 - src/Functions/extractAllGroups.h | 2 -- .../keyvaluepair/tests/gtest_extractKeyValuePairs.cpp | 4 +--- src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp | 3 --- 6 files changed, 1 insertion(+), 14 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 157f8c9fac01..55bcf5fbf3c5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -208,9 +208,6 @@ option(OMIT_HEAVY_DEBUG_SYMBOLS "Do not generate debugger info for heavy modules (ClickHouse functions and dictionaries, some contrib)" ${OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT}) -if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - set(USE_DEBUG_HELPERS ON) -endif() option(USE_DEBUG_HELPERS "Enable debug helpers" ${USE_DEBUG_HELPERS}) option(BUILD_STANDALONE_KEEPER "Build keeper as small standalone binary" OFF) diff --git a/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp b/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp index b755bd109d08..0373e55a62df 100644 --- a/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp +++ b/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include #include #include diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 0733f1e2d43b..2e28a0f455db 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -13,7 +13,6 @@ #include #include -// for better debug: #include /** The function will enumerate distinct values of the passed multidimensional arrays looking inside at the specified depths. * This is very unusual function made as a special order for our dear customer - Metrica web analytics system. diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index 3a7987be93e5..f9637e59461c 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -14,8 +14,6 @@ #include #include -#include - namespace DB { diff --git a/src/Functions/keyvaluepair/tests/gtest_extractKeyValuePairs.cpp b/src/Functions/keyvaluepair/tests/gtest_extractKeyValuePairs.cpp index 507d9c0e5c74..55a08023cbd3 100644 --- a/src/Functions/keyvaluepair/tests/gtest_extractKeyValuePairs.cpp +++ b/src/Functions/keyvaluepair/tests/gtest_extractKeyValuePairs.cpp @@ -9,11 +9,11 @@ #include #include -#include #include #include #include + namespace { using namespace DB; @@ -174,5 +174,3 @@ INSTANTIATE_TEST_SUITE_P(InvalidEscapeSeqInValue, extractKVPairKeyValuePairExtra } ) ); - - diff --git a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index 43294355f2f5..c8c00086e8ca 100644 --- a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -3,9 +3,6 @@ #include #include #include - - -#include #include From aa757490bdaae835ef5beb466f05154689879f86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 02:19:02 +0200 Subject: [PATCH 055/101] Ditch tons of garbage --- base/base/JSON.cpp | 2 -- base/base/wide_integer_impl.h | 1 - base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp | 1 - base/poco/Foundation/src/Task.cpp | 1 - base/poco/JSON/src/Object.cpp | 1 - base/poco/Net/src/HTTPClientSession.cpp | 1 - programs/disks/ICommand.cpp | 2 ++ src/Access/MemoryAccessStorage.h | 1 + src/AggregateFunctions/tests/gtest_ranks.cpp | 1 - src/Analyzer/Passes/FuseFunctionsPass.cpp | 3 +++ src/Analyzer/examples/query_analyzer.cpp | 1 - src/Client/ConnectionParameters.cpp | 1 - src/Client/ConnectionString.cpp | 1 - src/Columns/ColumnObject.cpp | 4 ++-- src/Columns/tests/gtest_column_sparse.cpp | 1 + src/Common/Config/configReadClient.cpp | 1 - src/Common/CounterInFile.h | 1 - src/Common/DateLUTImpl.cpp | 1 - src/Common/EventNotifier.h | 2 +- src/Common/FST.cpp | 1 - src/Common/SpaceSaving.h | 2 +- src/Common/StudentTTest.cpp | 1 - src/Common/ThreadPool.cpp | 1 - src/Common/UnicodeBar.cpp | 1 - src/Common/VersionNumber.h | 7 +------ .../integer_hash_tables_and_hashes.cpp | 1 - src/Common/examples/average.cpp | 1 - src/Common/examples/chaos_sanitizer.cpp | 1 - src/Common/examples/compact_array.cpp | 2 +- src/Common/tests/gtest_hash_table.cpp | 2 +- src/Common/tests/gtest_lru_cache.cpp | 1 - src/Common/tests/gtest_lru_hash_map.cpp | 1 - src/Common/tests/gtest_lru_resource_cache.cpp | 1 - src/Common/tests/gtest_slru_cache.cpp | 1 - src/Common/tests/gtest_thread_pool_limit.cpp | 1 - src/Common/tests/gtest_thread_pool_loop.cpp | 1 - .../gtest_thread_pool_schedule_exception.cpp | 1 - .../CompressionCodecDeflateQpl.cpp | 4 +++- .../fuzzers/compressed_buffer_fuzzer.cpp | 1 - .../fuzzers/delta_decompress_fuzzer.cpp | 1 - .../double_delta_decompress_fuzzer.cpp | 1 - .../fuzzers/encrypted_decompress_fuzzer.cpp | 1 - .../fuzzers/lz4_decompress_fuzzer.cpp | 1 - src/Coordination/SnapshotableHashTable.h | 2 +- src/Coordination/SummingStateMachine.cpp | 1 - src/Coordination/pathUtils.cpp | 1 - src/Core/MySQL/MySQLCharset.cpp | 1 - src/Core/fuzzers/names_and_types_fuzzer.cpp | 1 - src/Daemon/BaseDaemon.h | 1 - src/Daemon/SentryWriter.cpp | 2 -- src/Dictionaries/HashedDictionary.cpp | 1 + src/Dictionaries/PolygonDictionaryUtils.h | 1 + src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Functions/array/arrayShuffle.cpp | 1 + src/Functions/evalMLMethod.cpp | 1 - .../keyvaluepair/impl/StateHandler.h | 1 - src/Functions/translate.cpp | 2 ++ src/IO/Archives/LibArchiveReader.h | 1 - src/IO/HTTPCommon.h | 1 - src/IO/ReadBufferFromIStream.h | 2 -- src/IO/S3/SessionAwareIOStream.h | 2 +- src/IO/StdStreamFromReadBuffer.h | 1 - src/IO/VarInt.h | 1 - src/IO/WriteBuffer.h | 1 - src/IO/WriteBufferFromOStream.h | 2 +- src/IO/examples/read_buffer_from_hdfs.cpp | 1 - src/Interpreters/Context.cpp | 19 ------------------- src/Interpreters/GraceHashJoin.cpp | 7 +++---- .../InterpreterKillQueryQuery.cpp | 1 - src/Loggers/OwnSplitChannel.cpp | 1 - src/Parsers/ASTSelectWithUnionQuery.cpp | 1 - .../Executors/StreamingFormatExecutor.cpp | 1 - src/Processors/IAccumulatingTransform.cpp | 1 - src/Processors/ResizeProcessor.cpp | 1 - .../Transforms/CountingTransform.cpp | 1 - .../Transforms/ExceptionKeepingTransform.cpp | 1 - .../Transforms/SquashingChunksTransform.cpp | 1 - src/Server/HTTP/HTTPServerResponse.h | 2 +- src/Storages/MergeTree/GinIndexStore.cpp | 1 - .../fuzzers/mergetree_checksum_fuzzer.cpp | 1 - 80 files changed, 29 insertions(+), 102 deletions(-) diff --git a/base/base/JSON.cpp b/base/base/JSON.cpp index 4c6d97b44443..0b43be381493 100644 --- a/base/base/JSON.cpp +++ b/base/base/JSON.cpp @@ -7,8 +7,6 @@ #include #include -#include - #define JSON_MAX_DEPTH 100 diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 411841e6d9fd..278d62d38a19 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -12,7 +12,6 @@ #include #include -#include #include // NOLINTBEGIN(*) diff --git a/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp b/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp index 4caf097c28a8..1c5555f8cf35 100644 --- a/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp +++ b/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp @@ -19,7 +19,6 @@ #include "Poco/UTF16Encoding.h" #include "Poco/Buffer.h" #include "Poco/Exception.h" -#include using Poco::Buffer; diff --git a/base/poco/Foundation/src/Task.cpp b/base/poco/Foundation/src/Task.cpp index a850ae37eff2..4303d50d6e89 100644 --- a/base/poco/Foundation/src/Task.cpp +++ b/base/poco/Foundation/src/Task.cpp @@ -16,7 +16,6 @@ #include "Poco/TaskManager.h" #include "Poco/Exception.h" -#include #include diff --git a/base/poco/JSON/src/Object.cpp b/base/poco/JSON/src/Object.cpp index 7fca65c5b01d..b041f570934a 100644 --- a/base/poco/JSON/src/Object.cpp +++ b/base/poco/JSON/src/Object.cpp @@ -14,7 +14,6 @@ #include "Poco/JSON/Object.h" #include -#include using Poco::Dynamic::Var; diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index c5697b556d1b..2712c0c452ed 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -26,7 +26,6 @@ #include "Poco/CountingStream.h" #include "Poco/RegularExpression.h" #include -#include using Poco::NumberFormatter; diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp index 52d1a2196a9a..86188fb6db1d 100644 --- a/programs/disks/ICommand.cpp +++ b/programs/disks/ICommand.cpp @@ -1,4 +1,6 @@ #include "ICommand.h" +#include + namespace DB { diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index b63132147da2..deb5a30f4b37 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB diff --git a/src/AggregateFunctions/tests/gtest_ranks.cpp b/src/AggregateFunctions/tests/gtest_ranks.cpp index b29271cbec73..99c7fbd26d70 100644 --- a/src/AggregateFunctions/tests/gtest_ranks.cpp +++ b/src/AggregateFunctions/tests/gtest_ranks.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 2cb7afa4ad6b..7b27bd70ce30 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -14,6 +14,9 @@ #include #include +#include + + namespace DB { diff --git a/src/Analyzer/examples/query_analyzer.cpp b/src/Analyzer/examples/query_analyzer.cpp index 5a20b46b3460..869113b10f8d 100644 --- a/src/Analyzer/examples/query_analyzer.cpp +++ b/src/Analyzer/examples/query_analyzer.cpp @@ -1,4 +1,3 @@ -#include int main(int argc, char ** argv) { diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 8c7a4e830fef..1f90e757b73b 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -1,6 +1,5 @@ #include "ConnectionParameters.h" #include -#include #include #include #include diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index 8150ae98c8b7..0893ef1af004 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 078727745597..2fb6d2c30281 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2,17 +2,17 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include #include #include +#include + namespace DB { diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index 6062ea51941e..c3450ff91b47 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -10,6 +10,7 @@ #include + using namespace DB; static pcg64 rng(randomSeed()); diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 44d338c07af5..9f137124be06 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -3,7 +3,6 @@ #include #include "ConfigProcessor.h" #include -#include #include namespace fs = std::filesystem; diff --git a/src/Common/CounterInFile.h b/src/Common/CounterInFile.h index d52f4a100500..fe3b74173f69 100644 --- a/src/Common/CounterInFile.h +++ b/src/Common/CounterInFile.h @@ -4,7 +4,6 @@ #include #include -#include #include #include diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index d5e04238ef99..4c21d9c9783e 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,7 +10,6 @@ #include #include #include -#include /// Embedded timezones. diff --git a/src/Common/EventNotifier.h b/src/Common/EventNotifier.h index edf7622512dd..1d34f7ad9b1e 100644 --- a/src/Common/EventNotifier.h +++ b/src/Common/EventNotifier.h @@ -7,11 +7,11 @@ #include #include #include -#include #include #include + namespace DB { diff --git a/src/Common/FST.cpp b/src/Common/FST.cpp index 30e10610eab9..af2acc859e74 100644 --- a/src/Common/FST.cpp +++ b/src/Common/FST.cpp @@ -1,7 +1,6 @@ #include "FST.h" #include #include -#include #include #include #include diff --git a/src/Common/SpaceSaving.h b/src/Common/SpaceSaving.h index 3452402ba829..5d3e8fa321bf 100644 --- a/src/Common/SpaceSaving.h +++ b/src/Common/SpaceSaving.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include @@ -19,6 +18,7 @@ #include #include + /* * Implementation of the Filtered Space-Saving for TopK streaming analysis. * http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf diff --git a/src/Common/StudentTTest.cpp b/src/Common/StudentTTest.cpp index 59842488fd07..03159dca2ca0 100644 --- a/src/Common/StudentTTest.cpp +++ b/src/Common/StudentTTest.cpp @@ -1,7 +1,6 @@ #include "StudentTTest.h" #include -#include #include #include #include diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index cc88594d84f8..4a5bdeffcee4 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp index bad39d8080c4..253d720e8a1c 100644 --- a/src/Common/UnicodeBar.cpp +++ b/src/Common/UnicodeBar.cpp @@ -7,7 +7,6 @@ #include #include -#include namespace DB { diff --git a/src/Common/VersionNumber.h b/src/Common/VersionNumber.h index 94bf234c1da3..050c94b95749 100644 --- a/src/Common/VersionNumber.h +++ b/src/Common/VersionNumber.h @@ -2,9 +2,9 @@ #include #include -#include #include + namespace DB { @@ -27,11 +27,6 @@ struct VersionNumber std::string toString() const; - friend std::ostream & operator<<(std::ostream & os, const VersionNumber & v) - { - return os << v.toString(); - } - private: using Components = std::vector; Components components; diff --git a/src/Common/benchmarks/integer_hash_tables_and_hashes.cpp b/src/Common/benchmarks/integer_hash_tables_and_hashes.cpp index c245fc471cc2..2a4e366f1c51 100644 --- a/src/Common/benchmarks/integer_hash_tables_and_hashes.cpp +++ b/src/Common/benchmarks/integer_hash_tables_and_hashes.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include diff --git a/src/Common/examples/average.cpp b/src/Common/examples/average.cpp index f281abdced26..cd4f9321cf0d 100644 --- a/src/Common/examples/average.cpp +++ b/src/Common/examples/average.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Common/examples/chaos_sanitizer.cpp b/src/Common/examples/chaos_sanitizer.cpp index 0b7d21aa33e7..76e22411a2e8 100644 --- a/src/Common/examples/chaos_sanitizer.cpp +++ b/src/Common/examples/chaos_sanitizer.cpp @@ -6,7 +6,6 @@ #include -#include #include diff --git a/src/Common/examples/compact_array.cpp b/src/Common/examples/compact_array.cpp index 58c4ea3be1ef..9e16372deffe 100644 --- a/src/Common/examples/compact_array.cpp +++ b/src/Common/examples/compact_array.cpp @@ -4,11 +4,11 @@ #include #include #include -#include #include #include #include + namespace fs = std::filesystem; static std::string createTmpPath(const std::string & filename) diff --git a/src/Common/tests/gtest_hash_table.cpp b/src/Common/tests/gtest_hash_table.cpp index 0221a682577b..72941126cfd9 100644 --- a/src/Common/tests/gtest_hash_table.cpp +++ b/src/Common/tests/gtest_hash_table.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/src/Common/tests/gtest_lru_cache.cpp b/src/Common/tests/gtest_lru_cache.cpp index 1185dd58e5e9..0360b5b33246 100644 --- a/src/Common/tests/gtest_lru_cache.cpp +++ b/src/Common/tests/gtest_lru_cache.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Common/tests/gtest_lru_hash_map.cpp b/src/Common/tests/gtest_lru_hash_map.cpp index f45a503be43e..cbb3d302efc4 100644 --- a/src/Common/tests/gtest_lru_hash_map.cpp +++ b/src/Common/tests/gtest_lru_hash_map.cpp @@ -1,5 +1,4 @@ #include -#include #include diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index f88eded531e6..bc037824ff88 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Common/tests/gtest_slru_cache.cpp b/src/Common/tests/gtest_slru_cache.cpp index 52549592f0ec..ed04f427d9d0 100644 --- a/src/Common/tests/gtest_slru_cache.cpp +++ b/src/Common/tests/gtest_slru_cache.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Common/tests/gtest_thread_pool_limit.cpp b/src/Common/tests/gtest_thread_pool_limit.cpp index 17f79d17894f..b47c8cdad180 100644 --- a/src/Common/tests/gtest_thread_pool_limit.cpp +++ b/src/Common/tests/gtest_thread_pool_limit.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Common/tests/gtest_thread_pool_loop.cpp b/src/Common/tests/gtest_thread_pool_loop.cpp index 556c39df9495..170a888ff725 100644 --- a/src/Common/tests/gtest_thread_pool_loop.cpp +++ b/src/Common/tests/gtest_thread_pool_loop.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp index 5dbad00848dd..d8e00b5314ca 100644 --- a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp +++ b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 5dcd6008b517..0737e523ba05 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -1,15 +1,17 @@ #ifdef ENABLE_QPL_COMPRESSION + #include #include #include #include #include -#include #include #include #include "libaccel_config.h" #include #include +#include + namespace DB { diff --git a/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp b/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp index 1f669696fb9d..bdab11ef8ab3 100644 --- a/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp +++ b/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp index b039777da158..eaef1d1896c4 100644 --- a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp index f9822daa3bd7..c32120bacbfd 100644 --- a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp index 3e3d0e164feb..eb95c83e1d7a 100644 --- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp index 85c4c9bd329c..f03fc716c2cc 100644 --- a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 7db546bd4c85..093126237ef1 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -6,7 +6,7 @@ #include #include #include -#include + namespace DB { diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index 4c21f1d8658b..b70509fe6f02 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -1,5 +1,4 @@ #include -#include #include namespace DB diff --git a/src/Coordination/pathUtils.cpp b/src/Coordination/pathUtils.cpp index afa42b4a639c..25f8e25cf060 100644 --- a/src/Coordination/pathUtils.cpp +++ b/src/Coordination/pathUtils.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/Core/MySQL/MySQLCharset.cpp b/src/Core/MySQL/MySQLCharset.cpp index 869941ebd840..787e4edcf88b 100644 --- a/src/Core/MySQL/MySQLCharset.cpp +++ b/src/Core/MySQL/MySQLCharset.cpp @@ -1,6 +1,5 @@ #include "MySQLCharset.h" #include "config.h" -#include #include #if USE_ICU diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index cc4a2920c664..94f0872fff6d 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index 7aa1e8ad1a03..952cf61d8e0f 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Daemon/SentryWriter.cpp b/src/Daemon/SentryWriter.cpp index e38d339d0885..942d1e306ae9 100644 --- a/src/Daemon/SentryWriter.cpp +++ b/src/Daemon/SentryWriter.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -13,7 +12,6 @@ #include #include #include -#include #include "config.h" #include "config_version.h" diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 5f25600db8f3..e336ca808563 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -25,6 +25,7 @@ #include #include + namespace CurrentMetrics { extern const Metric HashedDictionaryThreads; diff --git a/src/Dictionaries/PolygonDictionaryUtils.h b/src/Dictionaries/PolygonDictionaryUtils.h index 94b8b9615778..0238ef0b2b90 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.h +++ b/src/Dictionaries/PolygonDictionaryUtils.h @@ -13,6 +13,7 @@ #include + namespace DB { diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index cc75f18bc0d3..dd33395fbfa8 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -2,7 +2,6 @@ #include -#include #include #include #include diff --git a/src/Functions/array/arrayShuffle.cpp b/src/Functions/array/arrayShuffle.cpp index 7c20e195098b..faa5ae47b29e 100644 --- a/src/Functions/array/arrayShuffle.cpp +++ b/src/Functions/array/arrayShuffle.cpp @@ -16,6 +16,7 @@ #include #include + namespace DB { diff --git a/src/Functions/evalMLMethod.cpp b/src/Functions/evalMLMethod.cpp index 346c8249905a..4d5657f0aabe 100644 --- a/src/Functions/evalMLMethod.cpp +++ b/src/Functions/evalMLMethod.cpp @@ -5,7 +5,6 @@ #include #include -#include #include diff --git a/src/Functions/keyvaluepair/impl/StateHandler.h b/src/Functions/keyvaluepair/impl/StateHandler.h index 27c1a0b44bee..178974e9d366 100644 --- a/src/Functions/keyvaluepair/impl/StateHandler.h +++ b/src/Functions/keyvaluepair/impl/StateHandler.h @@ -2,7 +2,6 @@ #include -#include namespace DB { diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index 83779eee23c5..836cb4de2f39 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -6,6 +6,8 @@ #include #include #include +#include + namespace DB { diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index 700e8f70d049..ef50d9de56e1 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -4,7 +4,6 @@ #include -#include namespace DB { diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 082491b2851e..f10fd748200d 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/IO/ReadBufferFromIStream.h b/src/IO/ReadBufferFromIStream.h index 67cc60c053f4..8c3f62728b55 100644 --- a/src/IO/ReadBufferFromIStream.h +++ b/src/IO/ReadBufferFromIStream.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include diff --git a/src/IO/S3/SessionAwareIOStream.h b/src/IO/S3/SessionAwareIOStream.h index f7e42f99f51b..babe52545d1f 100644 --- a/src/IO/S3/SessionAwareIOStream.h +++ b/src/IO/S3/SessionAwareIOStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB::S3 diff --git a/src/IO/StdStreamFromReadBuffer.h b/src/IO/StdStreamFromReadBuffer.h index eae939a28b5f..ff327dc342ec 100644 --- a/src/IO/StdStreamFromReadBuffer.h +++ b/src/IO/StdStreamFromReadBuffer.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index 2a2743e34073..8d10055a3df0 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index ea0323022354..d29ca6d5c6c3 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/IO/WriteBufferFromOStream.h b/src/IO/WriteBufferFromOStream.h index 5a933739cb18..3f9d3ee3d924 100644 --- a/src/IO/WriteBufferFromOStream.h +++ b/src/IO/WriteBufferFromOStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/IO/examples/read_buffer_from_hdfs.cpp b/src/IO/examples/read_buffer_from_hdfs.cpp index da4e52986813..977dd2ae227f 100644 --- a/src/IO/examples/read_buffer_from_hdfs.cpp +++ b/src/IO/examples/read_buffer_from_hdfs.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a210a9efbc7f..8f98759f5926 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2,14 +2,10 @@ #include #include #include -#include #include -#include #include #include -#include #include -#include #include #include #include @@ -17,12 +13,10 @@ #include #include #include -#include #include #include #include #include -#include #include #include #include @@ -34,8 +28,6 @@ #include #include #include -#include -#include #include #include #include @@ -44,7 +36,6 @@ #include #include #include -#include #include #include #include @@ -56,7 +47,6 @@ #include #include #include -#include #include #include #include @@ -70,7 +60,6 @@ #include #include #include -#include #include #include #include @@ -87,8 +76,6 @@ #include #include #include -#include -#include #include #include #include @@ -98,14 +85,12 @@ #include #include #include -#include #include #include #include #include #include #include -#include #include #include #include @@ -118,12 +103,8 @@ #include #include #include -#include #include -#if USE_ROCKSDB -#include -#endif namespace fs = std::filesystem; diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 5d72cf20740f..89ea3a326cc6 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -1,15 +1,11 @@ #include #include #include -#include #include #include #include -#include -#include -#include #include #include @@ -18,6 +14,9 @@ #include +#include + + namespace CurrentMetrics { extern const Metric TemporaryFilesForJoin; diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 3330159aff51..82fe2f57a0ee 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index cdf8402745f7..618d453c2381 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -1,7 +1,6 @@ #include "OwnSplitChannel.h" #include "OwnFormattingChannel.h" -#include #include #include #include diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 9550752b1f36..48b4ae3c38de 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -5,7 +5,6 @@ #include #include -#include namespace DB { diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 2223721439e0..468189890320 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB { diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 00d5b2ee0892..ea3c3c2c1b01 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/Processors/ResizeProcessor.cpp b/src/Processors/ResizeProcessor.cpp index 8167fae9baf2..3a8c6fb2bff3 100644 --- a/src/Processors/ResizeProcessor.cpp +++ b/src/Processors/ResizeProcessor.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/Processors/Transforms/CountingTransform.cpp b/src/Processors/Transforms/CountingTransform.cpp index 646256d60c0d..3dfb9fe178f1 100644 --- a/src/Processors/Transforms/CountingTransform.cpp +++ b/src/Processors/Transforms/CountingTransform.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp index 3c40c0782251..b50f66b0240c 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.cpp +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index e89aec31655e..7de9538e435c 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/Server/HTTP/HTTPServerResponse.h b/src/Server/HTTP/HTTPServerResponse.h index f5b7a70dc79b..236a56e2323f 100644 --- a/src/Server/HTTP/HTTPServerResponse.h +++ b/src/Server/HTTP/HTTPServerResponse.h @@ -5,9 +5,9 @@ #include #include -#include #include + namespace DB { diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index 91e831270d48..5b798ecc8a5f 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/fuzzers/mergetree_checksum_fuzzer.cpp b/src/Storages/fuzzers/mergetree_checksum_fuzzer.cpp index 9a5a68f09cca..0f2ce8a2e44d 100644 --- a/src/Storages/fuzzers/mergetree_checksum_fuzzer.cpp +++ b/src/Storages/fuzzers/mergetree_checksum_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include From 2f6e66c8e3809d80bcab0afe7bf5d9d03761640d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 03:19:24 +0200 Subject: [PATCH 056/101] Remove unnecessary templates --- src/Functions/array/arrayAUC.cpp | 124 +++++++++++++++++++++++-------- 1 file changed, 91 insertions(+), 33 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 4d2b8175f5bd..a0aca7a563bd 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -1,7 +1,9 @@ -#include #include +#include +#include +#include +#include #include -#include "arrayScalarProduct.h" namespace DB @@ -10,6 +12,8 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; } @@ -70,44 +74,32 @@ namespace ErrorCodes * The "curve" will be present by a line that moves one step either towards right or top on each threshold change. */ - -struct NameArrayAUC -{ - static constexpr auto name = "arrayAUC"; -}; - - -class ArrayAUCImpl +class FunctionArrayAUC : public IFunction { public: - using ResultType = Float64; - - static DataTypePtr getReturnType(const DataTypePtr & /* score_type */, const DataTypePtr & label_type) - { - if (!(isNumber(label_type) || isEnum(label_type))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} label must have numeric type.", std::string(NameArrayAUC::name)); - - return std::make_shared>(); - } - - template - static ResultType apply( - const T * scores, - const U * labels, - size_t size) + static constexpr auto name = "arrayAUC"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + +private: + static Float64 apply( + const IColumn & data1, + const IColumn & data2, + ColumnArray::Offset current_offset, + ColumnArray::Offset next_offset) { struct ScoreLabel { - T score; + Float64 score; bool label; }; + size_t size = next_offset - current_offset; PODArrayWithStackMemory sorted_labels(size); for (size_t i = 0; i < size; ++i) { - bool label = labels[i] > 0; - sorted_labels[i].score = scores[i]; + bool label = data2.getFloat64(current_offset + i) > 0; + sorted_labels[i].score = data1.getFloat64(current_offset + i); sorted_labels[i].label = label; } @@ -129,18 +121,84 @@ class ArrayAUCImpl /// Then divide the area to the area of rectangle. if (count_positive == 0 || count_positive == size) - return std::numeric_limits::quiet_NaN(); + return std::numeric_limits::quiet_NaN(); - return static_cast(area) / count_positive / (size - count_positive); + return static_cast(area) / count_positive / (size - count_positive); } -}; + static void vector( + const IColumn & data1, + const IColumn & data2, + const ColumnArray::Offsets & offsets, + PaddedPODArray & result) + { + size_t size = offsets.size(); + result.resize(size); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + auto next_offset = offsets[i]; + result[i] = apply(data1, data2, current_offset, next_offset); + current_offset = next_offset; + } + } + +public: + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + // Basic type check + std::vector nested_types(2, nullptr); + for (size_t i = 0; i < getNumberOfArguments(); ++i) + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for function {} must be an array.", getName()); + + const auto & nested_type = array_type->getNestedType(); + if (!isNativeNumber(nested_type) && !isEnum(nested_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", + getName(), nested_type->getName()); + nested_types[i] = nested_type; + } + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); + ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); + + const ColumnArray * col_array1 = checkAndGetColumn(col1.get()); + const ColumnArray * col_array2 = checkAndGetColumn(col2.get()); + if (!col_array1 || !col_array2) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); + + if (!col_array1->hasEqualOffsets(*col_array2)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); + + auto col_res = ColumnVector::create(); + + vector( + col_array1->getData(), + col_array2->getData(), + col_array1->getOffsets(), + col_res->getData()); + + return col_res; + } +}; -/// auc(array_score, array_label) - Calculate AUC with array of score and label -using FunctionArrayAUC = FunctionArrayScalarProduct; REGISTER_FUNCTION(ArrayAUC) { factory.registerFunction(); } + } From 4d02a924e60979c688c27822ab785bd1aca62068 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 03:29:07 +0200 Subject: [PATCH 057/101] Fix AArch64 --- base/base/wide_integer_impl.h | 1 + 1 file changed, 1 insertion(+) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 278d62d38a19..fc4e9e551ca7 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -21,6 +21,7 @@ #define CONSTEXPR_FROM_DOUBLE constexpr using FromDoubleIntermediateType = long double; #else +#include /// `wide_integer_from_builtin` can't be constexpr with non-literal `cpp_bin_float_double_extended` #define CONSTEXPR_FROM_DOUBLE using FromDoubleIntermediateType = boost::multiprecision::cpp_bin_float_double_extended; From 29221188baaa6b601824f47319ad888462dbf296 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 04:07:31 +0200 Subject: [PATCH 058/101] Fix error --- src/Formats/ReadSchemaUtils.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index d183442ba44a..b40fb20dadfb 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -217,6 +217,8 @@ try } catch (Exception & e) { + if (!buf) + throw; auto file_name = getFileNameFromReadBuffer(*buf); if (!file_name.empty()) e.addMessage(fmt::format("(in file/uri {})", file_name)); From fa9abc5038957ab2af34dc933825473231164ff4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 05:02:50 +0200 Subject: [PATCH 059/101] Better usage of ccache --- programs/keeper/Keeper.cpp | 2 +- programs/server/Server.cpp | 2 +- src/CMakeLists.txt | 3 ++- src/Client/ClientBase.cpp | 2 +- src/Client/Connection.cpp | 6 ++--- src/Common/config_version.cpp.in | 3 +++ src/Common/config_version.h.in | 22 ++----------------- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/FourLetterCommand.h | 4 +++- src/Interpreters/ClientInfo.cpp | 10 ++++----- .../QueryPlan/DistributedCreateLocalPlan.cpp | 2 +- src/Server/TCPHandler.cpp | 8 +++---- .../System/StorageSystemBuildOptions.cpp.in | 1 - 13 files changed, 27 insertions(+), 40 deletions(-) create mode 100644 src/Common/config_version.cpp.in diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 49009fffb90f..1723c274fdb2 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -150,7 +150,7 @@ int Keeper::run() } if (config().hasOption("version")) { - std::cout << DBMS_NAME << " keeper version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; + std::cout << VERSION_NAME << " keeper version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; return 0; } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e6d5837dd0e8..47607ec06309 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -389,7 +389,7 @@ int Server::run() } if (config().hasOption("version")) { - std::cout << DBMS_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; + std::cout << VERSION_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; return 0; } return Application::run(); // NOLINT diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ca38a447c70b..ddb6fcebd230 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -19,6 +19,7 @@ message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION} ${VERSI include (configure_config.cmake) configure_file (Common/config.h.in ${CONFIG_INCLUDE_PATH}/config.h) configure_file (Common/config_version.h.in ${CONFIG_INCLUDE_PATH}/config_version.h) +configure_file (Common/config_version.cpp.in ${CONFIG_INCLUDE_PATH}/config_version.cpp) if (USE_DEBUG_HELPERS) get_target_property(MAGIC_ENUM_INCLUDE_DIR ch_contrib::magic_enum INTERFACE_INCLUDE_DIRECTORIES) @@ -150,7 +151,7 @@ else() message(STATUS "StorageFileLog is only supported on Linux") endif () -list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) +list (APPEND clickhouse_common_io_sources ${CONFIG_INCLUDE_PATH}/config_version.cpp) list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/FunctionsLogical.cpp Functions/indexHint.cpp) list (APPEND dbms_headers Functions/IFunction.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/FunctionsLogical.h Functions/indexHint.h) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a72de2645d4f..c7288d4793a4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2505,7 +2505,7 @@ void ClientBase::clearTerminal() void ClientBase::showClientVersion() { - std::cout << DBMS_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; + std::cout << VERSION_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; } namespace diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index cac5600fbcba..3e12e60be08e 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -280,9 +280,9 @@ void Connection::sendHello() "Parameters 'default_database', 'user' and 'password' must not contain ASCII control characters"); writeVarUInt(Protocol::Client::Hello, *out); - writeStringBinary((DBMS_NAME " ") + client_name, *out); - writeVarUInt(DBMS_VERSION_MAJOR, *out); - writeVarUInt(DBMS_VERSION_MINOR, *out); + writeStringBinary((VERSION_NAME " ") + client_name, *out); + writeVarUInt(VERSION_MAJOR, *out); + writeVarUInt(VERSION_MINOR, *out); // NOTE For backward compatibility of the protocol, client cannot send its version_patch. writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); writeStringBinary(default_database, *out); diff --git a/src/Common/config_version.cpp.in b/src/Common/config_version.cpp.in new file mode 100644 index 000000000000..f31e82bf582b --- /dev/null +++ b/src/Common/config_version.cpp.in @@ -0,0 +1,3 @@ +/// This file was autogenerated by CMake + +const char * VERSION_GITHASH = "@VERSION_GITHASH@"; diff --git a/src/Common/config_version.h.in b/src/Common/config_version.h.in index 7c768bbfad9b..aa8ddeeb860b 100644 --- a/src/Common/config_version.h.in +++ b/src/Common/config_version.h.in @@ -6,7 +6,6 @@ // only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. #cmakedefine VERSION_REVISION @VERSION_REVISION@ #cmakedefine VERSION_NAME "@VERSION_NAME@" -#define DBMS_NAME VERSION_NAME #cmakedefine VERSION_MAJOR @VERSION_MAJOR@ #cmakedefine VERSION_MINOR @VERSION_MINOR@ #cmakedefine VERSION_PATCH @VERSION_PATCH@ @@ -15,27 +14,10 @@ #cmakedefine VERSION_OFFICIAL "@VERSION_OFFICIAL@" #cmakedefine VERSION_FULL "@VERSION_FULL@" #cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@" -#cmakedefine VERSION_GITHASH "@VERSION_GITHASH@" #cmakedefine VERSION_INTEGER @VERSION_INTEGER@ -#cmakedefine VERSION_DATE @VERSION_DATE@ -#if defined(VERSION_MAJOR) -#define DBMS_VERSION_MAJOR VERSION_MAJOR -#else -#define DBMS_VERSION_MAJOR 0 -#endif - -#if defined(VERSION_MINOR) -#define DBMS_VERSION_MINOR VERSION_MINOR -#else -#define DBMS_VERSION_MINOR 0 -#endif - -#if defined(VERSION_PATCH) -#define DBMS_VERSION_PATCH VERSION_PATCH -#else -#define DBMS_VERSION_PATCH 0 -#endif +/// These fields are frequently changing and we don't want to have them in the header file to allow caching. +extern const char * VERSION_GITHASH; #if !defined(VERSION_OFFICIAL) # define VERSION_OFFICIAL "" diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index d1ff03482b6a..6ac22e8bd15c 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -466,7 +466,7 @@ String EnviCommand::run() StringBuffer buf; buf << "Environment:\n"; - buf << "clickhouse.keeper.version=" << (String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH) << '\n'; + buf << "clickhouse.keeper.version=" << VERSION_DESCRIBE << '-' << VERSION_GITHASH << '\n'; buf << "host.name=" << Environment::nodeName() << '\n'; buf << "os.name=" << Environment::osDisplayName() << '\n'; diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index eb2cf9419aef..0520da06b6d0 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -9,8 +9,10 @@ #include "config_version.h" + namespace DB { + struct IFourLetterCommand; using FourLetterCommandPtr = std::shared_ptr; @@ -43,7 +45,7 @@ struct FourLetterCommandFactory : private boost::noncopyable using Commands = std::unordered_map; using AllowList = std::vector; - ///represent '*' which is used in allow list + /// Represents '*' which is used in allow list. static constexpr int32_t ALLOW_LIST_ALL = 0; bool isKnown(int32_t code); diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 413ff1db6bcd..6c09b327ca13 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -194,9 +194,9 @@ void ClientInfo::setInitialQuery() query_kind = QueryKind::INITIAL_QUERY; fillOSUserHostNameAndVersionInfo(); if (client_name.empty()) - client_name = DBMS_NAME; + client_name = VERSION_NAME; else - client_name = (DBMS_NAME " ") + client_name; + client_name = (VERSION_NAME " ") + client_name; } @@ -210,9 +210,9 @@ void ClientInfo::fillOSUserHostNameAndVersionInfo() client_hostname = getFQDNOrHostName(); - client_version_major = DBMS_VERSION_MAJOR; - client_version_minor = DBMS_VERSION_MINOR; - client_version_patch = DBMS_VERSION_PATCH; + client_version_major = VERSION_MAJOR; + client_version_minor = VERSION_MINOR; + client_version_patch = VERSION_PATCH; client_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION; } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index b251eec2d288..2ff50ca4fe3b 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -75,7 +75,7 @@ std::unique_ptr createLocalPlan( new_context->setClientInterface(ClientInfo::Interface::LOCAL); new_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); new_context->setReplicaInfo(true, replica_count, replica_num); - new_context->setConnectionClientVersion(DBMS_VERSION_MAJOR, DBMS_VERSION_MINOR, DBMS_VERSION_PATCH, DBMS_TCP_PROTOCOL_VERSION); + new_context->setConnectionClientVersion(VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH, DBMS_TCP_PROTOCOL_VERSION); new_context->setParallelReplicasGroupUUID(group_uuid); new_context->setMergeTreeAllRangesCallback([coordinator](InitialAllRangesAnnouncement announcement) { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5f3a7614eee4..cd17c2bcb553 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1303,16 +1303,16 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { writeVarUInt(Protocol::Server::Hello, *out); - writeStringBinary(DBMS_NAME, *out); - writeVarUInt(DBMS_VERSION_MAJOR, *out); - writeVarUInt(DBMS_VERSION_MINOR, *out); + writeStringBinary(VERSION_NAME, *out); + writeVarUInt(VERSION_MAJOR, *out); + writeVarUInt(VERSION_MINOR, *out); writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) writeStringBinary(DateLUT::instance().getTimeZone(), *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) writeStringBinary(server_display_name, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) - writeVarUInt(DBMS_VERSION_PATCH, *out); + writeVarUInt(VERSION_PATCH, *out); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { auto rules = server.context()->getAccessControl().getPasswordComplexityRules(); diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 4e7a25d7726b..796b134ba56f 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -8,7 +8,6 @@ const char * auto_config_build[] "SYSTEM", "@CMAKE_SYSTEM_NAME@", "VERSION_GITHASH", "@VERSION_GITHASH@", "VERSION_REVISION", "@VERSION_REVISION@", - "VERSION_DATE", "@VERSION_DATE@", "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM_PROCESSOR", "@CMAKE_SYSTEM_PROCESSOR@", "CMAKE_VERSION", "@CMAKE_VERSION@", From 922542b4973510bc8c6cd1c09d69117c92b9ad60 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 9 Aug 2023 09:11:55 +0000 Subject: [PATCH 060/101] Improved efficiency --- src/Functions/FunctionBinaryArithmetic.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 723abc7c308c..08baf6b23a51 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1180,7 +1180,7 @@ class FunctionBinaryArithmetic : public IFunction /// Unpacking non-const arrays and checking sizes of them. for (auto offset_index = 0U; offset_index < left_offsets.size(); ++offset_index) { - if (left_offsets[offset_index] != right_offsets[offset_index]) + if (right_array_col->hasEqualOffsets(*left_array_col)) { throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Cannot apply operation for arrays of different sizes. Size of the first argument: {}, size of the second argument: {}", From 8ab4f8c9cfaa6665967ccd6c26cb216a35c2943f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 9 Aug 2023 09:12:39 +0000 Subject: [PATCH 061/101] Improved efficiency --- src/Functions/FunctionBinaryArithmetic.h | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 08baf6b23a51..86a5ce29d9cf 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1173,21 +1173,8 @@ class FunctionBinaryArithmetic : public IFunction const auto * left_array_col = typeid_cast(arguments[0].column.get()); const auto * right_array_col = typeid_cast(arguments[1].column.get()); - const auto & left_offsets = left_array_col->getOffsets(); - const auto & right_offsets = right_array_col->getOffsets(); - - chassert(left_offsets.size() == right_offsets.size() && "Unexpected difference in number of offsets"); - /// Unpacking non-const arrays and checking sizes of them. - for (auto offset_index = 0U; offset_index < left_offsets.size(); ++offset_index) - { - if (right_array_col->hasEqualOffsets(*left_array_col)) - { - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "Cannot apply operation for arrays of different sizes. Size of the first argument: {}, size of the second argument: {}", - *left_array_col->getOffsets().data(), - *right_array_col ->getOffsets().data()); - } - } + if (!left_array_col->hasEqualOffsets(*right_array_col)) + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName()); const auto & left_array_type = typeid_cast(arguments[0].type.get())->getNestedType(); new_arguments[0] = {left_array_col->getDataPtr(), left_array_type, arguments[0].name}; @@ -1198,6 +1185,7 @@ class FunctionBinaryArithmetic : public IFunction result_array_type = typeid_cast(result_type.get())->getNestedType(); size_t rows_count = 0; + const auto & left_offsets = left_array_col->getOffsets(); if (!left_offsets.empty()) rows_count = left_offsets.back(); auto res = executeImpl(new_arguments, result_array_type, rows_count); From 85af996d5c7651d2fb2a4f54e8f557aad410e35e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Aug 2023 12:53:05 +0300 Subject: [PATCH 062/101] Update test.py (#53195) --- tests/integration/test_reverse_dns_query/test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/test_reverse_dns_query/test.py b/tests/integration/test_reverse_dns_query/test.py index c5d4304a5d2e..6a39bd7b586b 100644 --- a/tests/integration/test_reverse_dns_query/test.py +++ b/tests/integration/test_reverse_dns_query/test.py @@ -1,4 +1,10 @@ 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 From 48ed54e822f6ed6f6bdd67db3df7a4a58e550bbb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Aug 2023 14:35:50 +0300 Subject: [PATCH 063/101] Allow experimantal features when recovering Replicated db replica (#53167) * allow experimantal features when recovering Replicated db replica * Automatic style fix --------- Co-authored-by: robot-clickhouse --- src/Databases/DatabaseReplicated.cpp | 25 +++++++++++++++++++ src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 4 ++- src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 +++ .../configs/settings.xml | 2 ++ .../test_replicated_database/test.py | 15 ++++++++--- .../02439_merge_selecting_partitions.sql | 7 +++++- 6 files changed, 50 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b93ba731d4af..304c0a20e26f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -818,6 +818,31 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setQueryKindReplicatedDatabaseInternal(); query_context->setCurrentDatabase(getDatabaseName()); query_context->setCurrentQueryId(""); + + /// We will execute some CREATE queries for recovery (not ATTACH queries), + /// so we need to allow experimental features that can be used in a CREATE query + query_context->setSetting("allow_experimental_inverted_index", 1); + query_context->setSetting("allow_experimental_codecs", 1); + query_context->setSetting("allow_experimental_live_view", 1); + query_context->setSetting("allow_experimental_window_view", 1); + query_context->setSetting("allow_experimental_funnel_functions", 1); + query_context->setSetting("allow_experimental_nlp_functions", 1); + query_context->setSetting("allow_experimental_hash_functions", 1); + query_context->setSetting("allow_experimental_object_type", 1); + query_context->setSetting("allow_experimental_annoy_index", 1); + query_context->setSetting("allow_experimental_bigint_types", 1); + query_context->setSetting("allow_experimental_window_functions", 1); + query_context->setSetting("allow_experimental_geo_types", 1); + query_context->setSetting("allow_experimental_map_type", 1); + + query_context->setSetting("allow_suspicious_low_cardinality_types", 1); + query_context->setSetting("allow_suspicious_fixed_string_types", 1); + query_context->setSetting("allow_suspicious_indices", 1); + query_context->setSetting("allow_suspicious_codecs", 1); + query_context->setSetting("allow_hyperscan", 1); + query_context->setSetting("allow_simdjson", 1); + query_context->setSetting("allow_deprecated_syntax_for_merge_tree", 1); + auto txn = std::make_shared(current_zookeeper, zookeeper_path, false, ""); query_context->initZooKeeperMetadataTransaction(txn); return query_context; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 514baeb7b070..de4e0a806e02 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -98,7 +98,9 @@ void S3QueueFilesMetadata::S3QueueProcessedCollection::parse(const String & coll void S3QueueFilesMetadata::S3QueueProcessedCollection::add(const String & file_name) { - TrackedCollectionItem processed_file = { .file_path=file_name, .timestamp = getCurrentTime() }; + TrackedCollectionItem processed_file; + processed_file.file_path = file_name; + processed_file.timestamp = getCurrentTime(); files.push_back(processed_file); /// TODO: it is strange that in parse() we take into account only max_age, but here only max_size. diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index c436de946ffa..577c71b2227f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -17,6 +17,9 @@ class S3QueueFilesMetadata public: struct TrackedCollectionItem { + TrackedCollectionItem() = default; + TrackedCollectionItem(const String & file_path_, UInt64 timestamp_, UInt64 retries_count_, const String & last_exception_) + : file_path(file_path_), timestamp(timestamp_), retries_count(retries_count_), last_exception(last_exception_) {} String file_path; UInt64 timestamp = 0; UInt64 retries_count = 0; diff --git a/tests/integration/test_replicated_database/configs/settings.xml b/tests/integration/test_replicated_database/configs/settings.xml index 5666ffeace85..7c0e60a044ea 100644 --- a/tests/integration/test_replicated_database/configs/settings.xml +++ b/tests/integration/test_replicated_database/configs/settings.xml @@ -4,6 +4,8 @@ 1 1 1 + 0 + 0 diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ed034a326da1..1235f7d34df4 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -672,7 +672,11 @@ def test_alters_from_different_replicas(started_cluster): def create_some_tables(db): - settings = {"distributed_ddl_task_timeout": 0} + settings = { + "distributed_ddl_task_timeout": 0, + "allow_experimental_object_type": 1, + "allow_suspicious_codecs": 1, + } main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( f"CREATE TABLE {db}.t2 (s String) ENGINE=Memory", settings=settings @@ -690,11 +694,11 @@ def create_some_tables(db): settings=settings, ) dummy_node.query( - f"CREATE TABLE {db}.rmt2 (n int) ENGINE=ReplicatedMergeTree order by n", + f"CREATE TABLE {db}.rmt2 (n int CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12))) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) main_node.query( - f"CREATE TABLE {db}.rmt3 (n int) ENGINE=ReplicatedMergeTree order by n", + f"CREATE TABLE {db}.rmt3 (n int, json Object('json') materialized '') ENGINE=ReplicatedMergeTree order by n", settings=settings, ) dummy_node.query( @@ -868,7 +872,10 @@ def test_recover_staled_replica(started_cluster): ]: assert main_node.query(f"SELECT (*,).1 FROM recover.{table}") == "42\n" for table in ["t2", "rmt1", "rmt2", "rmt4", "d1", "d2", "mt2", "mv1", "mv3"]: - assert dummy_node.query(f"SELECT (*,).1 FROM recover.{table}") == "42\n" + assert ( + dummy_node.query(f"SELECT '{table}', (*,).1 FROM recover.{table}") + == f"{table}\t42\n" + ) for table in ["m1", "mt1"]: assert dummy_node.query(f"SELECT count() FROM recover.{table}") == "0\n" global test_recover_staled_replica_run diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql index dfdbbf5d5975..1d01fde56d66 100644 --- a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql @@ -24,6 +24,11 @@ system flush logs; select * from system.zookeeper_log where path like '/test/02439/s1/' || currentDatabase() || '/block_numbers/%' and op_num in ('List', 'SimpleList', 'FilteredList') and path not like '%/block_numbers/1' and path not like '%/block_numbers/123' - and event_time >= now() - interval 1 minute; + and event_time >= now() - interval 1 minute + -- avoid race with tests like 02311_system_zookeeper_insert + and (query_id is null or query_id='' or query_id in + (select query_id from system.query_log + where event_time >= now() - interval 1 minute and current_database=currentDatabase()) + ); drop table rmt; From 63183c56f162b3bbcecff6aaa377dacf2ca21917 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 9 Aug 2023 13:58:05 +0200 Subject: [PATCH 064/101] Revert "Upload build time-trace data to CI database" --- .github/workflows/backport_branches.yml | 3 - .github/workflows/master.yml | 3 - .github/workflows/pull_request.yml | 3 - .github/workflows/release_branches.yml | 3 - docker/packager/binary/Dockerfile | 1 - docker/packager/binary/build.sh | 13 +--- docker/packager/packager | 7 +- src/Storages/StorageFile.cpp | 4 +- tests/ci/build_check.py | 78 +------------------ .../prepare-time-trace/prepare-time-trace.sh | 3 +- 10 files changed, 11 insertions(+), 107 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 1df8cea4ad52..d69168b01ee5 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -3,9 +3,6 @@ name: BackportPR env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 - # Export system tables to ClickHouse Cloud - CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} - CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f25611e86669..ae1862e327f6 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3,9 +3,6 @@ name: MasterCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 - # Export system tables to ClickHouse Cloud - CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} - CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 25927954809c..d97b9975c3c1 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3,9 +3,6 @@ name: PullRequestCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 - # Export system tables to ClickHouse Cloud - CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} - CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy pull_request: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index e5527177aa50..212848155831 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -3,9 +3,6 @@ name: ReleaseBranchCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 - # Export system tables to ClickHouse Cloud - CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} - CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 15e0bad93555..99e748c41d4b 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -101,7 +101,6 @@ RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ python3-boto3 \ yasm \ zstd \ - jq \ && apt-get clean \ && rm -rf /var/lib/apt/lists diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index e36212bfeeff..38b2ef75d248 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -59,7 +59,7 @@ if [ "$BUILD_MUSL_KEEPER" == "1" ] then # build keeper with musl separately # and without rust bindings - cmake --debug-trycompile -DENABLE_RUST=OFF -DBUILD_STANDALONE_KEEPER=1 -DENABLE_CLICKHOUSE_KEEPER=1 -DCMAKE_VERBOSE_MAKEFILE=1 -DUSE_MUSL=1 -LA -DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DENABLE_RUST=OFF -DBUILD_STANDALONE_KEEPER=1 -DENABLE_CLICKHOUSE_KEEPER=1 -DCMAKE_VERBOSE_MAKEFILE=1 -DUSE_MUSL=1 -LA -DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. ninja $NINJA_FLAGS clickhouse-keeper @@ -74,10 +74,10 @@ then rm -f CMakeCache.txt # Build the rest of binaries - cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. else # Build everything - cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. fi # No quotes because I want it to expand to nothing if empty. @@ -181,11 +181,4 @@ then tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi -# Prepare profile info (time-trace) -mkdir -p profile-tmp -../utils/prepare-time-trace/prepare-time-trace.sh . profile-tmp -find profile-tmp -type f -print0 | xargs -0 cat > /profile/profile.json - -wc -c /profile/profile.json - ls -l /output diff --git a/docker/packager/packager b/docker/packager/packager index 9f949f63eda6..b6b9682f418f 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -78,14 +78,11 @@ def run_docker_image_with_env( image_name: str, as_root: bool, output_dir: Path, - profile_dir: Path, env_variables: List[str], ch_root: Path, ccache_dir: Optional[Path], ): output_dir.mkdir(parents=True, exist_ok=True) - profile_dir.mkdir(parents=True, exist_ok=True) - env_part = " -e ".join(env_variables) if env_part: env_part = " -e " + env_part @@ -106,7 +103,7 @@ def run_docker_image_with_env( cmd = ( f"docker run --network=host --user={user} --rm {ccache_mount}" - f"--volume={output_dir}:/output --volume={ch_root}:/build --volume={profile_dir}:/profile {env_part} " + f"--volume={output_dir}:/output --volume={ch_root}:/build {env_part} " f"{interactive} {image_name}" ) @@ -364,7 +361,6 @@ def parse_args() -> argparse.Namespace: help="ClickHouse git repository", ) parser.add_argument("--output-dir", type=dir_name, required=True) - parser.add_argument("--profile-dir", type=dir_name, required=True) parser.add_argument("--debug-build", action="store_true") parser.add_argument( @@ -492,7 +488,6 @@ def main(): image_with_version, args.as_root, args.output_dir, - args.profile_dir, env_prepared, ch_root, args.ccache_dir, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5c03540de9a8..e3908c75a588 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -396,9 +396,9 @@ std::unique_ptr createReadBuffer( throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", current_path, matcher->error()); - return reader->readFile([my_matcher = std::move(matcher)](const std::string & path) + return reader->readFile([matcher = std::move(matcher)](const std::string & path) { - return re2::RE2::FullMatch(path, *my_matcher); + return re2::RE2::FullMatch(path, *matcher); }); } else diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 5e22557edd88..592c27c2c68e 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -1,14 +1,12 @@ #!/usr/bin/env python3 -from typing import List, Tuple import subprocess import logging import json import os import sys import time -import urllib.parse -import requests # type: ignore +from typing import List, Tuple from ci_config import CI_CONFIG, BuildConfig from docker_pull_helper import get_image_with_version @@ -32,7 +30,6 @@ from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, - get_instance_type, ) from stopwatch import Stopwatch @@ -54,7 +51,6 @@ def get_packager_cmd( build_config: BuildConfig, packager_path: str, output_path: str, - profile_path: str, build_version: str, image_version: str, official: bool, @@ -63,8 +59,8 @@ def get_packager_cmd( comp = build_config.compiler cmake_flags = "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=1" cmd = ( - f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} --profile-dir={profile_path}" - f" --package-type={package_type} --compiler={comp}" + f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} " + f"--package-type={package_type} --compiler={comp}" ) if build_config.debug_build: @@ -290,15 +286,10 @@ def main(): if not os.path.exists(build_output_path): os.makedirs(build_output_path) - build_profile_path = os.path.join(TEMP_PATH, f"{build_name}_profile") - if not os.path.exists(build_profile_path): - os.makedirs(build_profile_path) - packager_cmd = get_packager_cmd( build_config, os.path.join(REPO_COPY, "docker/packager"), build_output_path, - build_profile_path, version.string, image_version, official_flag, @@ -369,69 +360,6 @@ def main(): upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) - # Upload profile data - - instance_type = get_instance_type() - query = urllib.parse.quote( - f""" - INSERT INTO build_time_trace - ( - pull_request_number, - commit_sha, - check_start_time, - check_name, - instance_type, - file, - library, - time, - pid, - tid, - ph, - ts, - dur, - cat, - name, - detail, - count, - avgMs, - args_name - ) - SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * - FROM input(' - file String, - library String, - time DateTime64(6), - pid UInt32, - tid UInt32, - ph String, - ts UInt64, - dur UInt64, - cat String, - name String, - detail String, - count UInt64, - avgMs UInt64, - args_name String') - FORMAT JSONCompactEachRow - """ - ) - clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST") - maybe_clickhouse_ci_logs_password: str = ( - os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD") or "" - ) - url = f"https://{clickhouse_ci_logs_host}/?query={query}" - file_path = os.path.join(build_profile_path, "profile.json") - file_size = os.path.getsize(file_path) - - print( - f"::notice ::Log Uploading profile data, path: {file_path}, size: {file_size}, query: {query}" - ) - - with open(file_path, "rb") as file: - requests.post(url, data=file, auth=("ci", maybe_clickhouse_ci_logs_password)) - - # Upload statistics to CI database - ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( pr_info, diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh index 7cacdec8c94b..300a32b0fd17 100755 --- a/utils/prepare-time-trace/prepare-time-trace.sh +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -35,6 +35,7 @@ ENGINE = MergeTree ORDER BY (date, file, name, args_name); INPUT_DIR=$1 OUTPUT_DIR=$2 +EXTRA_COLUMN_VALUES=$3 find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P $(nproc) -I{} bash -c " @@ -42,7 +43,7 @@ find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P LIBRARY_NAME=\$(echo '{}' | sed -r -e 's!^.*/CMakeFiles/([^/]+)\.dir/.*\$!\1!') START_TIME=\$(jq '.beginningOfTime' '{}') - jq -c '.traceEvents[] | [\"'\"\$ORIGINAL_FILENAME\"'\", \"'\"\$LIBRARY_NAME\"'\", '\$START_TIME', .pid, .tid, .ph, .ts, .dur, .cat, .name, .args.detail, .args.count, .args[\"avg ms\"], .args.name]' '{}' > \"${OUTPUT_DIR}/\$\$\" + jq -c '.traceEvents[] | [${EXTRA_COLUMN_VALUES} \"'\"\$ORIGINAL_FILENAME\"'\", \"'\"\$LIBRARY_NAME\"'\", '\$START_TIME', .pid, .tid, .ph, .ts, .dur, .cat, .name, .args.detail, .args.count, .args[\"avg ms\"], .args.name]' '{}' > \"${OUTPUT_DIR}/\$\$\" " # Now you can upload it as follows: From 54a82072e7749130d2717fe96f3f1a831cdbf4b0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Aug 2023 15:06:31 +0300 Subject: [PATCH 065/101] Update test.py --- .../test_host_regexp_multiple_ptr_records/test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py index 82ae0b6c5278..36f76140a2cb 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -1,4 +1,10 @@ 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 From a9580644eb0cb55361da4249c0d30b1c2d249b8d Mon Sep 17 00:00:00 2001 From: Slach Date: Wed, 9 Aug 2023 16:30:33 +0400 Subject: [PATCH 066/101] add ** wildcard to docs, which available from 22.11 Signed-off-by: Slach --- docs/en/engines/table-engines/integrations/azureBlobStorage.md | 2 +- docs/en/engines/table-engines/integrations/s3.md | 3 ++- docs/en/engines/table-engines/integrations/s3queue.md | 3 ++- docs/en/sql-reference/functions/files.md | 2 +- docs/en/sql-reference/table-functions/azureBlobStorage.md | 2 +- .../sql-reference/table-functions/azureBlobStorageCluster.md | 2 +- docs/en/sql-reference/table-functions/gcs.md | 2 +- docs/en/sql-reference/table-functions/hdfsCluster.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- docs/en/sql-reference/table-functions/s3Cluster.md | 2 +- 10 files changed, 12 insertions(+), 10 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 60e448377d0c..3df08ee2ffba 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -21,7 +21,7 @@ CREATE TABLE azure_blob_storage_table (name String, value UInt32) - `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) - `container_name` - Container name -- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. - `account_name` - if storage_account_url is used, then account name can be specified here - `account_key` - if storage_account_url is used, then account key can be specified here - `format` — The [format](/docs/en/interfaces/formats.md) of the file. diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index c1752ea488c9..2967a15494ce 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -37,7 +37,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) ### Engine parameters -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). - `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. - `format` — The [format](../../../interfaces/formats.md#formats) of the file. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). @@ -164,6 +164,7 @@ For more information about virtual columns see [here](../../../engines/table-eng `path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment). - `*` — Substitutes any number of any characters except `/` including empty string. +- `**` — Substitutes any number of any character include `/` including empty string. - `?` — Substitutes any single character. - `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. - `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`. diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 3e6cf88312fd..68d41d4b0a77 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -27,7 +27,7 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) **Engine parameters** -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). - `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. - `format` — The [format](../../../interfaces/formats.md#formats) of the file. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). @@ -213,6 +213,7 @@ For more information about virtual columns see [here](../../../engines/table-eng `path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment). - `*` — Substitutes any number of any characters except `/` including empty string. +- `**` — Substitutes any number of any characters include `/` including empty string. - `?` — Substitutes any single character. - `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. - `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`. diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 73d72aa50e53..d62cd1db88d0 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -18,7 +18,7 @@ file(path[, default]) **Arguments** -- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports wildcards `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. +- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports wildcards `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. - `default` — The value returned if the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). **Example** diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 7bb5d892c47d..59c92e1327e5 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -19,7 +19,7 @@ azureBlobStorage(- connection_string|storage_account_url, container_name, blobpa - `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) - `container_name` - Container name -- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. - `account_name` - if storage_account_url is used, then account name can be specified here - `account_key` - if storage_account_url is used, then account key can be specified here - `format` — The [format](../../interfaces/formats.md#formats) of the file. diff --git a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md index 77acdf515493..20dfd35d5db9 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md @@ -19,7 +19,7 @@ azureBlobStorageCluster(cluster_name, connection_string|storage_account_url, con - `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. - `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) - `container_name` - Container name -- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. - `account_name` - if storage_account_url is used, then account name can be specified here - `account_key` - if storage_account_url is used, then account key can be specified here - `format` — The [format](../../interfaces/formats.md#formats) of the file. diff --git a/docs/en/sql-reference/table-functions/gcs.md b/docs/en/sql-reference/table-functions/gcs.md index 01b4e4f6a69a..48c2381696eb 100644 --- a/docs/en/sql-reference/table-functions/gcs.md +++ b/docs/en/sql-reference/table-functions/gcs.md @@ -22,7 +22,7 @@ The GCS Table Function integrates with Google Cloud Storage by using the GCS XML **Arguments** -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. :::note GCS The GCS path is in this format as the endpoint for the Google XML API is different than the JSON API: diff --git a/docs/en/sql-reference/table-functions/hdfsCluster.md b/docs/en/sql-reference/table-functions/hdfsCluster.md index 832be46d05f7..75100eeb4f30 100644 --- a/docs/en/sql-reference/table-functions/hdfsCluster.md +++ b/docs/en/sql-reference/table-functions/hdfsCluster.md @@ -17,7 +17,7 @@ hdfsCluster(cluster_name, URI, format, structure) **Arguments** - `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. -- `URI` — URI to a file or a bunch of files. Supports following wildcards in readonly mode: `*`, `?`, `{'abc','def'}` and `{N..M}` where `N`, `M` — numbers, `abc`, `def` — strings. For more information see [Wildcards In Path](../../engines/table-engines/integrations/s3.md#wildcards-in-path). +- `URI` — URI to a file or a bunch of files. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{'abc','def'}` and `{N..M}` where `N`, `M` — numbers, `abc`, `def` — strings. For more information see [Wildcards In Path](../../engines/table-engines/integrations/s3.md#wildcards-in-path). - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 55c825b8b9b0..07addafcf58e 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -23,7 +23,7 @@ For GCS, substitute your HMAC key and HMAC secret where you see `aws_access_key_ **Arguments** -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [here](../../engines/table-engines/integrations/s3.md#wildcards-in-path). +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [here](../../engines/table-engines/integrations/s3.md#wildcards-in-path). :::note GCS The GCS path is in this format as the endpoint for the Google XML API is different than the JSON API: diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index d5bdc85f9f89..675aef54d346 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -16,7 +16,7 @@ s3Cluster(cluster_name, source, [,access_key_id, secret_access_key] [,format] [, **Arguments** - `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. -- `source` — URL to a file or a bunch of files. Supports following wildcards in readonly mode: `*`, `?`, `{'abc','def'}` and `{N..M}` where `N`, `M` — numbers, `abc`, `def` — strings. For more information see [Wildcards In Path](../../engines/table-engines/integrations/s3.md#wildcards-in-path). +- `source` — URL to a file or a bunch of files. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{'abc','def'}` and `{N..M}` where `N`, `M` — numbers, `abc`, `def` — strings. For more information see [Wildcards In Path](../../engines/table-engines/integrations/s3.md#wildcards-in-path). - `access_key_id` and `secret_access_key` — Keys that specify credentials to use with given endpoint. Optional. - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. From dc4be338959ac307ef1e71570c8d1cda5e08ed45 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 9 Aug 2023 14:55:44 +0200 Subject: [PATCH 067/101] Updated logs to include retries --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 15b632a54d43..b60c48566729 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -105,7 +105,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() auto handle_exception = [&, this](const auto & e, size_t i) { - LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); + LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); if (i + 1 == max_single_read_retries) throw; @@ -224,7 +224,7 @@ void ReadBufferFromAzureBlobStorage::initialize() auto handle_exception = [&, this](const auto & e, size_t i) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_read_retries, e.Message); if (i + 1 == max_single_download_retries) throw; From 9953050b620b8be4a134829a20e84934097521dc Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 9 Aug 2023 14:57:17 +0200 Subject: [PATCH 068/101] Fixed logs for download --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index b60c48566729..fc0dbee373a9 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -224,7 +224,7 @@ void ReadBufferFromAzureBlobStorage::initialize() auto handle_exception = [&, this](const auto & e, size_t i) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_read_retries, e.Message); + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retriesg, e.Message); if (i + 1 == max_single_download_retries) throw; From 964c407d482f5e3f6ff6094c7707f4fe8924e84d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 16:11:17 +0300 Subject: [PATCH 069/101] Revert "Revert "Upload build time-trace data to CI database"" --- .github/workflows/backport_branches.yml | 3 + .github/workflows/master.yml | 3 + .github/workflows/pull_request.yml | 3 + .github/workflows/release_branches.yml | 3 + docker/packager/binary/Dockerfile | 1 + docker/packager/binary/build.sh | 13 +++- docker/packager/packager | 7 +- src/Storages/StorageFile.cpp | 4 +- tests/ci/build_check.py | 78 ++++++++++++++++++- .../prepare-time-trace/prepare-time-trace.sh | 3 +- 10 files changed, 107 insertions(+), 11 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index d69168b01ee5..1df8cea4ad52 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -3,6 +3,9 @@ name: BackportPR env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index ae1862e327f6..f25611e86669 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3,6 +3,9 @@ name: MasterCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index d97b9975c3c1..25927954809c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3,6 +3,9 @@ name: PullRequestCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy pull_request: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 212848155831..e5527177aa50 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -3,6 +3,9 @@ name: ReleaseBranchCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 99e748c41d4b..15e0bad93555 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -101,6 +101,7 @@ RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ python3-boto3 \ yasm \ zstd \ + jq \ && apt-get clean \ && rm -rf /var/lib/apt/lists diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 38b2ef75d248..e36212bfeeff 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -59,7 +59,7 @@ if [ "$BUILD_MUSL_KEEPER" == "1" ] then # build keeper with musl separately # and without rust bindings - cmake --debug-trycompile -DENABLE_RUST=OFF -DBUILD_STANDALONE_KEEPER=1 -DENABLE_CLICKHOUSE_KEEPER=1 -DCMAKE_VERBOSE_MAKEFILE=1 -DUSE_MUSL=1 -LA -DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DENABLE_RUST=OFF -DBUILD_STANDALONE_KEEPER=1 -DENABLE_CLICKHOUSE_KEEPER=1 -DCMAKE_VERBOSE_MAKEFILE=1 -DUSE_MUSL=1 -LA -DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. ninja $NINJA_FLAGS clickhouse-keeper @@ -74,10 +74,10 @@ then rm -f CMakeCache.txt # Build the rest of binaries - cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. else # Build everything - cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. fi # No quotes because I want it to expand to nothing if empty. @@ -181,4 +181,11 @@ then tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi +# Prepare profile info (time-trace) +mkdir -p profile-tmp +../utils/prepare-time-trace/prepare-time-trace.sh . profile-tmp +find profile-tmp -type f -print0 | xargs -0 cat > /profile/profile.json + +wc -c /profile/profile.json + ls -l /output diff --git a/docker/packager/packager b/docker/packager/packager index b6b9682f418f..9f949f63eda6 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -78,11 +78,14 @@ def run_docker_image_with_env( image_name: str, as_root: bool, output_dir: Path, + profile_dir: Path, env_variables: List[str], ch_root: Path, ccache_dir: Optional[Path], ): output_dir.mkdir(parents=True, exist_ok=True) + profile_dir.mkdir(parents=True, exist_ok=True) + env_part = " -e ".join(env_variables) if env_part: env_part = " -e " + env_part @@ -103,7 +106,7 @@ def run_docker_image_with_env( cmd = ( f"docker run --network=host --user={user} --rm {ccache_mount}" - f"--volume={output_dir}:/output --volume={ch_root}:/build {env_part} " + f"--volume={output_dir}:/output --volume={ch_root}:/build --volume={profile_dir}:/profile {env_part} " f"{interactive} {image_name}" ) @@ -361,6 +364,7 @@ def parse_args() -> argparse.Namespace: help="ClickHouse git repository", ) parser.add_argument("--output-dir", type=dir_name, required=True) + parser.add_argument("--profile-dir", type=dir_name, required=True) parser.add_argument("--debug-build", action="store_true") parser.add_argument( @@ -488,6 +492,7 @@ def main(): image_with_version, args.as_root, args.output_dir, + args.profile_dir, env_prepared, ch_root, args.ccache_dir, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e3908c75a588..5c03540de9a8 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -396,9 +396,9 @@ std::unique_ptr createReadBuffer( throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", current_path, matcher->error()); - return reader->readFile([matcher = std::move(matcher)](const std::string & path) + return reader->readFile([my_matcher = std::move(matcher)](const std::string & path) { - return re2::RE2::FullMatch(path, *matcher); + return re2::RE2::FullMatch(path, *my_matcher); }); } else diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 592c27c2c68e..5e22557edd88 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -1,12 +1,14 @@ #!/usr/bin/env python3 +from typing import List, Tuple import subprocess import logging import json import os import sys import time -from typing import List, Tuple +import urllib.parse +import requests # type: ignore from ci_config import CI_CONFIG, BuildConfig from docker_pull_helper import get_image_with_version @@ -30,6 +32,7 @@ from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, + get_instance_type, ) from stopwatch import Stopwatch @@ -51,6 +54,7 @@ def get_packager_cmd( build_config: BuildConfig, packager_path: str, output_path: str, + profile_path: str, build_version: str, image_version: str, official: bool, @@ -59,8 +63,8 @@ def get_packager_cmd( comp = build_config.compiler cmake_flags = "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=1" cmd = ( - f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} " - f"--package-type={package_type} --compiler={comp}" + f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} --profile-dir={profile_path}" + f" --package-type={package_type} --compiler={comp}" ) if build_config.debug_build: @@ -286,10 +290,15 @@ def main(): if not os.path.exists(build_output_path): os.makedirs(build_output_path) + build_profile_path = os.path.join(TEMP_PATH, f"{build_name}_profile") + if not os.path.exists(build_profile_path): + os.makedirs(build_profile_path) + packager_cmd = get_packager_cmd( build_config, os.path.join(REPO_COPY, "docker/packager"), build_output_path, + build_profile_path, version.string, image_version, official_flag, @@ -360,6 +369,69 @@ def main(): upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) + # Upload profile data + + instance_type = get_instance_type() + query = urllib.parse.quote( + f""" + INSERT INTO build_time_trace + ( + pull_request_number, + commit_sha, + check_start_time, + check_name, + instance_type, + file, + library, + time, + pid, + tid, + ph, + ts, + dur, + cat, + name, + detail, + count, + avgMs, + args_name + ) + SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * + FROM input(' + file String, + library String, + time DateTime64(6), + pid UInt32, + tid UInt32, + ph String, + ts UInt64, + dur UInt64, + cat String, + name String, + detail String, + count UInt64, + avgMs UInt64, + args_name String') + FORMAT JSONCompactEachRow + """ + ) + clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST") + maybe_clickhouse_ci_logs_password: str = ( + os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD") or "" + ) + url = f"https://{clickhouse_ci_logs_host}/?query={query}" + file_path = os.path.join(build_profile_path, "profile.json") + file_size = os.path.getsize(file_path) + + print( + f"::notice ::Log Uploading profile data, path: {file_path}, size: {file_size}, query: {query}" + ) + + with open(file_path, "rb") as file: + requests.post(url, data=file, auth=("ci", maybe_clickhouse_ci_logs_password)) + + # Upload statistics to CI database + ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( pr_info, diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh index 300a32b0fd17..7cacdec8c94b 100755 --- a/utils/prepare-time-trace/prepare-time-trace.sh +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -35,7 +35,6 @@ ENGINE = MergeTree ORDER BY (date, file, name, args_name); INPUT_DIR=$1 OUTPUT_DIR=$2 -EXTRA_COLUMN_VALUES=$3 find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P $(nproc) -I{} bash -c " @@ -43,7 +42,7 @@ find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P LIBRARY_NAME=\$(echo '{}' | sed -r -e 's!^.*/CMakeFiles/([^/]+)\.dir/.*\$!\1!') START_TIME=\$(jq '.beginningOfTime' '{}') - jq -c '.traceEvents[] | [${EXTRA_COLUMN_VALUES} \"'\"\$ORIGINAL_FILENAME\"'\", \"'\"\$LIBRARY_NAME\"'\", '\$START_TIME', .pid, .tid, .ph, .ts, .dur, .cat, .name, .args.detail, .args.count, .args[\"avg ms\"], .args.name]' '{}' > \"${OUTPUT_DIR}/\$\$\" + jq -c '.traceEvents[] | [\"'\"\$ORIGINAL_FILENAME\"'\", \"'\"\$LIBRARY_NAME\"'\", '\$START_TIME', .pid, .tid, .ph, .ts, .dur, .cat, .name, .args.detail, .args.count, .args[\"avg ms\"], .args.name]' '{}' > \"${OUTPUT_DIR}/\$\$\" " # Now you can upload it as follows: From 901901a76271e64637bc7c33f60dc38677998de3 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 9 Aug 2023 16:50:40 +0200 Subject: [PATCH 070/101] Fixed typo --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index fc0dbee373a9..cd8502c913fd 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -224,7 +224,7 @@ void ReadBufferFromAzureBlobStorage::initialize() auto handle_exception = [&, this](const auto & e, size_t i) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retriesg, e.Message); + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); if (i + 1 == max_single_download_retries) throw; From 5341c5c8dd90adccbe6cb850819e3d155e51426b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 16:00:10 +0200 Subject: [PATCH 071/101] Use CMAKE_FLAGS and args for cmake profiler --- docker/packager/binary/build.sh | 6 +++--- docker/packager/packager | 16 +++++++++++----- tests/ci/build_check.py | 1 + 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index e36212bfeeff..9277ac0491e6 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -59,7 +59,7 @@ if [ "$BUILD_MUSL_KEEPER" == "1" ] then # build keeper with musl separately # and without rust bindings - cmake --debug-trycompile -DENABLE_RUST=OFF -DBUILD_STANDALONE_KEEPER=1 -DENABLE_CLICKHOUSE_KEEPER=1 -DCMAKE_VERBOSE_MAKEFILE=1 -DUSE_MUSL=1 -LA -DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DENABLE_RUST=OFF -DBUILD_STANDALONE_KEEPER=1 -DENABLE_CLICKHOUSE_KEEPER=1 -DCMAKE_VERBOSE_MAKEFILE=1 -DUSE_MUSL=1 -LA -DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. ninja $NINJA_FLAGS clickhouse-keeper @@ -74,10 +74,10 @@ then rm -f CMakeCache.txt # Build the rest of binaries - cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. else # Build everything - cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. fi # No quotes because I want it to expand to nothing if empty. diff --git a/docker/packager/packager b/docker/packager/packager index 9f949f63eda6..5a710a8c57ce 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -22,7 +22,7 @@ def check_image_exists_locally(image_name: str) -> bool: output = subprocess.check_output( f"docker images -q {image_name} 2> /dev/null", shell=True ) - return output != "" + return output != b"" except subprocess.CalledProcessError: return False @@ -46,7 +46,7 @@ def build_image(image_name: str, filepath: Path) -> None: ) -def pre_build(repo_path: Path, env_variables: List[str]): +def pre_build(repo_path: Path, env_variables: List[str]) -> None: if "WITH_PERFORMANCE=1" in env_variables: current_branch = subprocess.check_output( "git branch --show-current", shell=True, encoding="utf-8" @@ -82,7 +82,7 @@ def run_docker_image_with_env( env_variables: List[str], ch_root: Path, ccache_dir: Optional[Path], -): +) -> None: output_dir.mkdir(parents=True, exist_ok=True) profile_dir.mkdir(parents=True, exist_ok=True) @@ -132,9 +132,10 @@ def parse_env_variables( version: str, official: bool, additional_pkgs: bool, + with_profiler: bool, with_coverage: bool, with_binaries: str, -): +) -> List[str]: DARWIN_SUFFIX = "-darwin" DARWIN_ARM_SUFFIX = "-darwin-aarch64" ARM_SUFFIX = "-aarch64" @@ -325,6 +326,9 @@ def parse_env_variables( # utils are not included into clickhouse-bundle, so build everything build_target = "all" + if with_profiler: + cmake_flags.append("-DENABLE_BUILD_PROFILING=1") + if with_coverage: cmake_flags.append("-DWITH_COVERAGE=1") @@ -420,6 +424,7 @@ def parse_args() -> argparse.Namespace: parser.add_argument("--version") parser.add_argument("--official", action="store_true") parser.add_argument("--additional-pkgs", action="store_true") + parser.add_argument("--with-profiler", action="store_true") parser.add_argument("--with-coverage", action="store_true") parser.add_argument( "--with-binaries", choices=("programs", "tests", ""), default="" @@ -455,7 +460,7 @@ def parse_args() -> argparse.Namespace: return args -def main(): +def main() -> None: logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") args = parse_args() @@ -483,6 +488,7 @@ def main(): args.version, args.official, args.additional_pkgs, + args.with_profiler, args.with_coverage, args.with_binaries, ) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 5e22557edd88..139c995c7ec3 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -82,6 +82,7 @@ def get_packager_cmd( cmd += " --additional-pkgs" cmd += f" --docker-image-version={image_version}" + cmd += " --with-profiler" cmd += f" --version={build_version}" if _can_export_binaries(build_config): From 140534683090d9bd4ee67e618f13802638b4f170 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 18:56:19 +0200 Subject: [PATCH 072/101] Rewrite build_check to use pathlib.Path --- tests/ci/build_check.py | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 139c995c7ec3..999c8b6d74f6 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +from pathlib import Path from typing import List, Tuple import subprocess import logging @@ -53,7 +54,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: def get_packager_cmd( build_config: BuildConfig, packager_path: str, - output_path: str, + output_path: Path, profile_path: str, build_version: str, image_version: str, @@ -95,13 +96,13 @@ def get_packager_cmd( def build_clickhouse( - packager_cmd: str, logs_path: str, build_output_path: str -) -> Tuple[str, bool]: - build_log_path = os.path.join(logs_path, BUILD_LOG_NAME) + packager_cmd: str, logs_path: Path, build_output_path: Path +) -> Tuple[Path, bool]: + build_log_path = logs_path / BUILD_LOG_NAME success = False with TeePopen(packager_cmd, build_log_path) as process: retcode = process.wait() - if os.path.exists(build_output_path): + if build_output_path.exists(): build_results = os.listdir(build_output_path) else: build_results = [] @@ -222,7 +223,7 @@ def upload_master_static_binaries( pr_info: PRInfo, build_config: BuildConfig, s3_helper: S3Helper, - build_output_path: str, + build_output_path: Path, ) -> None: """Upload binary artifacts to a static S3 links""" static_binary_name = build_config.static_binary_name @@ -234,7 +235,7 @@ def upload_master_static_binaries( return s3_path = "/".join((pr_info.base_ref, static_binary_name, "clickhouse")) - binary = os.path.join(build_output_path, "clickhouse") + binary = build_output_path / "clickhouse" url = s3_helper.upload_build_file_to_s3(binary, s3_path) print(f"::notice ::Binary static URL: {url}") @@ -247,8 +248,8 @@ def main(): build_config = CI_CONFIG.build_config[build_name] - if not os.path.exists(TEMP_PATH): - os.makedirs(TEMP_PATH) + temp_path = Path(TEMP_PATH) + os.makedirs(temp_path, exist_ok=True) pr_info = PRInfo() @@ -287,9 +288,8 @@ def main(): logging.info("Build short name %s", build_name) - build_output_path = os.path.join(TEMP_PATH, build_name) - if not os.path.exists(build_output_path): - os.makedirs(build_output_path) + build_output_path = temp_path / build_name + os.makedirs(build_output_path, exist_ok=True) build_profile_path = os.path.join(TEMP_PATH, f"{build_name}_profile") if not os.path.exists(build_profile_path): @@ -330,8 +330,8 @@ def main(): # FIXME performance performance_urls = [] - performance_path = os.path.join(build_output_path, "performance.tar.zst") - if os.path.exists(performance_path): + performance_path = build_output_path / "performance.tar.zst" + if performance_path.exists(): performance_urls.append( s3_helper.upload_build_file_to_s3(performance_path, s3_performance_path) ) @@ -354,9 +354,9 @@ def main(): print("::notice ::Build URLs: {}".format("\n".join(build_urls))) - if os.path.exists(log_path): + if log_path.exists(): log_url = s3_helper.upload_build_file_to_s3( - log_path, s3_path_prefix + "/" + os.path.basename(log_path) + log_path, s3_path_prefix + "/" + log_path.name ) logging.info("Log url %s", log_url) else: From da01d756f9c17d27b35bfcb3381b479b5c1e5a7f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 20:20:25 +0200 Subject: [PATCH 073/101] Addressed review comments --- src/Functions/array/arrayAUC.cpp | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index a0aca7a563bd..caf929ba0385 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -82,8 +82,8 @@ class FunctionArrayAUC : public IFunction private: static Float64 apply( - const IColumn & data1, - const IColumn & data2, + const IColumn & scores, + const IColumn & labels, ColumnArray::Offset current_offset, ColumnArray::Offset next_offset) { @@ -98,8 +98,8 @@ class FunctionArrayAUC : public IFunction for (size_t i = 0; i < size; ++i) { - bool label = data2.getFloat64(current_offset + i) > 0; - sorted_labels[i].score = data1.getFloat64(current_offset + i); + bool label = labels.getFloat64(current_offset + i) > 0; + sorted_labels[i].score = scores.getFloat64(current_offset + i); sorted_labels[i].label = label; } @@ -151,8 +151,6 @@ class FunctionArrayAUC : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - // Basic type check - std::vector nested_types(2, nullptr); for (size_t i = 0; i < getNumberOfArguments(); ++i) { const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); @@ -163,7 +161,6 @@ class FunctionArrayAUC : public IFunction if (!isNativeNumber(nested_type) && !isEnum(nested_type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", getName(), nested_type->getName()); - nested_types[i] = nested_type; } return std::make_shared(); @@ -175,11 +172,15 @@ class FunctionArrayAUC : public IFunction ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); const ColumnArray * col_array1 = checkAndGetColumn(col1.get()); - const ColumnArray * col_array2 = checkAndGetColumn(col2.get()); - if (!col_array1 || !col_array2) + if (!col_array1) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); + const ColumnArray * col_array2 = checkAndGetColumn(col2.get()); + if (!col_array2) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second argument of function {}", arguments[1].column->getName(), getName()); + if (!col_array1->hasEqualOffsets(*col_array2)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); From 6a3267ad3770187ed5d253e952c02f282cd1cb3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 20:21:51 +0200 Subject: [PATCH 074/101] A fix for clang-17 --- src/Storages/StorageFile.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e3908c75a588..5c03540de9a8 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -396,9 +396,9 @@ std::unique_ptr createReadBuffer( throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", current_path, matcher->error()); - return reader->readFile([matcher = std::move(matcher)](const std::string & path) + return reader->readFile([my_matcher = std::move(matcher)](const std::string & path) { - return re2::RE2::FullMatch(path, *matcher); + return re2::RE2::FullMatch(path, *my_matcher); }); } else From 239d198cf2049688790fc7607a61fbf44eb85329 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 21:11:21 +0200 Subject: [PATCH 075/101] Add insert_file to ClickHouseHelper, make insert_json_str public --- tests/ci/clickhouse_helper.py | 71 ++++++++++++++++++++++++++++++++--- 1 file changed, 66 insertions(+), 5 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 7f98f6e52d20..dbf26288469a 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 -from typing import List +from pathlib import Path +from typing import Dict, List, Optional import json import logging import time @@ -16,18 +17,78 @@ class InsertException(Exception): class ClickHouseHelper: - def __init__(self, url=None): + def __init__( + self, url: Optional[str] = None, auth: Optional[Dict[str, str]] = None + ): if url is None: url = get_parameter_from_ssm("clickhouse-test-stat-url") self.url = url - self.auth = { + self.auth = auth or { "X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"), "X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"), } @staticmethod - def _insert_json_str_info_impl(url, auth, db, table, json_str): + def insert_file( + url: str, + auth: Optional[Dict[str, str]], + query: str, + file: Path, + additional_options: Optional[Dict[str, str]] = None, + ) -> None: + params = { + "query": query, + "date_time_input_format": "best_effort", + "send_logs_level": "warning", + } + if additional_options: + for k, v in additional_options.items(): + 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) + + @staticmethod + def insert_json_str(url, auth, db, table, json_str): params = { "database": db, "query": f"INSERT INTO {table} FORMAT JSONEachRow", @@ -76,7 +137,7 @@ def _insert_json_str_info_impl(url, auth, db, table, json_str): raise InsertException(error) def _insert_json_str_info(self, db, table, json_str): - self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str) + self.insert_json_str(self.url, self.auth, db, table, json_str) def insert_event_into(self, db, table, event, safe=True): event_str = json.dumps(event) From c7b045fdde6a2108ee47036be770b8556df42b32 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 21:12:01 +0200 Subject: [PATCH 076/101] Process build profiles in the python script, not in docker --- docker/packager/binary/Dockerfile | 1 - docker/packager/binary/build.sh | 7 -- docker/packager/packager | 6 +- tests/ci/build_check.py | 144 +++++++++++++++--------------- 4 files changed, 73 insertions(+), 85 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 15e0bad93555..99e748c41d4b 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -101,7 +101,6 @@ RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ python3-boto3 \ yasm \ zstd \ - jq \ && apt-get clean \ && rm -rf /var/lib/apt/lists diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 9277ac0491e6..38b2ef75d248 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -181,11 +181,4 @@ then tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi -# Prepare profile info (time-trace) -mkdir -p profile-tmp -../utils/prepare-time-trace/prepare-time-trace.sh . profile-tmp -find profile-tmp -type f -print0 | xargs -0 cat > /profile/profile.json - -wc -c /profile/profile.json - ls -l /output diff --git a/docker/packager/packager b/docker/packager/packager index 5a710a8c57ce..c0baacccef28 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -78,13 +78,11 @@ def run_docker_image_with_env( image_name: str, as_root: bool, output_dir: Path, - profile_dir: Path, env_variables: List[str], ch_root: Path, ccache_dir: Optional[Path], ) -> None: output_dir.mkdir(parents=True, exist_ok=True) - profile_dir.mkdir(parents=True, exist_ok=True) env_part = " -e ".join(env_variables) if env_part: @@ -106,7 +104,7 @@ def run_docker_image_with_env( cmd = ( f"docker run --network=host --user={user} --rm {ccache_mount}" - f"--volume={output_dir}:/output --volume={ch_root}:/build --volume={profile_dir}:/profile {env_part} " + f"--volume={output_dir}:/output --volume={ch_root}:/build {env_part} " f"{interactive} {image_name}" ) @@ -368,7 +366,6 @@ def parse_args() -> argparse.Namespace: help="ClickHouse git repository", ) parser.add_argument("--output-dir", type=dir_name, required=True) - parser.add_argument("--profile-dir", type=dir_name, required=True) parser.add_argument("--debug-build", action="store_true") parser.add_argument( @@ -498,7 +495,6 @@ def main() -> None: image_with_version, args.as_root, args.output_dir, - args.profile_dir, env_prepared, ch_root, args.ccache_dir, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 999c8b6d74f6..9e6370c2e6ee 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -8,8 +8,6 @@ import os import sys import time -import urllib.parse -import requests # type: ignore from ci_config import CI_CONFIG, BuildConfig from docker_pull_helper import get_image_with_version @@ -21,12 +19,12 @@ S3_DOWNLOAD, TEMP_PATH, ) +from git_helper import Git, git_runner from pr_info import PRInfo from s3_helper import S3Helper from tee_popen import TeePopen from version_helper import ( ClickHouseVersion, - Git, get_version_from_repo, update_version_local, ) @@ -55,7 +53,6 @@ def get_packager_cmd( build_config: BuildConfig, packager_path: str, output_path: Path, - profile_path: str, build_version: str, image_version: str, official: bool, @@ -64,8 +61,8 @@ def get_packager_cmd( comp = build_config.compiler cmake_flags = "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=1" cmd = ( - f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} --profile-dir={profile_path}" - f" --package-type={package_type} --compiler={comp}" + f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager " + f"--output-dir={output_path} --package-type={package_type} --compiler={comp}" ) if build_config.debug_build: @@ -291,15 +288,10 @@ def main(): build_output_path = temp_path / build_name os.makedirs(build_output_path, exist_ok=True) - build_profile_path = os.path.join(TEMP_PATH, f"{build_name}_profile") - if not os.path.exists(build_profile_path): - os.makedirs(build_profile_path) - packager_cmd = get_packager_cmd( build_config, os.path.join(REPO_COPY, "docker/packager"), build_output_path, - build_profile_path, version.string, image_version, official_flag, @@ -307,9 +299,8 @@ def main(): logging.info("Going to run packager with %s", packager_cmd) - logs_path = os.path.join(TEMP_PATH, "build_log") - if not os.path.exists(logs_path): - os.makedirs(logs_path) + logs_path = temp_path / "build_log" + os.makedirs(logs_path, exist_ok=True) start = time.time() log_path, success = build_clickhouse(packager_cmd, logs_path, build_output_path) @@ -371,69 +362,78 @@ def main(): upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) # Upload profile data + ch_helper = ClickHouseHelper() - instance_type = get_instance_type() - query = urllib.parse.quote( - f""" - INSERT INTO build_time_trace - ( - pull_request_number, - commit_sha, - check_start_time, - check_name, - instance_type, - file, - library, - time, - pid, - tid, - ph, - ts, - dur, - cat, - name, - detail, - count, - avgMs, - args_name + clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "") + if clickhouse_ci_logs_host: + instance_type = get_instance_type() + query = f"""INSERT INTO build_time_trace +( + pull_request_number, + commit_sha, + check_start_time, + check_name, + instance_type, + file, + library, + time, + pid, + tid, + ph, + ts, + dur, + cat, + name, + detail, + count, + avgMs, + args_name +) +SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * +FROM input(' + file String, + library String, + time DateTime64(6), + pid UInt32, + tid UInt32, + ph String, + ts UInt64, + dur UInt64, + cat String, + name String, + detail String, + count UInt64, + avgMs UInt64, + args_name String') +FORMAT JSONCompactEachRow""" + + auth = { + "X-ClickHouse-User": "ci", + "X-ClickHouse-Key": os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD", ""), + } + url = f"https://{clickhouse_ci_logs_host}/" + profiles_dir = temp_path / "profiles_source" + os.makedirs(profiles_dir, exist_ok=True) + logging.info("Processing profile JSON files from {GIT_REPO_ROOT}/build_docker") + git_runner( + "./utils/prepare-time-trace/prepare-time-trace.sh " + f"build_docker {profiles_dir.absolute()}" ) - SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * - FROM input(' - file String, - library String, - time DateTime64(6), - pid UInt32, - tid UInt32, - ph String, - ts UInt64, - dur UInt64, - cat String, - name String, - detail String, - count UInt64, - avgMs UInt64, - args_name String') - FORMAT JSONCompactEachRow - """ - ) - clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST") - maybe_clickhouse_ci_logs_password: str = ( - os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD") or "" - ) - url = f"https://{clickhouse_ci_logs_host}/?query={query}" - file_path = os.path.join(build_profile_path, "profile.json") - file_size = os.path.getsize(file_path) + profile_data_file = temp_path / "profile.json" + with open(profile_data_file, "wb") as profile_fd: + for profile_sourse in os.listdir(profiles_dir): + with open(profiles_dir / profile_sourse, "rb") as ps_fd: + profile_fd.write(ps_fd.read()) - print( - f"::notice ::Log Uploading profile data, path: {file_path}, size: {file_size}, query: {query}" - ) - - with open(file_path, "rb") as file: - requests.post(url, data=file, auth=("ci", maybe_clickhouse_ci_logs_password)) + logging.info( + "::notice ::Log Uploading profile data, path: %s, size: %s, query: %s", + profile_data_file, + profile_data_file.stat().st_size, + query, + ) + ch_helper.insert_file(url, auth, query, profile_data_file) # Upload statistics to CI database - - ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( pr_info, [], From 481792181e6ac9515db966817d123afb99d9971f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 23:12:39 +0200 Subject: [PATCH 077/101] 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 2b6fcf6cac23..1bd3561f24fc 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 90287e19bd0b..000000000000 --- 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 ae9cd4e7c91e..bca7122a2a86 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 85c45d5df3c0..921fb4e4154e 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 92d681698bc5..6eedf63f95c6 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 05f638ec3371..8e45af446403 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 1da8fd3325af..8d6242f47113 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 034519331717..3632369154a5 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 e69de29bb2d1..000000000000 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 7de1391e56cf..000000000000 --- 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 82c525ea4147..000000000000 --- 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 495ea201870b..000000000000 --- 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 791ae03f9f66..000000000000 --- 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 e69de29bb2d1..000000000000 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 7a2141e6c7e2..000000000000 --- 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 58ef55cd3f35..000000000000 --- 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 3edf37dafa51..000000000000 --- 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 9beb415c290a..000000000000 --- 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 36f76140a2cb..000000000000 --- 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 a5f92002450a..b1eb0df2d43a 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 1ec44d8a0026..b099d0513e11 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 e7d7abc8c7c5..0b4a750b38ea 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 48bcca7d5057..debb509de90e 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 e69de29bb2d1..000000000000 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 bb4aba94e0be..000000000000 --- 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 eb3d62eb7180..000000000000 --- 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 e69de29bb2d1..000000000000 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 5ce55afa2a7e..000000000000 --- 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 58ef55cd3f35..000000000000 --- 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 35d0a07c6a6c..000000000000 --- 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 84d297f7cdfa..000000000000 --- 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 6a39bd7b586b..000000000000 --- 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 2a4e0eece088..0ca81a278021 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 e69de29bb2d1..000000000000 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 54f7152690bc..000000000000 --- 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 7bb7fa875e4c..000000000000 --- 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 04bff4a44fbf..000000000000 --- 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 5fbe426074f8..000000000000 --- 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 fb9f3eb67b9c..000000000000 --- 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 e69de29bb2d1..000000000000 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 090ed917cddd..000000000000 --- 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 4aa547e049e3..000000000000 --- 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 15817f321f04..000000000000 --- 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 0a8be9122ad4..000000000000 --- 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 2cef0a6de3c3..000000000000 --- 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 689221c39e4b..000000000000 --- 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 e9e5cb723633..000000000000 --- 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 4d7e4cf813d2..000000000000 --- 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 959bb2aad74c..000000000000 --- 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 e69de29bb2d1..000000000000 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 27c7107ce5e2..000000000000 --- 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 664957001026..000000000000 --- 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 959bb2aad74c..000000000000 --- 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 751279f5e5a4..80d2050b394a 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 a2f28e21666c..c1c076277bb9 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1529,106 +1529,6 @@ def optimize_table(num): 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 ce30b93b34348700351e0d462edb5baa6ddb5685 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 23:46:45 +0200 Subject: [PATCH 078/101] 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 dbf26288469a..b165cda37e80 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -47,45 +47,9 @@ def insert_file( 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 @@ def insert_json_str(url, auth, db, table, json_str): "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 @@ def insert_json_str(url, auth, db, table, json_str): 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 a79f2f140d25657a248f0d96bbe0241065a4260d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 00:19:24 +0200 Subject: [PATCH 079/101] 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 1a3b47964c01..0dd1d78ac824 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 cd5b788a1470..e64e93fd4a0c 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 c071f6755d70a5ce9f1924fc44a905cc3122da5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:10:03 +0300 Subject: [PATCH 080/101] Revert "Added new tests for session_log and fixed the inconsistency between login and logout." --- src/Interpreters/Session.cpp | 42 +-- 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, 26 insertions(+), 752 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 0a6435cff758..de2a779b7408 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,35 +655,21 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - /// 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()) + if (!notified_session_log_about_login) { - 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); + 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; + notified_session_log_about_login = true; + } } + + return query_context; } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 81ef987b428a..51c0e3c71fa0 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,8 +97,6 @@ class Session 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 b847eaf9824c..0a8a7fc18c55 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 8757bc122705..1282ac09c4dd 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 10d5e7a0242d..ff5690a3b079 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 65587933fedd..2930262f63ec 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 @@ 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 edf565ec6329..000000000000 --- 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 e69de29bb2d1..000000000000 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 7a079b81e693..000000000000 --- 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 fbaefc16b3a8..000000000000 --- 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 a0e4e3e2216b..000000000000 --- 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 0416dfadc8ae..000000000000 --- 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 25d15f11e3bd..000000000000 --- 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 dbb39993ce39..000000000000 --- 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 bfe507e8eac0..000000000000 --- 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 d1d571c69850..000000000000 --- 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 e2680982ab0b..000000000000 --- 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 64f20a70cb13..000000000000 --- 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 7252faab8c6c..000000000000 --- 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 e848e1fe90da..000000000000 --- 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 081/101] 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 798d2a40b123..c04228167872 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 082/101] 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 c04228167872..f949e66ab179 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 4ba5273939804ed2f639cc4748eaaa7f3b9d4af2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:56:27 +0200 Subject: [PATCH 083/101] 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 6500306356cb..2a13e11e1160 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 084/101] 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 1c185daec758..000000000000 --- 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 bbcc7d607940..000000000000 --- 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 085/101] Update readme --- docker/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e5..bae02201add1 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 b8be7eef044052a24d0c00873d150d8de3a5ea4f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 03:45:59 +0000 Subject: [PATCH 086/101] Added new tests for session_log and bugfix --- src/Interpreters/Session.cpp | 42 ++- 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, 752 insertions(+), 22 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 de2a779b7408..0a6435cff758 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,21 +659,35 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - if (!notified_session_log_about_login) + /// 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()) { - if (auto session_log = getSessionLog()) - { - session_log->addLoginSuccess( - auth_id, - named_session ? std::optional(named_session->key.second) : std::nullopt, - *query_context, - user); + const auto & settings = login_context->getSettingsRef(); + const auto access = login_context->getAccess(); - notified_session_log_about_login = true; - } - } + session_log->addLoginSuccess(auth_id, + named_session ? named_session->key.second : "", + settings, + access, + getClientInfo(), + user); - return query_context; + notified_session_log_about_login = true; + } } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 51c0e3c71fa0..81ef987b428a 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,6 +97,8 @@ class Session 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 0a8a7fc18c55..b847eaf9824c 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 1282ac09c4dd..8757bc122705 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 ff5690a3b079..10d5e7a0242d 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 000000000000..edf565ec6329 --- /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 000000000000..e69de29bb2d1 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 000000000000..7a079b81e693 --- /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 000000000000..fbaefc16b3a8 --- /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 000000000000..a0e4e3e2216b --- /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 000000000000..0416dfadc8ae --- /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 000000000000..25d15f11e3bd --- /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 000000000000..dbb39993ce39 --- /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 000000000000..bfe507e8eac0 --- /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 000000000000..26b48462a760 --- /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 000000000000..e2680982ab0b --- /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 000000000000..3bedfb6c9eeb --- /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 000000000000..7252faab8c6c --- /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 000000000000..347ebd22f96c --- /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 087/101] 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 0a6435cff758..f8bd70afdb63 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 088/101] 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 2930262f63ec..65587933fedd 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 @@ 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 4b30900fed3dcafa015a7d6dc4d6d91ea362a966 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 14:44:16 +0300 Subject: [PATCH 089/101] Revert "Improve CHECK TABLE system query" --- .../sql-reference/statements/check-table.md | 99 +++------ src/Common/FileChecker.cpp | 42 ++-- src/Common/FileChecker.h | 37 +--- src/Interpreters/InterpreterCheckQuery.cpp | 191 +++--------------- 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, 129 insertions(+), 535 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 db8c32249efb..0209d59b018e 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 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. +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: -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,78 +26,44 @@ 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 -┌─result─┐ -│ 1 │ -└────────┘ +┌─part_path─┬─is_passed─┬─message─┐ +│ all_1_4_1 │ 1 │ │ +│ all_1_4_2 │ 1 │ │ +└───────────┴───────────┴─────────┘ ``` -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. +If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status. ```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 +SET check_query_single_value_result = 1; +CHECK TABLE test_table; ``` -```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. │ -└──────────────┴───────────┴──────────────────────────────────────────┘ +┌─result─┐ +│ 1 │ +└────────┘ ``` -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 122ea83835da..876bc4e641cd 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 8ffc310b84dc..bb0383e4b563 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 @@ class FileChecker 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 @@ class FileChecker /// 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 bd530654dd2b..333aed848734 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) -{ - return Block{{ColumnUInt8::create(1, value), std::make_shared(), "result"}}; -} - -Block getHeaderForCheckResult() +NamesAndTypes getBlockStructure() { - 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 +InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { -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_) -{ -} 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); - - const auto & settings = context->getSettingsRef(); - - BlockIO res; + Block block; + if (getContext()->getSettingsRef().check_query_single_value_result) { - auto processors = std::make_shared(); - - 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); - } + 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(); - OutputPort * resize_outport; + for (const auto & check_result : check_results) { - 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); + path_column->insert(check_result.fs_path); + is_passed_column->insert(static_cast(check_result.success)); + message_column->insert(check_result.failure_message); } - if (settings.check_query_single_value_result) - { - auto emitter_processor = std::make_shared(); - auto * input_port = &emitter_processor->getInputPort(); - processors->emplace_back(emitter_processor); + 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}}); + } - connect(*resize_outport, *input_port); - } + BlockIO res; + res.pipeline = QueryPipeline(std::make_shared(std::move(block))); - 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 2e4652fea293..b342b014fa47 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 85299f63165a..ae7659e074f5 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 b1e20c557821..ec92f57aeda5 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -595,45 +595,8 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// 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 0e9f83e886a9..87aa71f3e8d5 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -866,19 +866,16 @@ 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)); -} -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); + return file_checker.check(); } + IStorage::ColumnSizeByName StorageLog::getColumnSizes() const { ReadLock lock{rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)}; diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 95f95088aa2e..f1d05ed39ac2 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,8 +59,7 @@ class StorageLog final : public IStorage, public WithMutableContext 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 @@ class StorageLog final : public IStorage, public WithMutableContext 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 97fc7a6731f1..ad9013d9f131 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 ec4796e49410..c77e5140d75e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,8 +108,7 @@ class StorageMergeTree final : public MergeTreeData 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 @@ class StorageMergeTree final : public MergeTreeData 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 ea908bea0328..21ed4b91c621 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,9 +149,7 @@ class StorageProxy : public IStorage 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 c08f1ebcc485..7fce373e26bc 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 - { - return CheckResult(part_check_thread.checkPartAndFix(part->name)); - } - catch (const Exception & ex) + auto part_check_lock = part_check_thread.pausePartsCheck(); + + for (auto & part : data_parts) { - 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 2bc18aa3b0a3..78ef39f032fc 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,8 +230,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// 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 @@ class StorageReplicatedMergeTree final : public MergeTreeData 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 a3cbff961998..0bfef5ed5e58 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,19 +403,17 @@ 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)); -} -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); + return file_checker.check(); } + void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { disk->clearDirectory(table_path); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 5d4e2fcbd3ab..f889a1de71bc 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,8 +53,7 @@ friend class StripeLogSink; 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 @@ friend class StripeLogSink; 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 e8183f05f5db..6ed281c757a9 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 263cf94fb4af..e7362074a059 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 079acc8cdbb2..0e0b2c3b4837 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 eccb2d258781..b66aff8384d7 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 625be63e0c07..0ac97404c46f 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 541dab48def8..000000000000 --- 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 166386b999b5..000000000000 --- 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 090/101] 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 caf929ba0385..b7bd7dcc0adb 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -127,8 +127,8 @@ class FunctionArrayAUC : public IFunction } static void vector( - const IColumn & data1, - const IColumn & data2, + const IColumn & scores, + const IColumn & labels, const ColumnArray::Offsets & offsets, PaddedPODArray & result) { From 3aca2408548bc149f933379506250e49238a24de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:59:33 +0200 Subject: [PATCH 091/101] 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 14b8954fc392..85cdda635588 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 a0e4e3e2216b..000000000000 --- 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 0416dfadc8ae..000000000000 --- 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 25d15f11e3bd..000000000000 --- 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 dbb39993ce39..000000000000 --- 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 bfe507e8eac0..000000000000 --- 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 26b48462a760..000000000000 --- 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 e2680982ab0b..000000000000 --- 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 3bedfb6c9eeb..000000000000 --- 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 7252faab8c6c..000000000000 --- 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 347ebd22f96c..000000000000 --- 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 3193c6d4ccc7ef02857f14825e1a781e8c2e83c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:55:57 +0200 Subject: [PATCH 099/101] 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 b7bd7dcc0adb..499fe4ce7b20 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -139,7 +139,7 @@ class FunctionArrayAUC : public IFunction 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 4952cd1bd24c..29ba7224c75a 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 1fab492222ae..eafe4f941ddf 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 1ef301f5b1da2ee161f1194981da883d15dd1085 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 03:11:16 +0200 Subject: [PATCH 100/101] Fix `01485_256_bit_multiply` --- tests/queries/0_stateless/01485_256_bit_multiply.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01485_256_bit_multiply.sql b/tests/queries/0_stateless/01485_256_bit_multiply.sql index f3f6cc05058a..5c8c47c91274 100644 --- a/tests/queries/0_stateless/01485_256_bit_multiply.sql +++ b/tests/queries/0_stateless/01485_256_bit_multiply.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings, no-asan, no-msan, no-tsan, no-ubsan, no-debug + select count() from ( select toInt128(number) * number x, toInt256(number) * number y from numbers_mt(100000000) where x != y From a6f89c05465753379f481cfe91130a9cd87c66b0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 11 Aug 2023 13:24:16 +0300 Subject: [PATCH 101/101] =?UTF-8?q?Added=20suggestions=20for=20mistyped=20?= =?UTF-8?q?names=20for=20db=20and=20tables=20with=20different=E2=80=A6=20(?= =?UTF-8?q?#49801)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Common/NamePrompter.h | 5 + src/Databases/IDatabase.cpp | 11 +- src/Databases/IDatabase.h | 1 + src/Interpreters/DatabaseCatalog.cpp | 167 ++++++++++++++---- src/Interpreters/DatabaseCatalog.h | 30 +++- tests/analyzer_integration_broken_tests.txt | 1 + .../test.py | 2 +- .../test_drop_is_lock_free/test.py | 2 +- tests/integration/test_drop_replica/test.py | 4 +- .../test_insert_into_distributed/test.py | 5 +- tests/integration/test_mysql_protocol/test.py | 6 +- .../test_system_flush_logs/test.py | 2 +- .../test_wrong_db_or_table_name/__init__.py | 0 .../test_wrong_db_or_table_name/test.py | 108 +++++++++++ ..._log_and_exception_messages_formatting.sql | 6 +- .../01192_rename_database_zookeeper.sh | 4 +- .../0_stateless/01600_detach_permanently.sh | 2 +- .../0_stateless/01848_http_insert_segfault.sh | 2 +- 18 files changed, 307 insertions(+), 51 deletions(-) create mode 100644 tests/integration/test_wrong_db_or_table_name/__init__.py create mode 100644 tests/integration/test_wrong_db_or_table_name/test.py diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 4a087548a72f..79e78529d955 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -106,6 +106,11 @@ class IHints return prompter.getHints(name, getAllRegisteredNames()); } + std::vector getHints(const String & name, const std::vector & prompting_strings) const + { + return prompter.getHints(name, prompting_strings); + } + void appendHintsMessage(String & error_message, const String & name) const { auto hints = getHints(name); diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 9e33548b0dd3..7d12ae6c588a 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -1,7 +1,10 @@ +#include #include #include #include #include +#include +#include namespace DB @@ -18,7 +21,13 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const { if (auto storage = tryGetTable(name, context)) return storage; - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); + TableNameHints hints(this->shared_from_this(), context); + std::vector names = hints.getHints(name); + if (!names.empty()) + { + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist. Maybe you meant {}?", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name), backQuoteIfNeed(names[0])); + } + else throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); } std::vector> IDatabase::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 9bed3c4bfc58..01d940b04297 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -372,6 +372,7 @@ class IDatabase : public std::enable_shared_from_this }; using DatabasePtr = std::shared_ptr; +using ConstDatabasePtr = std::shared_ptr; using Databases = std::map; } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f8481e3f1d88..3baaa182d516 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -23,6 +25,7 @@ #include #include +#include "Interpreters/Context_fwd.h" #include "config.h" #if USE_MYSQL @@ -35,7 +38,6 @@ # include #endif - namespace CurrentMetrics { extern const Metric TablesToDropQueueSize; @@ -59,6 +61,29 @@ namespace ErrorCodes extern const int UNFINISHED; } +class DatabaseNameHints : public IHints<1, DatabaseNameHints> +{ +public: + explicit DatabaseNameHints(const DatabaseCatalog & database_catalog_) + : database_catalog(database_catalog_) + { + } + Names getAllRegisteredNames() const override + { + Names result; + auto databases_list = database_catalog.getDatabases(); + for (const auto & database_name : databases_list | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; + result.emplace_back(database_name); + } + return result; + } +private: + const DatabaseCatalog & database_catalog; +}; + TemporaryTableHolder::TemporaryTableHolder(ContextPtr context_, const TemporaryTableHolder::Creator & creator, const ASTPtr & query) : WithContext(context_->getGlobalContext()) , temporary_tables(DatabaseCatalog::instance().getDatabaseForTemporaryTables().get()) @@ -313,7 +338,18 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( { assert(!db_and_table.first && !db_and_table.second); if (exception) - exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs())); + { + TableNameHints hints(this->tryGetDatabase(table_id.getDatabaseName()), getContext()); + std::vector names = hints.getHints(table_id.getTableName()); + if (!names.empty()) + { + /// There is two options: first is to print just the name of the table + /// and the second is to print the result in format: db_name.table_name. I'll comment out the second option below + /// I also leave possibility to print several suggestions + exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist. Maybe you meant {}?", table_id.getNameForLogs(), backQuoteIfNeed(names[0]))); + } + else exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_id.getNameForLogs())); + } return {}; } @@ -359,13 +395,26 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( std::lock_guard lock{databases_mutex}; auto it = databases.find(table_id.getDatabaseName()); - if (databases.end() == it) + if (databases.end() != it) + database = it->second; + } + + if (!database) + { + if (exception) { - if (exception) - exception->emplace(Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exist", backQuoteIfNeed(table_id.getDatabaseName()))); - return {}; + DatabaseNameHints hints(*this); + std::vector names = hints.getHints(table_id.getDatabaseName()); + if (names.empty()) + { + exception->emplace(Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(table_id.getDatabaseName()))); + } + else + { + exception->emplace(Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist. Maybe you meant {}?", backQuoteIfNeed(table_id.getDatabaseName()), backQuoteIfNeed(names[0]))); + } } - database = it->second; + return {}; } StoragePtr table; @@ -386,8 +435,18 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( } if (!table && exception && !exception->has_value()) - exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs())); - + { + TableNameHints hints(this->tryGetDatabase(table_id.getDatabaseName()), getContext()); + std::vector names = hints.getHints(table_id.getTableName()); + if (names.empty()) + { + exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_id.getNameForLogs())); + } + else + { + exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist. Maybe you meant {}?", table_id.getNameForLogs(), backQuoteIfNeed(names[0]))); + } + } if (!table) database = nullptr; @@ -438,8 +497,26 @@ bool DatabaseCatalog::isPredefinedTable(const StorageID & table_id) const void DatabaseCatalog::assertDatabaseExists(const String & database_name) const { - std::lock_guard lock{databases_mutex}; - assertDatabaseExistsUnlocked(database_name); + DatabasePtr db; + { + std::lock_guard lock{databases_mutex}; + assert(!database_name.empty()); + if (auto it = databases.find(database_name); it != databases.end()) + db = it->second; + } + if (!db) + { + DatabaseNameHints hints(*this); + std::vector names = hints.getHints(database_name); + if (names.empty()) + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(database_name)); + } + else + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist. Maybe you meant {}?", backQuoteIfNeed(database_name), backQuoteIfNeed(names[0])); + } + } } void DatabaseCatalog::assertDatabaseDoesntExist(const String & database_name) const @@ -448,19 +525,11 @@ void DatabaseCatalog::assertDatabaseDoesntExist(const String & database_name) co assertDatabaseDoesntExistUnlocked(database_name); } -void DatabaseCatalog::assertDatabaseExistsUnlocked(const String & database_name) const -{ - assert(!database_name.empty()); - if (databases.end() == databases.find(database_name)) - throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exist", backQuoteIfNeed(database_name)); -} - - void DatabaseCatalog::assertDatabaseDoesntExistUnlocked(const String & database_name) const { assert(!database_name.empty()); if (databases.end() != databases.find(database_name)) - throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists.", backQuoteIfNeed(database_name)); + throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists", backQuoteIfNeed(database_name)); } void DatabaseCatalog::attachDatabase(const String & database_name, const DatabasePtr & database) @@ -480,18 +549,34 @@ DatabasePtr DatabaseCatalog::detachDatabase(ContextPtr local_context, const Stri { if (database_name == TEMPORARY_DATABASE) throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "Cannot detach database with temporary tables."); - + assert(!database_name.empty()); DatabasePtr db; { std::lock_guard lock{databases_mutex}; - assertDatabaseExistsUnlocked(database_name); - db = databases.find(database_name)->second; - UUID db_uuid = db->getUUID(); - if (db_uuid != UUIDHelpers::Nil) - removeUUIDMapping(db_uuid); - databases.erase(database_name); - } + if (auto it = databases.find(database_name); it != databases.end()) + { + db = it->second; + + UUID db_uuid = db->getUUID(); + if (db_uuid != UUIDHelpers::Nil) + removeUUIDMapping(db_uuid); + databases.erase(database_name); + } + } + if (!db) + { + DatabaseNameHints hints(*this); + std::vector names = hints.getHints(database_name); + if (names.empty()) + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(database_name)); + } + else + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist. Maybe you meant {}?", backQuoteIfNeed(database_name), backQuoteIfNeed(names[0])); + } + } if (check_empty) { try @@ -527,7 +612,6 @@ DatabasePtr DatabaseCatalog::detachDatabase(ContextPtr local_context, const Stri if (db_uuid != UUIDHelpers::Nil) removeUUIDMappingFinally(db_uuid); } - return db; } @@ -553,9 +637,28 @@ void DatabaseCatalog::updateDatabaseName(const String & old_name, const String & DatabasePtr DatabaseCatalog::getDatabase(const String & database_name) const { - std::lock_guard lock{databases_mutex}; - assertDatabaseExistsUnlocked(database_name); - return databases.find(database_name)->second; + assert(!database_name.empty()); + DatabasePtr db; + { + std::lock_guard lock{databases_mutex}; + if (auto it = databases.find(database_name); it != databases.end()) + db = it->second; + } + + if (!db) + { + DatabaseNameHints hints(*this); + std::vector names = hints.getHints(database_name); + if (names.empty()) + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(database_name)); + } + else + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist. Maybe you meant {}?", backQuoteIfNeed(database_name), backQuoteIfNeed(names[0])); + } + } + return db; } DatabasePtr DatabaseCatalog::tryGetDatabase(const String & database_name) const diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 805d7786569e..d734e6bc149b 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -6,7 +6,10 @@ #include #include #include +#include "Common/NamePrompter.h" #include +#include "Storages/IStorage.h" +#include "Databases/IDatabase.h" #include #include @@ -27,6 +30,32 @@ namespace fs = std::filesystem; namespace DB { +class TableNameHints : public IHints<1, TableNameHints> +{ +public: + TableNameHints(ConstDatabasePtr database_, ContextPtr context_) + : context(context_), + database(database_) + { + } + Names getAllRegisteredNames() const override + { + Names result; + if (database) + { + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) + { + const auto & storage_id = table_it->table()->getStorageID(); + result.emplace_back(storage_id.getTableName()); + } + } + return result; + } +private: + ContextPtr context; + ConstDatabasePtr database; +}; + class IDatabase; class Exception; class ColumnsDescription; @@ -262,7 +291,6 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext static std::unique_ptr database_catalog; explicit DatabaseCatalog(ContextMutablePtr global_context_); - void assertDatabaseExistsUnlocked(const String & database_name) const TSA_REQUIRES(databases_mutex); void assertDatabaseDoesntExistUnlocked(const String & database_name) const TSA_REQUIRES(databases_mutex); void shutdownImpl(); diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index b9cca4bf6771..68822fbf311d 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -139,6 +139,7 @@ test_quota/test.py::test_tracking_quota test_quota/test.py::test_users_xml_is_readonly test_replicating_constants/test.py::test_different_versions test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] +test_wrong_db_or_table_name/test.py::test_wrong_table_name test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table] test_odbc_interaction/test.py::test_postgres_insert test_zookeeper_config/test.py::test_chroot_with_different_root diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index a973b697d0d9..99d08f1b6ea3 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -281,7 +281,7 @@ def test_reload_after_fail_in_cache_dictionary(started_cluster): query_and_get_error = instance.query_and_get_error # Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond. - expected_error = "Table test.xypairs doesn't exist" + expected_error = "Table test.xypairs does not exist" update_error = "Could not update cache dictionary cache_xypairs now" assert expected_error in query_and_get_error( "SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))" diff --git a/tests/integration/test_drop_is_lock_free/test.py b/tests/integration/test_drop_is_lock_free/test.py index 9f595800beab..61d52a1d9b11 100644 --- a/tests/integration/test_drop_is_lock_free/test.py +++ b/tests/integration/test_drop_is_lock_free/test.py @@ -125,7 +125,7 @@ def test_query_is_lock_free(lock_free_query, exclusive_table): SELECT count() FROM {exclusive_table}; """ ) - assert f"Table default.{exclusive_table} doesn't exist" in result + assert f"Table default.{exclusive_table} does not exist" in result else: assert 0 == int( node.query( diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index 0941e6649827..e0928c6ab084 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -159,11 +159,11 @@ def test_drop_replica(start_cluster): for i in range(1, 5): node_1_1.query("DETACH DATABASE test{}".format(i)) - assert "doesn't exist" in node_1_3.query_and_get_error( + assert "does not exist" in node_1_3.query_and_get_error( "SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table" ) - assert "doesn't exist" in node_1_3.query_and_get_error( + assert "does not exist" in node_1_3.query_and_get_error( "SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1" ) diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index 3bee21493872..09a14c117277 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -246,7 +246,7 @@ def test_inserts_local(started_cluster): def test_inserts_single_replica_local_internal_replication(started_cluster): with pytest.raises( - QueryRuntimeException, match="Table default.single_replicated doesn't exist" + QueryRuntimeException, match="Table default.single_replicated does not exist" ): node1.query( "INSERT INTO distributed_one_replica_internal_replication VALUES ('2000-01-01', 1)", @@ -279,7 +279,8 @@ def test_inserts_single_replica_internal_replication(started_cluster): def test_inserts_single_replica_no_internal_replication(started_cluster): try: with pytest.raises( - QueryRuntimeException, match="Table default.single_replicated doesn't exist" + QueryRuntimeException, + match="Table default.single_replicated does not exist", ): node1.query( "INSERT INTO distributed_one_replica_no_internal_replication VALUES ('2000-01-01', 1)", diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 494af3c219d9..f2bffe694952 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -207,7 +207,7 @@ def test_mysql_client(started_cluster): expected_msg = "\n".join( [ "mysql: [Warning] Using a password on the command line interface can be insecure.", - "ERROR 81 (00000) at line 1: Code: 81. DB::Exception: Database system2 doesn't exist", + "ERROR 81 (00000) at line 1: Code: 81. DB::Exception: Database system2 does not exist", ] ) assert stderr[: len(expected_msg)].decode() == expected_msg @@ -621,7 +621,7 @@ def test_python_client(started_cluster): client.select_db("system2") assert exc_info.value.args[1].startswith( - "Code: 81. DB::Exception: Database system2 doesn't exist" + "Code: 81. DB::Exception: Database system2 does not exist" ), exc_info.value.args[1] cursor = client.cursor(pymysql.cursors.DictCursor) @@ -646,7 +646,7 @@ def test_golang_client(started_cluster, golang_container): ) assert code == 1 - assert stderr.decode() == "Error 81: Database abc doesn't exist\n" + assert stderr.decode() == "Error 81: Database abc does not exist\n" code, (stdout, stderr) = golang_container.exec_run( "./main --host {host} --port {port} --user default --password 123 --database " diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 5bf3eff8b2d5..084d342d7367 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -47,7 +47,7 @@ def test_system_logs(flush_logs, table, exists): else: response = node.query_and_get_error(q) assert ( - "Table {} doesn't exist".format(table) in response + "Table {} does not exist".format(table) in response or "Unknown table expression identifier '{}'".format(table) in response ) diff --git a/tests/integration/test_wrong_db_or_table_name/__init__.py b/tests/integration/test_wrong_db_or_table_name/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_wrong_db_or_table_name/test.py b/tests/integration/test_wrong_db_or_table_name/test.py new file mode 100644 index 000000000000..68af383b6c34 --- /dev/null +++ b/tests/integration/test_wrong_db_or_table_name/test.py @@ -0,0 +1,108 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node") + + +@pytest.fixture(scope="module") +def start(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_wrong_database_name(start): + node.query( + """ + CREATE DATABASE test; + CREATE TABLE test.table_test (i Int64) ENGINE=Memory; + INSERT INTO test.table_test SELECT 1; + """ + ) + + with pytest.raises( + QueryRuntimeException, + match="DB::Exception: Database tes does not exist. Maybe you meant test?.", + ): + node.query("SELECT * FROM tes.table_test LIMIT 1;") + assert int(node.query("SELECT count() FROM test.table_test;")) == 1 + node.query( + """ + DROP TABLE test.table_test; + DROP DATABASE test; + """ + ) + + +def test_drop_wrong_database_name(start): + node.query( + """ + CREATE DATABASE test; + CREATE TABLE test.table_test (i Int64) ENGINE=Memory; + INSERT INTO test.table_test SELECT 1; + """ + ) + + with pytest.raises( + QueryRuntimeException, + match="DB::Exception: Database tes does not exist. Maybe you meant test?.", + ): + node.query("DROP DATABASE tes;") + assert int(node.query("SELECT count() FROM test.table_test;")) == 1 + node.query("DROP DATABASE test;") + + +def test_wrong_table_name(start): + node.query( + """ + CREATE DATABASE test; + CREATE TABLE test.table_test (i Int64) ENGINE=Memory; + CREATE TABLE test.table_test2 (i Int64) ENGINE=Memory; + INSERT INTO test.table_test SELECT 1; + """ + ) + with pytest.raises( + QueryRuntimeException, + match="DB::Exception: Table test.table_test1 does not exist. Maybe you meant table_test?.", + ): + node.query( + """ + SELECT * FROM test.table_test1 LIMIT 1; + """ + ) + assert int(node.query("SELECT count() FROM test.table_test;")) == 1 + node.query( + """ + DROP TABLE test.table_test; + DROP TABLE test.table_test2; + DROP DATABASE test; + """ + ) + + +def test_drop_wrong_table_name(start): + node.query( + """ + CREATE DATABASE test; + CREATE TABLE test.table_test (i Int64) ENGINE=Memory; + INSERT INTO test.table_test SELECT 1; + """ + ) + + with pytest.raises( + QueryRuntimeException, + match="DB::Exception: Table test.table_tes does not exist. Maybe you meant table_test?.", + ): + node.query("DROP TABLE test.table_tes;") + assert int(node.query("SELECT count() FROM test.table_test;")) == 1 + node.query( + """ + DROP TABLE test.table_test; + DROP DATABASE test; + """ + ) 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 86fe01dc0e3f..a993c330d2af 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 @@ -32,8 +32,8 @@ create temporary table known_short_messages (s String) as select * from (select 'brotli decode error{}', 'Invalid H3 index: {}', 'Too large node state size', 'No additional keys found.', 'Attempt to read after EOF.', 'Replication was stopped', '{} building file infos', 'Cannot parse uuid {}', 'Query was cancelled', 'Cancelled merging parts', 'Cancelled mutating parts', 'Log pulling is cancelled', -'Transaction was cancelled', 'Could not find table: {}', 'Table {} doesn''t exist', -'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', +'Transaction was cancelled', 'Could not find table: {}', 'Table {} does not exist', +'Database {} does not 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: {}' @@ -48,7 +48,7 @@ select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_st -- Unlike above, here we look at length of the formatted message, not format string. Most short format strings are fine because they end up decorated with context from outer or inner exceptions, e.g.: -- "Expected end of line" -> "Code: 117. DB::Exception: Expected end of line: (in file/uri /var/lib/clickhouse/user_files/data_02118): (at row 1)" -- But we have to cut out the boilerplate, e.g.: --- "Code: 60. DB::Exception: Table default.a doesn't exist. (UNKNOWN_TABLE), Stack trace" -> "Table default.a doesn't exist." +-- "Code: 60. DB::Exception: Table default.a does not exist. (UNKNOWN_TABLE), Stack trace" -> "Table default.a does not exist." -- This table currently doesn't have enough information to do this reliably, so we just regex search for " (ERROR_NAME_IN_CAPS)" and hope that's good enough. -- For the "Code: 123. DB::Exception: " part, we just subtract 26 instead of searching for it. Because sometimes it's not at the start, e.g.: -- "Unexpected error, will try to restart main thread: Code: 341. DB::Exception: Unexpected error: Code: 57. DB::Exception:[...]" diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index 6dd7ff3cdc85..d1a7144e8860 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15000000 -q "INSE sleep 1 $CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192 TO default" 2>&1| grep -F "already exists" > /dev/null && echo "ok" -$CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192_notexisting TO test_01192_renamed" 2>&1| grep -F "doesn't exist" > /dev/null && echo "ok" +$CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192_notexisting TO test_01192_renamed" 2>&1| grep -F "does not exist" > /dev/null && echo "ok" $CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192 TO test_01192_renamed" && echo "renamed" wait @@ -50,7 +50,7 @@ $CLICKHOUSE_CLIENT -q "RENAME TABLE test_01192.mt TO test_01192_atomic.mt, test_ # 6. check data after RENAME $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01192_atomic.mt" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01192_atomic.rmt" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01192_atomic.mv" 2>&1| grep -F "doesn't exist" > /dev/null && echo "ok" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01192_atomic.mv" 2>&1| grep -F "does not exist" > /dev/null && echo "ok" # 7. create dictionary and check it $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.mt (n UInt64, _part String) ENGINE=Memory" # mock diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index c32a255448e0..4b270265cdce 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -108,7 +108,7 @@ clickhouse_local "INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" clickhouse_local "SELECT if(count() = 10, 'MV is working', 'MV failed') FROM db_ordinary.src_mv_with_inner" clickhouse_local "DETACH VIEW db_ordinary.src_mv_with_inner PERMANENTLY; INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" --stacktrace -clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner" 2>&1 | grep -c "db_ordinary.src_mv_with_inner doesn't exist" +clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner" 2>&1 | grep -c "db_ordinary.src_mv_with_inner does not exist" ## Quite silly: ATTACH MATERIALIZED VIEW don't work with short syntax (w/o select), but i can attach it using ATTACH TABLE ... clickhouse_local "ATTACH TABLE db_ordinary.src_mv_with_inner" clickhouse_local "INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" diff --git a/tests/queries/0_stateless/01848_http_insert_segfault.sh b/tests/queries/0_stateless/01848_http_insert_segfault.sh index 1f2e9eebcdce..6397068a77ab 100755 --- a/tests/queries/0_stateless/01848_http_insert_segfault.sh +++ b/tests/queries/0_stateless/01848_http_insert_segfault.sh @@ -6,4 +6,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_LOCAL} --query "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | ${CLICKHOUSE_CURL} -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||: +${CLICKHOUSE_LOCAL} --query "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | ${CLICKHOUSE_CURL} -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table does not exist" && echo 'Ok.' || echo 'FAIL' ||: